From 04151c24fc7b22fc04e9898c28b6473219bed9bb Mon Sep 17 00:00:00 2001 From: Yingjun Wu Date: Mon, 23 Oct 2023 23:37:45 -0700 Subject: [PATCH 01/48] chore: beautify readme. (#13019) Co-authored-by: hengm3467 <100685635+hengm3467@users.noreply.github.com> --- README.md | 39 ++++++++++++++++++++++++++------------- 1 file changed, 26 insertions(+), 13 deletions(-) diff --git a/README.md b/README.md index 05f8ed8e3fb75..29a7d7e51888a 100644 --- a/README.md +++ b/README.md @@ -80,22 +80,35 @@ Learn more at [Quick Start](https://docs.risingwave.com/docs/current/get-started ## Why RisingWave for stream processing? RisingWave adaptly tackles some of the most challenging problems in stream processing. Compared to existing stream processing systems, RisingWave shines through with the following key features: -* **Easy to learn:** RisingWave speaks PostgreSQL-style SQL, enabling users to dive into stream processing in much the same way as operating a PostgreSQL database. -* **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. -* **High resource utilization:** Queries within RisingWave benefit from shared computational resources, obviating the need for users to manually allocate resources for individual queries. -* **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. -* **Transparent dynamic scaling:** RisingWave supports 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. -* **Easy 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. -* **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 link RisingWave to their preferred BI tools or through client libraries. -* **Simple to maintain and operate:** RisingWave abstracts away unnecessary low-level details, allowing users to concentrate solely on SQL code-level issues. -* **Rich ecosystem:** With integrations to a diverse range of cloud systems and the PostgreSQL ecosystem, RisingWave boasts a rich and expansive ecosystem. +* **Easy to learn** + * RisingWave speaks PostgreSQL-style SQL, enabling users to dive into stream processing in much the same way as operating a PostgreSQL database. +* **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. +* **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. +* **Transparent dynamic scaling** + * RisingWave supports 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. +* **Easy 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. +* **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. +* **Simple to maintain and operate** + * RisingWave abstracts away unnecessary low-level details, allowing users to concentrate solely on SQL code-level issues. +* **Rich ecosystem** + * With integrations to a diverse range of cloud systems and the PostgreSQL ecosystem, RisingWave boasts a rich and expansive ecosystem. ## RisingWave's limitations RisingWave isn’t a panacea for all data engineering hurdles. It has its own set of limitations: -* **No programmable interfaces:** RisingWave does not provide low-level APIs in languages like Java and Scala, and does not allow users to manage internal states manually (unless you want to hack!). For coding in Java, Scala, and other languages, please consider using RisingWave's User-Defined Functions (UDF). -* **No support for transaction processing:** RisingWave isn’t cut out for transactional workloads, thus it’s not a viable substitute for operational databases dedicated to transaction processing. However, it supports read-only transactions, ensuring data freshness and consistency. It also comprehends the transactional semantics of upstream database Change Data Capture (CDC). -* **Not tailored for ad-hoc analytical queries:** RisingWave's row store design is tailored for optimal stream processing performance rather than interactive analytical workloads. Hence, it's not a suitable replacement for OLAP databases. Yet, a reliable integration with many OLAP databases exists, and a collaborative use of RisingWave and OLAP databases is a common practice among many users. +* **No programmable interfaces** + * RisingWave does not provide low-level APIs in languages like Java and Scala, and does not allow users to manage internal states manually (unless you want to hack!). For coding in Java, Scala, and other languages, please consider using RisingWave's User-Defined Functions (UDF). +* **No support for transaction processing** + * RisingWave isn’t cut out for transactional workloads, thus it’s not a viable substitute for operational databases dedicated to transaction processing. However, it supports read-only transactions, ensuring data freshness and consistency. It also comprehends the transactional semantics of upstream database Change Data Capture (CDC). +* **Not tailored for ad-hoc analytical queries** + * RisingWave's row store design is tailored for optimal stream processing performance rather than interactive analytical workloads. Hence, it's not a suitable replacement for OLAP databases. Yet, a reliable integration with many OLAP databases exists, and a collaborative use of RisingWave and OLAP databases is a common practice among many users. ## RisingWave Cloud From 6939da61d9669f19d6b1193cc5eb53780b41a4b9 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 24 Oct 2023 15:21:09 +0800 Subject: [PATCH 02/48] fix(ci): upload microbench only when it runs (#12988) --- ci/scripts/run-micro-benchmarks.sh | 2 ++ ci/scripts/upload-micro-bench-results.sh | 13 +++++++++++++ 2 files changed, 15 insertions(+) diff --git a/ci/scripts/run-micro-benchmarks.sh b/ci/scripts/run-micro-benchmarks.sh index 568c90de425ca..371cc416e7ac5 100755 --- a/ci/scripts/run-micro-benchmarks.sh +++ b/ci/scripts/run-micro-benchmarks.sh @@ -46,6 +46,8 @@ main() { echo "--- Getting aws instance type" local instance_type=$(get_instance_type) echo "instance_type: $instance_type" + echo "$instance_type" > microbench_instance_type.txt + buildkite-agent artifact upload ./microbench_instance_type.txt if [[ $instance_type != "m6i.4xlarge" ]]; then echo "Only m6i.4xlarge is supported, skipping microbenchmark" exit 0 diff --git a/ci/scripts/upload-micro-bench-results.sh b/ci/scripts/upload-micro-bench-results.sh index 2644ca936c5da..e72b69950bb7b 100755 --- a/ci/scripts/upload-micro-bench-results.sh +++ b/ci/scripts/upload-micro-bench-results.sh @@ -36,6 +36,19 @@ get_commit() { | sed 's/\"//g' } +get_machine() { + buildkite-agent artifact download microbench_instance_type.txt ./ + cat ./microbench_instance_type.txt +} + +echo "--- Checking microbench_instance_type" +INSTANCE_TYPE=$(get_machine) +echo "instance type: $INSTANCE_TYPE" +if [[ $INSTANCE_TYPE != "m6i.4xlarge" ]]; then + echo "Only m6i.4xlarge is supported, microbenchmark was skipped" + exit 0 +fi + setup BUILDKITE_BUILD_URL="https://buildkite.com/risingwavelabs/main-cron/builds/$BUILDKITE_BUILD_NUMBER" From 23b33460e2df0d4efa22c1574bede5dcf82cf16a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 24 Oct 2023 16:36:40 +0800 Subject: [PATCH 03/48] chore(deps): Bump comfy-table from 7.0.1 to 7.1.0 (#13013) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b7e9b6c45ec06..bbc112fa4bb5b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1825,13 +1825,13 @@ dependencies = [ [[package]] name = "comfy-table" -version = "7.0.1" +version = "7.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ab77dbd8adecaf3f0db40581631b995f312a8a5ae3aa9993188bb8f23d83a5b" +checksum = "7c64043d6c7b7a4c58e39e7efccfdea7b93d885a795d0c054a69dbbf4dd52686" dependencies = [ - "crossterm 0.26.1", - "strum 0.24.1", - "strum_macros 0.24.3", + "crossterm 0.27.0", + "strum 0.25.0", + "strum_macros 0.25.2", "unicode-width", ] @@ -2140,17 +2140,14 @@ dependencies = [ [[package]] name = "crossterm" -version = "0.26.1" +version = "0.27.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a84cda67535339806297f1b331d6dd6320470d2a0fe65381e79ee9e156dd3d13" +checksum = "f476fe445d41c9e991fd07515a6f463074b782242ccf4a5b7b1d1012e70824df" dependencies = [ - "bitflags 1.3.2", + "bitflags 2.4.0", "crossterm_winapi", "libc", - "mio", "parking_lot 0.12.1", - "signal-hook", - "signal-hook-mio", "winapi", ] From fcad5e1f127c13229acb7e1ef3c4fcbf8b3e69e6 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 24 Oct 2023 16:57:40 +0800 Subject: [PATCH 04/48] fix(stream): clean dirty tables for barrier recovery (#12990) --- ci/scripts/deterministic-recovery-test.sh | 1 + src/meta/src/barrier/mod.rs | 7 ++-- src/meta/src/barrier/recovery.rs | 9 ++---- src/meta/src/manager/catalog/mod.rs | 32 +++++++++++-------- .../recovery/background_ddl.rs | 3 ++ 5 files changed, 27 insertions(+), 25 deletions(-) diff --git a/ci/scripts/deterministic-recovery-test.sh b/ci/scripts/deterministic-recovery-test.sh index 6514fe1f7c0c3..c5f89a2bbc7e0 100755 --- a/ci/scripts/deterministic-recovery-test.sh +++ b/ci/scripts/deterministic-recovery-test.sh @@ -11,6 +11,7 @@ chmod +x ./risingwave_simulation export RUST_LOG="info,\ risingwave_meta::barrier::recovery=debug,\ +risingwave_meta::manager::catalog=debug,\ risingwave_meta::rpc::ddl_controller=debug,\ risingwave_meta::barrier::mod=debug,\ risingwave_simulation=debug" diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index ed6ad289a5a68..d39dde51399d8 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -626,7 +626,7 @@ impl GlobalBarrierManager { let paused = self.take_pause_on_bootstrap().await.unwrap_or(false); let paused_reason = paused.then_some(PausedReason::Manual); - self.recovery(prev_epoch, paused_reason, true) + self.recovery(prev_epoch, paused_reason) .instrument(span) .await }; @@ -981,10 +981,7 @@ impl GlobalBarrierManager { // No need to clean dirty tables for barrier recovery, // The foreground stream job should cleanup their own tables. - *state = self - .recovery(prev_epoch, None, false) - .instrument(span) - .await; + *state = self.recovery(prev_epoch, None).instrument(span).await; self.set_status(BarrierManagerStatus::Running).await; } else { panic!("failed to execute barrier: {:?}", err); diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index 21197a8df98d4..3e319f0e69a52 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -219,7 +219,6 @@ impl GlobalBarrierManager { &self, prev_epoch: TracedEpoch, paused_reason: Option, - bootstrap_recovery: bool, ) -> BarrierManagerState { // Mark blocked and abort buffered schedules, they might be dirty already. self.scheduled_barriers @@ -227,11 +226,9 @@ impl GlobalBarrierManager { .await; tracing::info!("recovery start!"); - if bootstrap_recovery { - self.clean_dirty_tables() - .await - .expect("clean dirty tables should not fail"); - } + self.clean_dirty_tables() + .await + .expect("clean dirty tables should not fail"); self.clean_dirty_fragments() .await .expect("clean dirty fragments"); diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index bcac32922d180..f988646428aac 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -853,14 +853,18 @@ impl CatalogManager { database_core.clear_creating_stream_jobs(); let user_core = &mut core.user; for table in &tables_to_clean { - // Recovered when init database manager. - for relation_id in &table.dependent_relations { - database_core.decrease_ref_count(*relation_id); + // If table type is internal, no need to update the ref count OR + // user ref count. + if table.table_type != TableType::Internal as i32 { + // Recovered when init database manager. + for relation_id in &table.dependent_relations { + database_core.decrease_ref_count(*relation_id); + } + // Recovered when init user manager. + tracing::debug!("decrease ref for {}", table.id); + user_core.decrease_ref(table.owner); } - // Recovered when init user manager. - user_core.decrease_ref(table.owner); } - Ok(()) } @@ -919,10 +923,11 @@ impl CatalogManager { let database_core = &mut core.database; let tables = &mut database_core.tables; let Some(table) = tables.get(&table_id).cloned() else { - bail!( - "table_id {} missing when attempting to cancel job", + tracing::warn!( + "table_id {} missing when attempting to cancel job, could be cleaned on recovery", table_id - ) + ); + return Ok(()); }; table }; @@ -938,7 +943,8 @@ impl CatalogManager { let tables = &mut database_core.tables; let mut tables = BTreeMapTransaction::new(tables); for table_id in table_ids { - tables.remove(table_id); + let res = tables.remove(table_id); + assert!(res.is_some()); } commit_meta!(self, tables)?; } @@ -2032,8 +2038,7 @@ impl CatalogManager { let user_core = &mut core.user; let key = (index.database_id, index.schema_id, index.name.clone()); assert!( - !database_core.indexes.contains_key(&index.id) - && database_core.has_in_progress_creation(&key), + !database_core.indexes.contains_key(&index.id), "index must be in creating procedure" ); @@ -2188,8 +2193,7 @@ impl CatalogManager { let user_core = &mut core.user; let key = (sink.database_id, sink.schema_id, sink.name.clone()); assert!( - !database_core.sinks.contains_key(&sink.id) - && database_core.has_in_progress_creation(&key), + !database_core.sinks.contains_key(&sink.id), "sink must be in creating procedure" ); diff --git a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs index 1fd5c90e59e4b..89df82d4c21a0 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs @@ -61,6 +61,9 @@ async fn test_background_mv_barrier_recovery() -> Result<()> { .run("create materialized view m1 as select * from t1;") .await?; + // If the CN is killed before first barrier pass for the MV, the MV will be dropped. + // This is because it's table fragments will NOT be committed until first barrier pass. + sleep(Duration::from_secs(5)).await; kill_cn_and_wait_recover(&cluster).await; // Send some upstream updates. From 88a8331028674fa98b4584751bc7937c18e26ab3 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 24 Oct 2023 17:09:47 +0800 Subject: [PATCH 05/48] refactor(expr): type-safe infallible evaluation (#12921) Signed-off-by: Bugen Zhao --- src/batch/src/executor/aggregation/filter.rs | 2 +- src/batch/src/executor/project_set.rs | 2 +- src/expr/core/src/expr/build.rs | 21 ++-- src/expr/core/src/expr/mod.rs | 106 ++++++++++++------ src/expr/core/src/expr/wrapper/checked.rs | 2 +- src/expr/core/src/expr/wrapper/mod.rs | 7 +- src/expr/core/src/expr/wrapper/non_strict.rs | 14 ++- src/expr/impl/src/scalar/cast.rs | 4 +- .../src/table_function/generate_series.rs | 2 +- src/storage/src/row_serde/value_serde.rs | 3 +- src/stream/clippy.toml | 4 +- src/stream/src/executor/aggregation/mod.rs | 8 +- src/stream/src/executor/dynamic_filter.rs | 6 +- src/stream/src/executor/filter.rs | 8 +- src/stream/src/executor/hash_join.rs | 20 ++-- src/stream/src/executor/hop_window.rs | 21 ++-- src/stream/src/executor/integration_tests.rs | 2 +- src/stream/src/executor/mod.rs | 6 +- src/stream/src/executor/project.rs | 11 +- src/stream/src/executor/project_set.rs | 7 +- src/stream/src/executor/temporal_join.rs | 6 +- src/stream/src/executor/test_utils.rs | 10 +- src/stream/src/executor/values.rs | 17 +-- src/stream/src/executor/watermark_filter.rs | 11 +- src/stream/src/from_proto/hash_join.rs | 9 +- src/stream/src/from_proto/temporal_join.rs | 4 +- .../tests/integration_tests/hop_window.rs | 11 +- .../tests/integration_tests/project_set.rs | 8 +- 28 files changed, 212 insertions(+), 120 deletions(-) diff --git a/src/batch/src/executor/aggregation/filter.rs b/src/batch/src/executor/aggregation/filter.rs index 2db2320ed3534..9cfbeabffe417 100644 --- a/src/batch/src/executor/aggregation/filter.rs +++ b/src/batch/src/executor/aggregation/filter.rs @@ -75,7 +75,7 @@ impl AggregateFunction for Filter { mod tests { use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_expr::aggregate::{build_append_only, AggCall}; - use risingwave_expr::expr::{build_from_pretty, Expression, LiteralExpression}; + use risingwave_expr::expr::{build_from_pretty, ExpressionBoxExt, LiteralExpression}; use super::*; diff --git a/src/batch/src/executor/project_set.rs b/src/batch/src/executor/project_set.rs index 670933a6bb50c..fa3dfac917e8a 100644 --- a/src/batch/src/executor/project_set.rs +++ b/src/batch/src/executor/project_set.rs @@ -171,7 +171,7 @@ mod tests { use risingwave_common::catalog::{Field, Schema}; use risingwave_common::test_prelude::*; use risingwave_common::types::DataType; - use risingwave_expr::expr::{Expression, InputRefExpression, LiteralExpression}; + use risingwave_expr::expr::{ExpressionBoxExt, InputRefExpression, LiteralExpression}; use risingwave_expr::table_function::repeat; use super::*; diff --git a/src/expr/core/src/expr/build.rs b/src/expr/core/src/expr/build.rs index 1ea03bd36f42a..7dffbcd42d66b 100644 --- a/src/expr/core/src/expr/build.rs +++ b/src/expr/core/src/expr/build.rs @@ -27,8 +27,13 @@ use super::expr_in::InExpression; use super::expr_some_all::SomeAllExpression; use super::expr_udf::UdfExpression; use super::expr_vnode::VnodeExpression; -use super::wrapper::{Checked, EvalErrorReport, NonStrict}; -use crate::expr::{BoxedExpression, Expression, InputRefExpression, LiteralExpression}; +use super::wrapper::checked::Checked; +use super::wrapper::non_strict::NonStrict; +use super::wrapper::EvalErrorReport; +use super::NonStrictExpression; +use crate::expr::{ + BoxedExpression, Expression, ExpressionBoxExt, InputRefExpression, LiteralExpression, +}; use crate::sig::FUNCTION_REGISTRY; use crate::{bail, ExprError, Result}; @@ -41,8 +46,10 @@ pub fn build_from_prost(prost: &ExprNode) -> Result { pub fn build_non_strict_from_prost( prost: &ExprNode, error_report: impl EvalErrorReport + 'static, -) -> Result { - ExprBuilder::new_non_strict(error_report).build(prost) +) -> Result { + ExprBuilder::new_non_strict(error_report) + .build(prost) + .map(NonStrictExpression) } /// Build an expression from protobuf with possibly some wrappers attached to each node. @@ -153,7 +160,7 @@ impl BuildBoxed for E { prost: &ExprNode, build_child: impl Fn(&ExprNode) -> Result, ) -> Result { - Self::build(prost, build_child).map(Expression::boxed) + Self::build(prost, build_child).map(ExpressionBoxExt::boxed) } } @@ -217,9 +224,9 @@ pub fn build_func_non_strict( ret_type: DataType, children: Vec, error_report: impl EvalErrorReport + 'static, -) -> Result { +) -> Result { let expr = build_func(func, ret_type, children)?; - let wrapped = ExprBuilder::new_non_strict(error_report).wrap(expr); + let wrapped = NonStrictExpression(ExprBuilder::new_non_strict(error_report).wrap(expr)); Ok(wrapped) } diff --git a/src/expr/core/src/expr/mod.rs b/src/expr/core/src/expr/mod.rs index 37e0104371a3e..48a46f640bf7b 100644 --- a/src/expr/core/src/expr/mod.rs +++ b/src/expr/core/src/expr/mod.rs @@ -58,7 +58,7 @@ pub use self::build::*; pub use self::expr_input_ref::InputRefExpression; pub use self::expr_literal::LiteralExpression; pub use self::value::{ValueImpl, ValueRef}; -pub use self::wrapper::EvalErrorReport; +pub use self::wrapper::*; pub use super::{ExprError, Result}; /// Interface of an expression. @@ -67,6 +67,7 @@ pub use super::{ExprError, Result}; /// should be implemented. Prefer calling and implementing `eval_v2` instead of `eval` if possible, /// to gain the performance benefit of scalar expression. #[async_trait::async_trait] +#[auto_impl::auto_impl(&, Box)] pub trait Expression: std::fmt::Debug + Sync + Send { /// Get the return data type. fn return_type(&self) -> DataType; @@ -101,23 +102,77 @@ pub trait Expression: std::fmt::Debug + Sync + Send { fn eval_const(&self) -> Result { Err(ExprError::NotConstant) } +} +/// An owned dynamically typed [`Expression`]. +pub type BoxedExpression = Box; + +/// Extension trait for boxing expressions. +/// +/// This is not directly made into [`Expression`] trait because... +/// - an expression does not have to be `'static`, +/// - and for the ease of `auto_impl`. +#[easy_ext::ext(ExpressionBoxExt)] +impl E { /// Wrap the expression in a Box. - fn boxed(self) -> BoxedExpression - where - Self: Sized + Send + 'static, - { + pub fn boxed(self) -> BoxedExpression { Box::new(self) } } -// TODO: make this an extension, or implement it on a `NonStrict` newtype. -impl dyn Expression { +/// An type-safe wrapper that indicates the inner expression can be evaluated in a non-strict +/// manner, i.e., developers can directly call `eval_infallible` and `eval_row_infallible` without +/// checking the result. +/// +/// This is usually created by non-strict build functions like [`crate::expr::build_non_strict_from_prost`] +/// and [`crate::expr::build_func_non_strict`]. It can also be created directly by +/// [`NonStrictExpression::new_topmost`], where only the evaluation of the topmost level expression +/// node is non-strict and should be treated as a TODO. +/// +/// Compared to [`crate::expr::wrapper::non_strict::NonStrict`], this is more like an indicator +/// applied on the root of an expression tree, while the latter is a wrapper that can be applied on +/// each node of the tree and actually changes the behavior. As a result, [`NonStrictExpression`] +/// does not implement [`Expression`] trait and instead deals directly with developers. +#[derive(Debug)] +pub struct NonStrictExpression(E); + +impl NonStrictExpression +where + E: Expression, +{ + /// Create a non-strict expression directly wrapping the given expression. + /// + /// Should only be used in tests as evaluation may panic. + pub fn for_test(inner: E) -> NonStrictExpression + where + E: 'static, + { + NonStrictExpression(inner.boxed()) + } + + /// Create a non-strict expression from the given expression, where only the evaluation of the + /// topmost level expression node is non-strict (which is subtly different from + /// [`crate::expr::build_non_strict_from_prost`] where every node is non-strict). + /// + /// This should be used as a TODO. + pub fn new_topmost( + inner: E, + error_report: impl EvalErrorReport, + ) -> NonStrictExpression { + let inner = wrapper::non_strict::NonStrict::new(inner, error_report); + NonStrictExpression(inner) + } + + /// Get the return data type. + pub fn return_type(&self) -> DataType { + self.0.return_type() + } + /// Evaluate the expression in vectorized execution and assert it succeeds. Returns an array. /// /// Use with expressions built in non-strict mode. pub async fn eval_infallible(&self, input: &DataChunk) -> ArrayRef { - self.eval(input).await.expect("evaluation failed") + self.0.eval(input).await.expect("evaluation failed") } /// Evaluate the expression in row-based execution and assert it succeeds. Returns a nullable @@ -125,38 +180,17 @@ impl dyn Expression { /// /// Use with expressions built in non-strict mode. pub async fn eval_row_infallible(&self, input: &OwnedRow) -> Datum { - self.eval_row(input).await.expect("evaluation failed") - } -} - -/// An owned dynamically typed [`Expression`]. -pub type BoxedExpression = Box; - -// TODO: avoid the overhead of extra boxing. -#[async_trait::async_trait] -impl Expression for BoxedExpression { - fn return_type(&self) -> DataType { - (**self).return_type() + self.0.eval_row(input).await.expect("evaluation failed") } - async fn eval(&self, input: &DataChunk) -> Result { - (**self).eval(input).await - } - - async fn eval_v2(&self, input: &DataChunk) -> Result { - (**self).eval_v2(input).await - } - - async fn eval_row(&self, input: &OwnedRow) -> Result { - (**self).eval_row(input).await - } - - fn eval_const(&self) -> Result { - (**self).eval_const() + /// Unwrap the inner expression. + pub fn into_inner(self) -> E { + self.0 } - fn boxed(self) -> BoxedExpression { - self + /// Get a reference to the inner expression. + pub fn inner(&self) -> &E { + &self.0 } } diff --git a/src/expr/core/src/expr/wrapper/checked.rs b/src/expr/core/src/expr/wrapper/checked.rs index 1e049ad481010..b3b1375c4fa82 100644 --- a/src/expr/core/src/expr/wrapper/checked.rs +++ b/src/expr/core/src/expr/wrapper/checked.rs @@ -22,7 +22,7 @@ use crate::expr::{Expression, ValueImpl}; /// A wrapper of [`Expression`] that does extra checks after evaluation. #[derive(Debug)] -pub struct Checked(pub E); +pub(crate) struct Checked(pub E); // TODO: avoid the overhead of extra boxing. #[async_trait] diff --git a/src/expr/core/src/expr/wrapper/mod.rs b/src/expr/core/src/expr/wrapper/mod.rs index 48241d05de45c..16988a050ad8d 100644 --- a/src/expr/core/src/expr/wrapper/mod.rs +++ b/src/expr/core/src/expr/wrapper/mod.rs @@ -12,8 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod checked; -mod non_strict; +pub(crate) mod checked; +pub(crate) mod non_strict; -pub use checked::Checked; -pub use non_strict::{EvalErrorReport, NonStrict}; +pub use non_strict::{EvalErrorReport, LogReport}; diff --git a/src/expr/core/src/expr/wrapper/non_strict.rs b/src/expr/core/src/expr/wrapper/non_strict.rs index 0859cea27aa49..782456023cdf7 100644 --- a/src/expr/core/src/expr/wrapper/non_strict.rs +++ b/src/expr/core/src/expr/wrapper/non_strict.rs @@ -23,7 +23,7 @@ use crate::expr::{Expression, ValueImpl}; use crate::ExprError; /// Report an error during evaluation. -#[auto_impl(Arc)] +#[auto_impl(&, Arc)] pub trait EvalErrorReport: Clone + Send + Sync { /// Perform the error reporting. /// @@ -42,11 +42,21 @@ impl EvalErrorReport for ! { } } +/// Log the error to report an error during evaluation. +#[derive(Clone)] +pub struct LogReport; + +impl EvalErrorReport for LogReport { + fn report(&self, error: ExprError) { + tracing::error!(%error, "failed to evaluate expression"); + } +} + /// A wrapper of [`Expression`] that evaluates in a non-strict way. Basically... /// - When an error occurs during chunk-level evaluation, recompute in row-based execution and pad /// with NULL for each failed row. /// - Report all error occurred during row-level evaluation to the [`EvalErrorReport`]. -pub struct NonStrict { +pub(crate) struct NonStrict { inner: E, report: R, } diff --git a/src/expr/impl/src/scalar/cast.rs b/src/expr/impl/src/scalar/cast.rs index 889cc43fe6b18..c173c76c330c5 100644 --- a/src/expr/impl/src/scalar/cast.rs +++ b/src/expr/impl/src/scalar/cast.rs @@ -22,7 +22,9 @@ use risingwave_common::cast; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, Int256, IntoOrdered, JsonbRef, ToText, F64}; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_expr::expr::{build_func, Context, Expression, InputRefExpression}; +use risingwave_expr::expr::{ + build_func, Context, Expression, ExpressionBoxExt, InputRefExpression, +}; use risingwave_expr::{function, ExprError, Result}; use risingwave_pb::expr::expr_node::PbType; diff --git a/src/expr/impl/src/table_function/generate_series.rs b/src/expr/impl/src/table_function/generate_series.rs index 586fa60de02c2..dfa09b0e215b8 100644 --- a/src/expr/impl/src/table_function/generate_series.rs +++ b/src/expr/impl/src/table_function/generate_series.rs @@ -159,7 +159,7 @@ mod tests { use risingwave_common::array::DataChunk; use risingwave_common::types::test_utils::IntervalTestExt; use risingwave_common::types::{DataType, Decimal, Interval, ScalarImpl, Timestamp}; - use risingwave_expr::expr::{BoxedExpression, Expression, LiteralExpression}; + use risingwave_expr::expr::{BoxedExpression, ExpressionBoxExt, LiteralExpression}; use risingwave_expr::table_function::build; use risingwave_expr::ExprError; use risingwave_pb::expr::table_function::PbType; diff --git a/src/storage/src/row_serde/value_serde.rs b/src/storage/src/row_serde/value_serde.rs index 5d56cdba2d96d..9048b90c23a53 100644 --- a/src/storage/src/row_serde/value_serde.rs +++ b/src/storage/src/row_serde/value_serde.rs @@ -114,9 +114,10 @@ impl ValueRowSerdeNew for ColumnAwareSerde { // It's okay since we previously banned impure expressions in default columns. build_from_prost(&expr.expect("expr should not be none")) .expect("build_from_prost error") - .eval_row_infallible(&OwnedRow::empty()) + .eval_row(&OwnedRow::empty()) .now_or_never() .expect("constant expression should not be async") + .expect("eval_row failed") }; Some((i, value)) } else { diff --git a/src/stream/clippy.toml b/src/stream/clippy.toml index a6969d5bd607b..b7257c4acb98c 100644 --- a/src/stream/clippy.toml +++ b/src/stream/clippy.toml @@ -3,8 +3,8 @@ disallowed-methods = [ { path = "risingwave_expr::expr::build_from_prost", reason = "Expressions in streaming must be in non-strict mode. Please use `build_non_strict_from_prost` instead." }, { path = "risingwave_expr::expr::build_func", reason = "Expressions in streaming must be in non-strict mode. Please use `build_func_non_strict` instead." }, - { path = "risingwave_expr::expr::Expression::eval", reason = "Please use `Expression::eval_infallible` instead." }, - { path = "risingwave_expr::expr::Expression::eval_row", reason = "Please use `Expression::eval_row_infallible` instead." }, + { path = "risingwave_expr::expr::Expression::eval", reason = "Please use `NonStrictExpression::eval_infallible` instead." }, + { path = "risingwave_expr::expr::Expression::eval_row", reason = "Please use `NonStrictExpression::eval_row_infallible` instead." }, { path = "risingwave_common::error::internal_err", reason = "Please use per-crate error type instead." }, { path = "risingwave_common::error::internal_error", reason = "Please use per-crate error type instead." }, diff --git a/src/stream/src/executor/aggregation/mod.rs b/src/stream/src/executor/aggregation/mod.rs index dd0ce9d01c544..9bb1113152962 100644 --- a/src/stream/src/executor/aggregation/mod.rs +++ b/src/stream/src/executor/aggregation/mod.rs @@ -21,6 +21,7 @@ use risingwave_common::bail; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{Field, Schema}; use risingwave_expr::aggregate::{AggCall, AggKind}; +use risingwave_expr::expr::{LogReport, NonStrictExpression}; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; @@ -74,7 +75,12 @@ pub async fn agg_call_filter_res( } if let Some(ref filter) = agg_call.filter { - if let Bool(filter_res) = filter.eval_infallible(chunk).await.as_ref() { + // TODO: should we build `filter` in non-strict mode? + if let Bool(filter_res) = NonStrictExpression::new_topmost(&**filter, LogReport) + .eval_infallible(chunk) + .await + .as_ref() + { vis &= filter_res.to_bitmap(); } else { bail!("Filter can only receive bool array"); diff --git a/src/stream/src/executor/dynamic_filter.rs b/src/stream/src/executor/dynamic_filter.rs index e8eb4da545f2e..ccb55b75c24fc 100644 --- a/src/stream/src/executor/dynamic_filter.rs +++ b/src/stream/src/executor/dynamic_filter.rs @@ -26,7 +26,7 @@ use risingwave_common::row::{self, once, OwnedRow, OwnedRow as RowData, Row}; use risingwave_common::types::{DataType, Datum, DefaultOrd, ScalarImpl, ToDatumRef, ToOwnedDatum}; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_expr::expr::{ - build_func_non_strict, BoxedExpression, InputRefExpression, LiteralExpression, + build_func_non_strict, InputRefExpression, LiteralExpression, NonStrictExpression, }; use risingwave_pb::expr::expr_node::Type as ExprNodeType; use risingwave_pb::expr::expr_node::Type::{ @@ -97,7 +97,7 @@ impl DynamicFilterExecutor, + condition: Option, ) -> Result<(Vec, Bitmap), StreamExecutorError> { let mut new_ops = Vec::with_capacity(chunk.capacity()); let mut new_visibility = BitmapBuilder::with_capacity(chunk.capacity()); @@ -265,7 +265,7 @@ impl DynamicFilterExecutor, - expr: BoxedExpression, + expr: NonStrictExpression, executor_id: u64, ) -> Self { let input_info = input.info(); @@ -190,8 +190,8 @@ mod tests { use risingwave_common::array::StreamChunk; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::DataType; - use risingwave_expr::expr::build_from_pretty; + use super::super::test_utils::expr::build_from_pretty; use super::super::test_utils::MockSource; use super::super::*; use super::*; diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index 7aed840679c82..4178012cb9d9e 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -28,7 +28,7 @@ use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, DefaultOrd, ToOwnedDatum}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_expr::expr::BoxedExpression; +use risingwave_expr::expr::NonStrictExpression; use risingwave_expr::ExprError; use risingwave_storage::StateStore; use tokio::time::Instant; @@ -242,9 +242,9 @@ pub struct HashJoinExecutor, /// Optional non-equi join conditions - cond: Option, + cond: Option, /// Column indices of watermark output and offset expression of each inequality, respectively. - inequality_pairs: Vec<(Vec, Option)>, + inequality_pairs: Vec<(Vec, Option)>, /// The output watermark of each inequality condition and its value is the minimum of the /// calculation result of both side. It will be used to generate watermark into downstream /// and do state cleaning if `clean_state` field of that inequality is `true`. @@ -313,7 +313,7 @@ struct EqJoinArgs<'a, K: HashKey, S: StateStore> { side_l: &'a mut JoinSide, side_r: &'a mut JoinSide, actual_output_data_types: &'a [DataType], - cond: &'a mut Option, + cond: &'a mut Option, inequality_watermarks: &'a [Option], chunk: StreamChunk, append_only_optimize: bool, @@ -448,8 +448,8 @@ impl HashJoinExecutor, executor_id: u64, - cond: Option, - inequality_pairs: Vec<(usize, usize, bool, Option)>, + cond: Option, + inequality_pairs: Vec<(usize, usize, bool, Option)>, op_info: String, state_table_l: StateTable, degree_state_table_l: StateTable, @@ -912,7 +912,7 @@ impl HashJoinExecutor input_watermark.val = value.unwrap(), @@ -1275,11 +1275,11 @@ mod tests { use risingwave_common::hash::{Key128, Key64}; use risingwave_common::types::ScalarImpl; use risingwave_common::util::sort_util::OrderType; - use risingwave_expr::expr::build_from_pretty; use risingwave_storage::memory::MemoryStateStore; use super::*; use crate::common::table::state_table::StateTable; + use crate::executor::test_utils::expr::build_from_pretty; use crate::executor::test_utils::{MessageSender, MockSource, StreamExecutorTestExt}; use crate::executor::{ActorContext, Barrier, EpochPair}; @@ -1327,7 +1327,7 @@ mod tests { (state_table, degree_state_table) } - fn create_cond(condition_text: Option) -> BoxedExpression { + fn create_cond(condition_text: Option) -> NonStrictExpression { build_from_pretty( condition_text .as_deref() @@ -1339,7 +1339,7 @@ mod tests { with_condition: bool, null_safe: bool, condition_text: Option, - inequality_pairs: Vec<(usize, usize, bool, Option)>, + inequality_pairs: Vec<(usize, usize, bool, Option)>, ) -> (MessageSender, MessageSender, BoxedMessageStream) { let schema = Schema { fields: vec![ diff --git a/src/stream/src/executor/hop_window.rs b/src/stream/src/executor/hop_window.rs index c6fffcd94896d..42d13d790da88 100644 --- a/src/stream/src/executor/hop_window.rs +++ b/src/stream/src/executor/hop_window.rs @@ -19,7 +19,7 @@ use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::{DataChunk, Op}; use risingwave_common::types::Interval; -use risingwave_expr::expr::BoxedExpression; +use risingwave_expr::expr::NonStrictExpression; use risingwave_expr::ExprError; use super::error::StreamExecutorError; @@ -33,8 +33,8 @@ pub struct HopWindowExecutor { pub time_col_idx: usize, pub window_slide: Interval, pub window_size: Interval, - window_start_exprs: Vec, - window_end_exprs: Vec, + window_start_exprs: Vec, + window_end_exprs: Vec, pub output_indices: Vec, chunk_size: usize, } @@ -48,8 +48,8 @@ impl HopWindowExecutor { time_col_idx: usize, window_slide: Interval, window_size: Interval, - window_start_exprs: Vec, - window_end_exprs: Vec, + window_start_exprs: Vec, + window_end_exprs: Vec, output_indices: Vec, chunk_size: usize, ) -> Self { @@ -251,6 +251,7 @@ mod tests { use risingwave_common::types::test_utils::IntervalTestExt; use risingwave_common::types::{DataType, Interval}; use risingwave_expr::expr::test_utils::make_hop_window_expression; + use risingwave_expr::expr::NonStrictExpression; use crate::executor::test_utils::MockSource; use crate::executor::{ActorContext, Executor, ExecutorInfo, StreamChunk}; @@ -302,8 +303,14 @@ mod tests { 2, window_slide, window_size, - window_start_exprs, - window_end_exprs, + window_start_exprs + .into_iter() + .map(NonStrictExpression::for_test) + .collect(), + window_end_exprs + .into_iter() + .map(NonStrictExpression::for_test) + .collect(), output_indices, CHUNK_SIZE, ) diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs index a9c219a25641f..cd505093294f1 100644 --- a/src/stream/src/executor/integration_tests.rs +++ b/src/stream/src/executor/integration_tests.rs @@ -152,7 +152,7 @@ async fn test_merger_sum_aggr() { vec![], vec![ // TODO: use the new streaming_if_null expression here, and add `None` tests - Box::new(InputRefExpression::new(DataType::Int64, 1)), + NonStrictExpression::for_test(InputRefExpression::new(DataType::Int64, 1)), ], 3, MultiMap::new(), diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index 99b090e21a240..c28d6ec8564d9 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -31,7 +31,7 @@ use risingwave_common::util::epoch::{Epoch, EpochPair}; use risingwave_common::util::tracing::TracingContext; use risingwave_common::util::value_encoding::{DatumFromProtoExt, DatumToProtoExt}; use risingwave_connector::source::SplitImpl; -use risingwave_expr::expr::BoxedExpression; +use risingwave_expr::expr::{Expression, NonStrictExpression}; use risingwave_pb::data::PbEpoch; use risingwave_pb::expr::PbInputRef; use risingwave_pb::stream_plan::barrier::{BarrierKind, PbMutation}; @@ -641,7 +641,7 @@ impl Watermark { pub async fn transform_with_expr( self, - expr: &BoxedExpression, + expr: &NonStrictExpression, new_col_idx: usize, ) -> Option { let Self { col_idx, val, .. } = self; @@ -651,7 +651,7 @@ impl Watermark { OwnedRow::new(row) }; let val = expr.eval_row_infallible(&row).await?; - Some(Self::new(new_col_idx, expr.return_type(), val)) + Some(Self::new(new_col_idx, expr.inner().return_type(), val)) } /// Transform the watermark with the given output indices. If this watermark is not in the diff --git a/src/stream/src/executor/project.rs b/src/stream/src/executor/project.rs index 56a31bde901b9..8cfebfecd3f33 100644 --- a/src/stream/src/executor/project.rs +++ b/src/stream/src/executor/project.rs @@ -21,7 +21,7 @@ use risingwave_common::catalog::{Field, Schema}; use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::ToOwnedDatum; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_expr::expr::BoxedExpression; +use risingwave_expr::expr::NonStrictExpression; use super::*; @@ -38,7 +38,7 @@ struct Inner { info: ExecutorInfo, /// Expressions of the current projection. - exprs: Vec, + exprs: Vec, /// All the watermark derivations, (input_column_index, output_column_index). And the /// derivation expression is the project's expression itself. watermark_derivations: MultiMap, @@ -58,7 +58,7 @@ impl ProjectExecutor { ctx: ActorContextRef, input: Box, pk_indices: PkIndices, - exprs: Vec, + exprs: Vec, executor_id: u64, watermark_derivations: MultiMap, nondecreasing_expr_indices: Vec, @@ -233,11 +233,12 @@ mod tests { use risingwave_common::array::{DataChunk, StreamChunk}; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::{DataType, Datum}; - use risingwave_expr::expr::{self, build_from_pretty, Expression, ValueImpl}; + use risingwave_expr::expr::{self, Expression, ValueImpl}; use super::super::test_utils::MockSource; use super::super::*; use super::*; + use crate::executor::test_utils::expr::build_from_pretty; use crate::executor::test_utils::StreamExecutorTestExt; #[tokio::test] @@ -345,7 +346,7 @@ mod tests { let a_expr = build_from_pretty("(add:int8 $0:int8 1:int8)"); let b_expr = build_from_pretty("(subtract:int8 $0:int8 1:int8)"); - let c_expr = DummyNondecreasingExpr.boxed(); + let c_expr = NonStrictExpression::for_test(DummyNondecreasingExpr); let project = Box::new(ProjectExecutor::new( ActorContext::create(123), diff --git a/src/stream/src/executor/project_set.rs b/src/stream/src/executor/project_set.rs index 6867e3d55bfde..ff3214db88eaa 100644 --- a/src/stream/src/executor/project_set.rs +++ b/src/stream/src/executor/project_set.rs @@ -24,6 +24,7 @@ use risingwave_common::catalog::{Field, Schema}; use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::{DataType, Datum, DatumRef, ToOwnedDatum}; use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_expr::expr::{LogReport, NonStrictExpression}; use risingwave_expr::table_function::ProjectSetSelectItem; use super::error::StreamExecutorError; @@ -260,7 +261,11 @@ impl Inner { ProjectSetSelectItem::Expr(expr) => { watermark .clone() - .transform_with_expr(expr, expr_idx + PROJ_ROW_ID_OFFSET) + .transform_with_expr( + // TODO: should we build `expr` in non-strict mode? + &NonStrictExpression::new_topmost(expr, LogReport), + expr_idx + PROJ_ROW_ID_OFFSET, + ) .await } ProjectSetSelectItem::TableFunction(_) => { diff --git a/src/stream/src/executor/temporal_join.rs b/src/stream/src/executor/temporal_join.rs index 3c8cde63c4ca9..82c1e56649672 100644 --- a/src/stream/src/executor/temporal_join.rs +++ b/src/stream/src/executor/temporal_join.rs @@ -32,7 +32,7 @@ use risingwave_common::hash::{HashKey, NullBitmap}; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_expr::expr::BoxedExpression; +use risingwave_expr::expr::NonStrictExpression; use risingwave_hummock_sdk::{HummockEpoch, HummockReadEpoch}; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -57,7 +57,7 @@ pub struct TemporalJoinExecutor, right_join_keys: Vec, null_safe: Vec, - condition: Option, + condition: Option, output_indices: Vec, pk_indices: PkIndices, schema: Schema, @@ -338,7 +338,7 @@ impl TemporalJoinExecutor left_join_keys: Vec, right_join_keys: Vec, null_safe: Vec, - condition: Option, + condition: Option, pk_indices: PkIndices, output_indices: Vec, table_output_indices: Vec, diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index bb4864ac04ef8..13a9237cf0159 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -34,11 +34,11 @@ pub mod prelude { pub use risingwave_common::test_prelude::StreamChunkTestExt; pub use risingwave_common::types::DataType; pub use risingwave_common::util::sort_util::OrderType; - pub use risingwave_expr::expr::build_from_pretty; pub use risingwave_storage::memory::MemoryStateStore; pub use risingwave_storage::StateStore; pub use crate::common::table::state_table::StateTable; + pub use crate::executor::test_utils::expr::build_from_pretty; pub use crate::executor::test_utils::{MessageSender, MockSource, StreamExecutorTestExt}; pub use crate::executor::{ActorContext, BoxedMessageStream, Executor, PkIndices}; } @@ -263,6 +263,14 @@ pub trait StreamExecutorTestExt: MessageStream + Unpin { // FIXME: implement on any `impl MessageStream` if the analyzer works well. impl StreamExecutorTestExt for BoxedMessageStream {} +pub mod expr { + use risingwave_expr::expr::NonStrictExpression; + + pub fn build_from_pretty(s: impl AsRef) -> NonStrictExpression { + NonStrictExpression::for_test(risingwave_expr::expr::build_from_pretty(s)) + } +} + pub mod agg_executor { use std::sync::atomic::AtomicU64; use std::sync::Arc; diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 624b2531bf7bd..8c09b56aa3551 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -21,7 +21,7 @@ use risingwave_common::array::{DataChunk, Op, StreamChunk}; use risingwave_common::catalog::Schema; use risingwave_common::ensure; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_expr::expr::BoxedExpression; +use risingwave_expr::expr::NonStrictExpression; use tokio::sync::mpsc::UnboundedReceiver; use super::{ @@ -40,7 +40,7 @@ pub struct ValuesExecutor { barrier_receiver: UnboundedReceiver, progress: CreateMviewProgress, - rows: vec::IntoIter>, + rows: vec::IntoIter>, pk_indices: PkIndices, identity: String, schema: Schema, @@ -51,7 +51,7 @@ impl ValuesExecutor { pub fn new( ctx: ActorContextRef, progress: CreateMviewProgress, - rows: Vec>, + rows: Vec>, schema: Schema, barrier_receiver: UnboundedReceiver, executor_id: u64, @@ -167,7 +167,7 @@ mod tests { }; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::{DataType, ScalarImpl, StructType}; - use risingwave_expr::expr::{BoxedExpression, LiteralExpression}; + use risingwave_expr::expr::{BoxedExpression, LiteralExpression, NonStrictExpression}; use tokio::sync::mpsc::unbounded_channel; use super::ValuesExecutor; @@ -202,11 +202,11 @@ mod tests { vec![], ), Some(ScalarImpl::Struct(value)), - )) as BoxedExpression, + )), Box::new(LiteralExpression::new( DataType::Int64, Some(ScalarImpl::Int64(0)), - )) as BoxedExpression, + )), ]; let fields = exprs .iter() // for each column @@ -215,7 +215,10 @@ mod tests { let values_executor_struct = ValuesExecutor::new( ActorContext::create(actor_id), progress, - vec![exprs], + vec![exprs + .into_iter() + .map(NonStrictExpression::for_test) + .collect()], Schema { fields }, barrier_receiver, 10005, diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs index ad332112ef269..5e5454cecff93 100644 --- a/src/stream/src/executor/watermark_filter.rs +++ b/src/stream/src/executor/watermark_filter.rs @@ -23,7 +23,8 @@ use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, DefaultOrd, ScalarImpl}; use risingwave_common::{bail, row}; use risingwave_expr::expr::{ - build_func_non_strict, BoxedExpression, Expression, InputRefExpression, LiteralExpression, + build_func_non_strict, ExpressionBoxExt, InputRefExpression, LiteralExpression, + NonStrictExpression, }; use risingwave_expr::Result as ExprResult; use risingwave_pb::expr::expr_node::Type; @@ -44,7 +45,7 @@ use crate::task::ActorEvalErrorReport; pub struct WatermarkFilterExecutor { input: BoxedExecutor, /// The expression used to calculate the watermark value. - watermark_expr: BoxedExpression, + watermark_expr: NonStrictExpression, /// The column we should generate watermark and filter on. event_time_col_idx: usize, ctx: ActorContextRef, @@ -55,7 +56,7 @@ pub struct WatermarkFilterExecutor { impl WatermarkFilterExecutor { pub fn new( input: BoxedExecutor, - watermark_expr: BoxedExpression, + watermark_expr: NonStrictExpression, event_time_col_idx: usize, ctx: ActorContextRef, table: StateTable, @@ -298,7 +299,7 @@ impl WatermarkFilterExecutor { event_time_col_idx: usize, watermark: ScalarImpl, eval_error_report: ActorEvalErrorReport, - ) -> ExprResult { + ) -> ExprResult { build_func_non_strict( Type::GreaterThanOrEqual, DataType::Boolean, @@ -350,11 +351,11 @@ mod tests { use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::types::Date; use risingwave_common::util::sort_util::OrderType; - use risingwave_expr::expr::build_from_pretty; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::table::Distribution; use super::*; + use crate::executor::test_utils::expr::build_from_pretty; use crate::executor::test_utils::{MessageSender, MockSource}; use crate::executor::ActorContext; diff --git a/src/stream/src/from_proto/hash_join.rs b/src/stream/src/from_proto/hash_join.rs index 44799af9405c2..87174282e517a 100644 --- a/src/stream/src/from_proto/hash_join.rs +++ b/src/stream/src/from_proto/hash_join.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use risingwave_common::hash::{HashKey, HashKeyDispatcher}; use risingwave_common::types::DataType; use risingwave_expr::expr::{ - build_func_non_strict, build_non_strict_from_prost, BoxedExpression, InputRefExpression, + build_func_non_strict, build_non_strict_from_prost, InputRefExpression, NonStrictExpression, }; pub use risingwave_pb::expr::expr_node::Type as ExprType; use risingwave_pb::plan_common::JoinType as JoinTypeProto; @@ -109,7 +109,8 @@ impl ExecutorBuilder for HashJoinExecutorBuilder { build_non_strict_from_prost( delta_expression.delta.as_ref().unwrap(), params.eval_error_report.clone(), - )?, + )? + .into_inner(), ], params.eval_error_report.clone(), )?) @@ -175,8 +176,8 @@ struct HashJoinExecutorDispatcherArgs { pk_indices: PkIndices, output_indices: Vec, executor_id: u64, - cond: Option, - inequality_pairs: Vec<(usize, usize, bool, Option)>, + cond: Option, + inequality_pairs: Vec<(usize, usize, bool, Option)>, op_info: String, state_table_l: StateTable, degree_state_table_l: StateTable, diff --git a/src/stream/src/from_proto/temporal_join.rs b/src/stream/src/from_proto/temporal_join.rs index 8b7b3b6af1335..58699089e8c27 100644 --- a/src/stream/src/from_proto/temporal_join.rs +++ b/src/stream/src/from_proto/temporal_join.rs @@ -18,7 +18,7 @@ use risingwave_common::catalog::{ColumnDesc, TableId, TableOption}; use risingwave_common::hash::{HashKey, HashKeyDispatcher}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; -use risingwave_expr::expr::{build_non_strict_from_prost, BoxedExpression}; +use risingwave_expr::expr::{build_non_strict_from_prost, NonStrictExpression}; use risingwave_pb::plan_common::{JoinType as JoinTypeProto, StorageTableDesc}; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::Distribution; @@ -190,7 +190,7 @@ struct TemporalJoinExecutorDispatcherArgs { left_join_keys: Vec, right_join_keys: Vec, null_safe: Vec, - condition: Option, + condition: Option, pk_indices: PkIndices, output_indices: Vec, table_output_indices: Vec, diff --git a/src/stream/tests/integration_tests/hop_window.rs b/src/stream/tests/integration_tests/hop_window.rs index 167857cc7d9fc..9d6d879240fc0 100644 --- a/src/stream/tests/integration_tests/hop_window.rs +++ b/src/stream/tests/integration_tests/hop_window.rs @@ -15,6 +15,7 @@ use risingwave_common::types::test_utils::IntervalTestExt; use risingwave_common::types::{Interval, Timestamp}; use risingwave_expr::expr::test_utils::make_hop_window_expression; +use risingwave_expr::expr::NonStrictExpression; use risingwave_stream::executor::{ExecutorInfo, HopWindowExecutor}; use crate::prelude::*; @@ -55,8 +56,14 @@ fn create_executor(output_indices: Vec) -> (MessageSender, BoxedMessageSt TIME_COL_IDX, window_slide, window_size, - window_start_exprs, - window_end_exprs, + window_start_exprs + .into_iter() + .map(NonStrictExpression::for_test) + .collect(), + window_end_exprs + .into_iter() + .map(NonStrictExpression::for_test) + .collect(), output_indices, CHUNK_SIZE, ) diff --git a/src/stream/tests/integration_tests/project_set.rs b/src/stream/tests/integration_tests/project_set.rs index bf1354c25b83b..61a879256108d 100644 --- a/src/stream/tests/integration_tests/project_set.rs +++ b/src/stream/tests/integration_tests/project_set.rs @@ -29,10 +29,10 @@ fn create_executor() -> (MessageSender, BoxedMessageStream) { }; let (tx, source) = MockSource::channel(schema, PkIndices::new()); - let test_expr = build_from_pretty("(add:int8 $0:int8 $1:int8)"); - let test_expr_watermark = build_from_pretty("(add:int8 $0:int8 1:int8)"); - let tf1 = repeat(build_from_pretty("1:int4"), 1); - let tf2 = repeat(build_from_pretty("2:int4"), 2); + let test_expr = build_from_pretty("(add:int8 $0:int8 $1:int8)").into_inner(); + let test_expr_watermark = build_from_pretty("(add:int8 $0:int8 1:int8)").into_inner(); + let tf1 = repeat(build_from_pretty("1:int4").into_inner(), 1); + let tf2 = repeat(build_from_pretty("2:int4").into_inner(), 2); let project_set = Box::new(ProjectSetExecutor::new( ActorContext::create(123), From 02282b9c3fdd52933fc8baf7032777c184577e57 Mon Sep 17 00:00:00 2001 From: lmatz Date: Tue, 24 Oct 2023 17:10:48 +0800 Subject: [PATCH 06/48] chore: upgrade the docker image version (#13020) --- docker/docker-compose.yml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index 89aa99a1c8b5d..d25c94daf2670 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -2,7 +2,7 @@ version: "3" services: compactor-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0}" command: - compactor-node - "--listen-addr" @@ -37,7 +37,7 @@ services: timeout: 5s retries: 5 compute-node-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0}" command: - compute-node - "--listen-addr" @@ -122,7 +122,7 @@ services: timeout: 5s retries: 5 frontend-node-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0}" command: - frontend-node - "--listen-addr" @@ -179,7 +179,7 @@ services: timeout: 5s retries: 5 meta-node-0: - image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0}" + image: "ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0}" command: - meta-node - "--listen-addr" @@ -295,7 +295,7 @@ services: timeout: 5s retries: 5 connector-node: - image: ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.2.0} + image: ghcr.io/risingwavelabs/risingwave:${RW_IMAGE_VERSION:-v1.3.0} entrypoint: "/risingwave/bin/connector-node/start-service.sh" ports: - 50051 From e64543f9e5a8b080ddf8d8717846ccd4c68dac80 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Tue, 24 Oct 2023 04:15:25 -0500 Subject: [PATCH 07/48] feat(agg): introduce an internal agg kind to avoid minput state table after #12349 (#12384) Signed-off-by: Richard Chien --- proto/expr.proto | 1 + src/expr/core/src/aggregate/def.rs | 14 +- src/expr/impl/src/aggregate/general.rs | 9 + .../tests/testdata/output/agg.yaml | 24 +-- .../testdata/output/batch_index_join.yaml | 6 +- .../tests/testdata/output/except.yaml | 59 +++--- .../tests/testdata/output/intersect.yaml | 59 +++--- .../tests/testdata/output/nexmark.yaml | 179 +++++++++--------- .../tests/testdata/output/subquery.yaml | 28 +-- .../output/subquery_expr_correlated.yaml | 8 +- .../src/optimizer/plan_node/generic/agg.rs | 7 +- .../rule/agg_group_by_simplify_rule.rs | 5 +- 12 files changed, 199 insertions(+), 200 deletions(-) diff --git a/proto/expr.proto b/proto/expr.proto index 769532d8dbe19..2f252d67c8400 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -348,6 +348,7 @@ message AggCall { MODE = 24; LAST_VALUE = 25; GROUPING = 26; + INTERNAL_LAST_SEEN_VALUE = 27; } Type type = 1; repeated InputRef args = 2; diff --git a/src/expr/core/src/aggregate/def.rs b/src/expr/core/src/aggregate/def.rs index f71bfd454a415..964ec46c9f9c4 100644 --- a/src/expr/core/src/aggregate/def.rs +++ b/src/expr/core/src/aggregate/def.rs @@ -233,6 +233,9 @@ pub enum AggKind { PercentileDisc, Mode, Grouping, + + /// Return last seen one of the input values. + InternalLastSeenValue, } impl AggKind { @@ -264,6 +267,7 @@ impl AggKind { PbType::PercentileDisc => Ok(AggKind::PercentileDisc), PbType::Mode => Ok(AggKind::Mode), PbType::Grouping => Ok(AggKind::Grouping), + PbType::InternalLastSeenValue => Ok(AggKind::InternalLastSeenValue), PbType::Unspecified => bail!("Unrecognized agg."), } } @@ -294,8 +298,9 @@ impl AggKind { Self::VarSamp => PbType::VarSamp, Self::PercentileCont => PbType::PercentileCont, Self::PercentileDisc => PbType::PercentileDisc, - Self::Grouping => PbType::Grouping, Self::Mode => PbType::Mode, + Self::Grouping => PbType::Grouping, + Self::InternalLastSeenValue => PbType::InternalLastSeenValue, } } } @@ -422,6 +427,7 @@ pub mod agg_kinds { | AggKind::BoolAnd | AggKind::BoolOr | AggKind::ApproxCountDistinct + | AggKind::InternalLastSeenValue }; } pub use single_value_state; @@ -450,7 +456,11 @@ impl AggKind { /// Get the total phase agg kind from the partial phase agg kind. pub fn partial_to_total(self) -> Option { match self { - AggKind::BitXor | AggKind::Min | AggKind::Max | AggKind::Sum => Some(self), + AggKind::BitXor + | AggKind::Min + | AggKind::Max + | AggKind::Sum + | AggKind::InternalLastSeenValue => Some(self), AggKind::Sum0 | AggKind::Count => Some(AggKind::Sum0), agg_kinds::simply_cannot_two_phase!() => None, agg_kinds::rewritten!() => None, diff --git a/src/expr/impl/src/aggregate/general.rs b/src/expr/impl/src/aggregate/general.rs index de1331c524063..f47c94d45f24d 100644 --- a/src/expr/impl/src/aggregate/general.rs +++ b/src/expr/impl/src/aggregate/general.rs @@ -62,6 +62,15 @@ fn last_value(_: T, input: T) -> T { input } +#[aggregate("internal_last_seen_value(*) -> auto", state = "ref")] +fn internal_last_seen_value(state: T, input: T, retract: bool) -> T { + if retract { + state + } else { + input + } +} + /// Note the following corner cases: /// /// ```slt diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index aefb4df98ef4e..baa77dc79c89b 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1395,20 +1395,20 @@ sq_1.col_2; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [max(max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))))] } - └─BatchHashAgg { group_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], aggs: [max(max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))))] } - └─BatchExchange { order: [], dist: HashShard(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))) } - └─BatchHashAgg { group_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], aggs: [max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)))] } - └─BatchSortAgg { group_key: [lineitem.l_orderkey], aggs: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))] } + └─BatchProject { exprs: [max(max(internal_last_seen_value(lineitem.l_commitdate)))] } + └─BatchHashAgg { group_key: [internal_last_seen_value(lineitem.l_commitdate)], aggs: [max(max(internal_last_seen_value(lineitem.l_commitdate)))] } + └─BatchExchange { order: [], dist: HashShard(internal_last_seen_value(lineitem.l_commitdate)) } + └─BatchHashAgg { group_key: [internal_last_seen_value(lineitem.l_commitdate)], aggs: [max(internal_last_seen_value(lineitem.l_commitdate))] } + └─BatchSortAgg { group_key: [lineitem.l_orderkey], aggs: [internal_last_seen_value(lineitem.l_commitdate)] } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_commitdate], distribution: UpstreamHashShard(lineitem.l_orderkey) } stream_plan: |- - StreamMaterialize { columns: [col_0, first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))(hidden)], stream_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], pk_columns: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], pk_conflict: NoCheck } - └─StreamProject { exprs: [max(max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)))), first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))] } - └─StreamHashAgg { group_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))], aggs: [max(max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)))), count] } - └─StreamExchange { dist: HashShard(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))) } - └─StreamHashAgg { group_key: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)), $expr1], aggs: [max(first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC))), count] } - └─StreamProject { exprs: [lineitem.l_orderkey, first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)), Vnode(lineitem.l_orderkey) as $expr1] } - └─StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [first_value(lineitem.l_commitdate order_by(lineitem.l_commitdate ASC)), count] } + StreamMaterialize { columns: [col_0, internal_last_seen_value(lineitem.l_commitdate)(hidden)], stream_key: [internal_last_seen_value(lineitem.l_commitdate)], pk_columns: [internal_last_seen_value(lineitem.l_commitdate)], pk_conflict: NoCheck } + └─StreamProject { exprs: [max(max(internal_last_seen_value(lineitem.l_commitdate))), internal_last_seen_value(lineitem.l_commitdate)] } + └─StreamHashAgg { group_key: [internal_last_seen_value(lineitem.l_commitdate)], aggs: [max(max(internal_last_seen_value(lineitem.l_commitdate))), count] } + └─StreamExchange { dist: HashShard(internal_last_seen_value(lineitem.l_commitdate)) } + └─StreamHashAgg { group_key: [internal_last_seen_value(lineitem.l_commitdate), $expr1], aggs: [max(internal_last_seen_value(lineitem.l_commitdate)), count] } + └─StreamProject { exprs: [lineitem.l_orderkey, internal_last_seen_value(lineitem.l_commitdate), Vnode(lineitem.l_orderkey) as $expr1] } + └─StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [internal_last_seen_value(lineitem.l_commitdate), count] } └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_commitdate], pk: [lineitem.l_orderkey], dist: UpstreamHashShard(lineitem.l_orderkey) } - name: two phase agg on hop window input should use two phase agg sql: | diff --git a/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml b/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml index 236bc31b2503e..2d1b0951089e8 100644 --- a/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/batch_index_join.yaml @@ -60,9 +60,9 @@ select t2.c, t2.d, count(distinct t.a) from t join t2 on t.a = t2.c group by t2.c, t2.d; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [first_value(t2.c order_by(t2.c ASC))], aggs: [first_value(first_value(t2.d order_by(t2.d ASC)) order_by(first_value(t2.d order_by(t2.d ASC)) ASC)), count(t.a)] } - └─BatchExchange { order: [], dist: HashShard(first_value(t2.c order_by(t2.c ASC))) } - └─BatchHashAgg { group_key: [t.a], aggs: [first_value(t2.c order_by(t2.c ASC)), first_value(t2.d order_by(t2.d ASC))] } + └─BatchHashAgg { group_key: [internal_last_seen_value(t2.c)], aggs: [internal_last_seen_value(internal_last_seen_value(t2.d)), count(t.a)] } + └─BatchExchange { order: [], dist: HashShard(internal_last_seen_value(t2.c)) } + └─BatchHashAgg { group_key: [t.a], aggs: [internal_last_seen_value(t2.c), internal_last_seen_value(t2.d)] } └─BatchLookupJoin { type: Inner, predicate: t.a = t2.c, output: [t2.c, t2.d, t.a] } └─BatchExchange { order: [], dist: UpstreamHashShard(t.a) } └─BatchScan { table: t, columns: [t.a], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/output/except.yaml b/src/frontend/planner_test/tests/testdata/output/except.yaml index 204a1814b8db7..1e27a7b74c0f0 100644 --- a/src/frontend/planner_test/tests/testdata/output/except.yaml +++ b/src/frontend/planner_test/tests/testdata/output/except.yaml @@ -108,20 +108,20 @@ create table t2 (a int, b numeric, c bigint, primary key(a)); select * from t1 except select * from t2; optimized_logical_plan_for_batch: |- - LogicalAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } + LogicalAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } └─LogicalJoin { type: LeftAnti, on: IsNotDistinctFrom(t1.a, t2.a) AND IsNotDistinctFrom(t1.b, t2.b) AND IsNotDistinctFrom(t1.c, t2.c), output: all } ├─LogicalScan { table: t1, columns: [t1.a, t1.b, t1.c] } └─LogicalScan { table: t2, columns: [t2.a, t2.b, t2.c] } batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } + └─BatchHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } └─BatchLookupJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } └─BatchScan { table: t1, columns: [t1.a, t1.b, t1.c], distribution: UpstreamHashShard(t1.a) } stream_plan: |- StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.a, first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } - └─StreamHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC)), count] } + └─StreamProject { exprs: [t1.a, internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } + └─StreamHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c), count] } └─StreamExchange { dist: HashShard(t1.a) } └─StreamHashJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } ├─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) } @@ -130,83 +130,70 @@ └─StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } { materialized table: 4294967294 } - └── StreamProject { exprs: [t1.a, first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } - └── StreamHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC)), count] } - ├── intermediate state table: 2 - ├── state tables: [ 0, 1 ] + StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } + ├── materialized table: 4294967294 + └── StreamProject { exprs: [t1.a, internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } + └── StreamHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c), count] } + ├── intermediate state table: 0 + ├── state tables: [] ├── distinct tables: [] └── StreamExchange Hash([0]) from 1 Fragment 1 StreamHashJoin { type: LeftAnti, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } - ├── left table: 3 - ├── right table: 5 - ├── left degree table: 4 - ├── right degree table: 6 + ├── left table: 1 + ├── right table: 3 + ├── left degree table: 2 + ├── right degree table: 4 ├── StreamExchange Hash([0, 1, 2]) from 2 └── StreamExchange Hash([0, 1, 2]) from 3 Fragment 2 - Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 7 } + Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 8 } + Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 6 } ├── Upstream └── BatchPlanNode Table 0 - ├── columns: [ t1_a, t1_b, t1_c ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] - ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 - - Table 1 - ├── columns: [ t1_a, t1_c, t1_b ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] - ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 - - Table 2 - ├── columns: [ t1_a, first_value(t1_b order_by(t1_b ASC)), first_value(t1_c order_by(t1_c ASC)), count ] + ├── columns: [ t1_a, internal_last_seen_value(t1_b), internal_last_seen_value(t1_c), count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 + Table 1 ├── columns: [ t1_a, t1_b, t1_c ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 4 + Table 2 ├── columns: [ t1_a, t1_b, t1_c, _degree ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 5 + Table 3 ├── columns: [ t2_a, t2_b, t2_c ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 6 + Table 4 ├── columns: [ t2_a, t2_b, t2_c, _degree ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 7 + Table 5 ├── columns: [ vnode, a, t1_backfill_finished, t1_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] @@ -214,7 +201,7 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 8 + Table 6 ├── columns: [ vnode, a, t2_backfill_finished, t2_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] diff --git a/src/frontend/planner_test/tests/testdata/output/intersect.yaml b/src/frontend/planner_test/tests/testdata/output/intersect.yaml index c203f1f953814..91839346824ec 100644 --- a/src/frontend/planner_test/tests/testdata/output/intersect.yaml +++ b/src/frontend/planner_test/tests/testdata/output/intersect.yaml @@ -108,20 +108,20 @@ create table t2 (a int, b numeric, c bigint, primary key(a)); select * from t1 intersect select * from t2; optimized_logical_plan_for_batch: |- - LogicalAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } + LogicalAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } └─LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(t1.a, t2.a) AND IsNotDistinctFrom(t1.b, t2.b) AND IsNotDistinctFrom(t1.c, t2.c), output: all } ├─LogicalScan { table: t1, columns: [t1.a, t1.b, t1.c] } └─LogicalScan { table: t2, columns: [t2.a, t2.b, t2.c] } batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } + └─BatchHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } └─BatchLookupJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } └─BatchScan { table: t1, columns: [t1.a, t1.b, t1.c], distribution: UpstreamHashShard(t1.a) } stream_plan: |- StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.a, first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } - └─StreamHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC)), count] } + └─StreamProject { exprs: [t1.a, internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } + └─StreamHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c), count] } └─StreamExchange { dist: HashShard(t1.a) } └─StreamHashJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } ├─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) } @@ -130,83 +130,70 @@ └─StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } { materialized table: 4294967294 } - └── StreamProject { exprs: [t1.a, first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC))] } - └── StreamHashAgg { group_key: [t1.a], aggs: [first_value(t1.b order_by(t1.b ASC)), first_value(t1.c order_by(t1.c ASC)), count] } - ├── intermediate state table: 2 - ├── state tables: [ 0, 1 ] + StreamMaterialize { columns: [a, b, c], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } + ├── materialized table: 4294967294 + └── StreamProject { exprs: [t1.a, internal_last_seen_value(t1.b), internal_last_seen_value(t1.c)] } + └── StreamHashAgg { group_key: [t1.a], aggs: [internal_last_seen_value(t1.b), internal_last_seen_value(t1.c), count] } + ├── intermediate state table: 0 + ├── state tables: [] ├── distinct tables: [] └── StreamExchange Hash([0]) from 1 Fragment 1 StreamHashJoin { type: LeftSemi, predicate: t1.a IS NOT DISTINCT FROM t2.a AND t1.b IS NOT DISTINCT FROM t2.b AND t1.c IS NOT DISTINCT FROM t2.c, output: all } - ├── left table: 3 - ├── right table: 5 - ├── left degree table: 4 - ├── right degree table: 6 + ├── left table: 1 + ├── right table: 3 + ├── left degree table: 2 + ├── right degree table: 4 ├── StreamExchange Hash([0, 1, 2]) from 2 └── StreamExchange Hash([0, 1, 2]) from 3 Fragment 2 - Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 7 } + Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 8 } + Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.a], dist: UpstreamHashShard(t2.a) } { state table: 6 } ├── Upstream └── BatchPlanNode Table 0 - ├── columns: [ t1_a, t1_b, t1_c ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] - ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 - - Table 1 - ├── columns: [ t1_a, t1_c, t1_b ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] - ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 - - Table 2 - ├── columns: [ t1_a, first_value(t1_b order_by(t1_b ASC)), first_value(t1_c order_by(t1_c ASC)), count ] + ├── columns: [ t1_a, internal_last_seen_value(t1_b), internal_last_seen_value(t1_c), count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 3 + Table 1 ├── columns: [ t1_a, t1_b, t1_c ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 4 + Table 2 ├── columns: [ t1_a, t1_b, t1_c, _degree ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 5 + Table 3 ├── columns: [ t2_a, t2_b, t2_c ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 6 + Table 4 ├── columns: [ t2_a, t2_b, t2_c, _degree ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] ├── value indices: [ 3 ] ├── distribution key: [ 0, 1, 2 ] └── read pk prefix len hint: 3 - Table 7 + Table 5 ├── columns: [ vnode, a, t1_backfill_finished, t1_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] @@ -214,7 +201,7 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 8 + Table 6 ├── columns: [ vnode, a, t2_backfill_finished, t2_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index f4b9e28ce0775..2f23b0674fa84 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -834,9 +834,9 @@ AND P.endtime = A.endtime; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, first_value(person.name order_by(person.name ASC)), $expr1] } + └─BatchHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1] } ├─BatchExchange { order: [], dist: HashShard(person.id, $expr1, $expr2) } - │ └─BatchHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [first_value(person.name order_by(person.name ASC))] } + │ └─BatchHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name)] } │ └─BatchProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } │ └─BatchProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] } │ └─BatchScan { table: person, columns: [person.id, person.name, person.date_time], distribution: UpstreamHashShard(person.id) } @@ -847,10 +847,10 @@ └─BatchScan { table: auction, columns: [auction.date_time, auction.seller], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2], pk_columns: [id, starttime, $expr2], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, first_value(person.name order_by(person.name ASC)), $expr1, $expr2, auction.seller, $expr3, $expr4] } + └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] } ├─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) } - │ └─StreamProject { exprs: [person.id, $expr1, $expr2, first_value(person.name order_by(person.name ASC))] } - │ └─StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [first_value(person.name order_by(person.name ASC)), count] } + │ └─StreamProject { exprs: [person.id, $expr1, $expr2, internal_last_seen_value(person.name)] } + │ └─StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name), count] } │ └─StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } │ └─StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] } │ └─StreamTableScan { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } @@ -864,33 +864,33 @@ Fragment 0 StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2], pk_columns: [id, starttime, $expr2], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, first_value(person.name order_by(person.name ASC)), $expr1, $expr2, auction.seller, $expr3, $expr4] } + └── StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] } ├── left table: 0 ├── right table: 2 ├── left degree table: 1 ├── right degree table: 3 ├── StreamExchange Hash([0, 1, 2]) from 1 └── StreamProject { exprs: [auction.seller, $expr3, $expr4] } - └── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { intermediate state table: 7, state tables: [], distinct tables: [] } + └── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } └── StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 - StreamProject { exprs: [person.id, $expr1, $expr2, first_value(person.name order_by(person.name ASC))] } - └── StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [first_value(person.name order_by(person.name ASC)), count] } { intermediate state table: 5, state tables: [ 4 ], distinct tables: [] } + StreamProject { exprs: [person.id, $expr1, $expr2, internal_last_seen_value(person.name)] } + └── StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name), count] } { intermediate state table: 4, state tables: [], distinct tables: [] } └── StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } └── StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] } - └── Chain { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 6 } + └── Chain { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 2 StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] } └── StreamProject { exprs: [auction.date_time, auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, auction.id] } - └── Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 8 } + └── Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } ├── Upstream └── BatchPlanNode - Table 0 { columns: [ person_id, $expr1, $expr2, first_value(person_name order_by(person_name ASC)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 0 { columns: [ person_id, $expr1, $expr2, internal_last_seen_value(person_name) ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } Table 1 { columns: [ person_id, $expr1, $expr2, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } @@ -898,17 +898,20 @@ Table 3 { columns: [ auction_seller, $expr3, $expr4, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4 { columns: [ person_id, $expr1, $expr2, person_name ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 3 ], distribution key: [ 0 ], read pk prefix len hint: 3 } + Table 4 { columns: [ person_id, $expr1, $expr2, internal_last_seen_value(person_name), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 } - Table 5 { columns: [ person_id, $expr1, $expr2, first_value(person_name order_by(person_name ASC)), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 } + Table 5 { columns: [ vnode, id, person_backfill_finished, person_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 6 { columns: [ vnode, id, person_backfill_finished, person_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 { columns: [ auction_seller, $expr3, $expr4, count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 7 { columns: [ auction_seller, $expr3, $expr4, count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 7 { columns: [ vnode, id, auction_backfill_finished, auction_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 8 { columns: [ vnode, id, auction_backfill_finished, auction_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 4294967294 { columns: [ id, name, starttime, $expr2, auction.seller, $expr3, $expr4 ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ id, name, starttime, $expr2, auction.seller, $expr3, $expr4 ] + ├── primary key: [ $0 ASC, $2 ASC, $3 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 0, 2, 3 ] + └── read pk prefix len hint: 3 - id: nexmark_q9 before: @@ -2026,9 +2029,9 @@ SELECT COUNT(*) / COUNT(DISTINCT auction) FROM bid ) batch_plan: |- - BatchNestedLoopJoin { type: Inner, predicate: (count(bid.auction) >= $expr1), output: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } + BatchNestedLoopJoin { type: Inner, predicate: (count(bid.auction) >= $expr1), output: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } ├─BatchExchange { order: [], dist: Single } - │ └─BatchHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } + │ └─BatchHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction)] } │ └─BatchHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } │ ├─BatchExchange { order: [], dist: HashShard(auction.id) } │ │ └─BatchScan { table: auction, columns: [auction.id, auction.item_name], distribution: UpstreamHashShard(auction.id) } @@ -2043,9 +2046,9 @@ └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } - └─StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } - ├─StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } - │ └─StreamHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), count] } + └─StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } + ├─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } + │ └─StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } │ └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } │ ├─StreamExchange { dist: HashShard(auction.id) } │ │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } @@ -2063,50 +2066,50 @@ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } + └── StreamDynamicFilter { predicate: (count(bid.auction) >= $expr1), output: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } ├── left table: 0 ├── right table: 1 - ├── StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } - │ └── StreamHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), count] } - │ ├── intermediate state table: 3 - │ ├── state tables: [ 2 ] + ├── StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } + │ └── StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } + │ ├── intermediate state table: 2 + │ ├── state tables: [] │ ├── distinct tables: [] │ └── StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } - │ ├── left table: 4 - │ ├── right table: 6 - │ ├── left degree table: 5 - │ ├── right degree table: 7 + │ ├── left table: 3 + │ ├── right table: 5 + │ ├── left degree table: 4 + │ ├── right degree table: 6 │ ├── StreamExchange Hash([0]) from 1 │ └── StreamExchange Hash([0]) from 2 └── StreamExchange Broadcast from 3 Fragment 1 - Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 8 } + Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } ├── Upstream └── BatchPlanNode Fragment 2 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 9 } + Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 8 } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(bid.auction))) as $expr1] } - └── StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(bid.auction)), count] } { intermediate state table: 10, state tables: [], distinct tables: [] } + └── StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(bid.auction)), count] } { intermediate state table: 9, state tables: [], distinct tables: [] } └── StreamExchange Single from 4 Fragment 4 StreamStatelessSimpleAgg { aggs: [sum0(count), count(bid.auction)] } - └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } { intermediate state table: 11, state tables: [], distinct tables: [] } + └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } { intermediate state table: 10, state tables: [], distinct tables: [] } └── StreamExchange Hash([0]) from 5 Fragment 5 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 12 } + Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 11 } ├── Upstream └── BatchPlanNode Table 0 - ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction) ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction) ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0 ] @@ -2115,28 +2118,36 @@ Table 1 { columns: [ $expr1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } Table 2 - ├── columns: [ auction_id, auction_item_name, bid__row_id ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] - ├── value indices: [ 0, 1, 2 ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 3 - ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction), count ] + ├── columns: [ auction_id, auction_item_name ] ├── primary key: [ $0 ASC ] - ├── value indices: [ 1, 2, 3 ] + ├── value indices: [ 0, 1 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 4 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 5 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ bid_auction, bid__row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 + ├── columns: [ bid_auction, bid__row_id ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 7 { columns: [ bid_auction, bid__row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 + ├── columns: [ bid_auction, bid__row_id, _degree ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 8 + Table 7 ├── columns: [ vnode, id, auction_backfill_finished, auction_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] @@ -2144,7 +2155,7 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 9 + Table 8 ├── columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] @@ -2152,11 +2163,16 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 10 { columns: [ sum0(sum0(count)), sum0(count(bid_auction)), count ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + Table 9 + ├── columns: [ sum0(sum0(count)), sum0(count(bid_auction)), count ] + ├── primary key: [] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 - Table 11 { columns: [ bid_auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ bid_auction, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 + Table 11 ├── columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] @@ -2427,7 +2443,7 @@ BatchTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } └─BatchExchange { order: [], dist: Single } └─BatchTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } - └─BatchHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } + └─BatchHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction)] } └─BatchHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } ├─BatchExchange { order: [], dist: HashShard(auction.id) } │ └─BatchScan { table: auction, columns: [auction.id, auction.item_name], distribution: UpstreamHashShard(auction.id) } @@ -2435,12 +2451,12 @@ └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [bid_count, auction_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } + └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } └─StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), Vnode(auction.id) as $expr1] } - └─StreamHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), count] } + └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction), Vnode(auction.id) as $expr1] } + └─StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } ├─StreamExchange { dist: HashShard(auction.id) } │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } @@ -2450,44 +2466,44 @@ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [bid_count, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction)] } + └── StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } └── StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } { state table: 0 } └── StreamExchange Single from 1 Fragment 1 StreamGroupTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } { state table: 1 } - └── StreamProject { exprs: [auction.id, first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), Vnode(auction.id) as $expr1] } - └── StreamHashAgg { group_key: [auction.id], aggs: [first_value(auction.item_name order_by(auction.item_name ASC)), count(bid.auction), count] } - ├── intermediate state table: 3 - ├── state tables: [ 2 ] + └── StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction), Vnode(auction.id) as $expr1] } + └── StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } + ├── intermediate state table: 2 + ├── state tables: [] ├── distinct tables: [] └── StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } - ├── left table: 4 - ├── right table: 6 - ├── left degree table: 5 - ├── right degree table: 7 + ├── left table: 3 + ├── right table: 5 + ├── left degree table: 4 + ├── right degree table: 6 ├── StreamExchange Hash([0]) from 2 └── StreamExchange Hash([0]) from 3 Fragment 2 - Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 8 } + Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } ├── Upstream └── BatchPlanNode Fragment 3 - Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 9 } + Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 8 } ├── Upstream └── BatchPlanNode Table 0 - ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction), $expr1 ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), $expr1 ] ├── primary key: [ $2 DESC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [] └── read pk prefix len hint: 0 Table 1 - ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction), $expr1 ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), $expr1 ] ├── primary key: [ $3 ASC, $2 DESC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0 ] @@ -2495,38 +2511,31 @@ └── vnode column idx: 3 Table 2 - ├── columns: [ auction_id, auction_item_name, bid__row_id ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] - ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 - - Table 3 - ├── columns: [ auction_id, first_value(auction_item_name order_by(auction_item_name ASC)), count(bid_auction), count ] + ├── columns: [ auction_id, internal_last_seen_value(auction_item_name), count(bid_auction), count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 4 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ auction_id, auction_item_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ auction_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 + Table 5 ├── columns: [ bid_auction, bid__row_id ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 0, 1 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 7 + Table 6 ├── columns: [ bid_auction, bid__row_id, _degree ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 8 + Table 7 ├── columns: [ vnode, id, auction_backfill_finished, auction_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] @@ -2534,7 +2543,7 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 9 + Table 8 ├── columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3 ] diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 1ad1d9f92c418..003e4185ada81 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -582,29 +582,29 @@ create table t(x int[], y int[], k int primary key); select *, (select sum(i) from (select unnest(x) i, 1 c) Q where k = c ) as sum_x from t; optimized_logical_plan_for_batch: |- - LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t.x, first_value(t.x order_by(t.x ASC))) AND IsNotDistinctFrom(t.k, t.k), output: [t.x, t.y, t.k, sum(Unnest($0))] } + LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t.x, internal_last_seen_value(t.x)) AND IsNotDistinctFrom(t.k, t.k), output: [t.x, t.y, t.k, sum(Unnest($0))] } ├─LogicalScan { table: t, columns: [t.x, t.y, t.k] } - └─LogicalAgg { group_key: [first_value(t.x order_by(t.x ASC)), t.k], aggs: [sum(Unnest($0))] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(first_value(t.x order_by(t.x ASC)), first_value(t.x order_by(t.x ASC))) AND IsNotDistinctFrom(t.k, t.k), output: [first_value(t.x order_by(t.x ASC)), t.k, Unnest($0)] } - ├─LogicalAgg { group_key: [t.k], aggs: [first_value(t.x order_by(t.x ASC))] } + └─LogicalAgg { group_key: [internal_last_seen_value(t.x), t.k], aggs: [sum(Unnest($0))] } + └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(internal_last_seen_value(t.x), internal_last_seen_value(t.x)) AND IsNotDistinctFrom(t.k, t.k), output: [internal_last_seen_value(t.x), t.k, Unnest($0)] } + ├─LogicalAgg { group_key: [t.k], aggs: [internal_last_seen_value(t.x)] } │ └─LogicalScan { table: t, columns: [t.x, t.k] } - └─LogicalProject { exprs: [first_value(t.x order_by(t.x ASC)), t.k, Unnest($0)] } + └─LogicalProject { exprs: [internal_last_seen_value(t.x), t.k, Unnest($0)] } └─LogicalProjectSet { select_list: [$0, $1, Unnest($0)] } - └─LogicalJoin { type: Inner, on: true, output: [first_value(t.x order_by(t.x ASC)), t.k] } - ├─LogicalAgg { group_key: [t.k], aggs: [first_value(t.x order_by(t.x ASC))] } + └─LogicalJoin { type: Inner, on: true, output: [internal_last_seen_value(t.x), t.k] } + ├─LogicalAgg { group_key: [t.k], aggs: [internal_last_seen_value(t.x)] } │ └─LogicalScan { table: t, columns: [t.x, t.k], predicate: (t.k = 1:Int32) } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } optimized_logical_plan_for_stream: |- - LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t.x, first_value(t.x order_by(t.x ASC))) AND IsNotDistinctFrom(t.k, t.k), output: [t.x, t.y, t.k, sum(Unnest($0))] } + LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t.x, internal_last_seen_value(t.x)) AND IsNotDistinctFrom(t.k, t.k), output: [t.x, t.y, t.k, sum(Unnest($0))] } ├─LogicalScan { table: t, columns: [t.x, t.y, t.k] } - └─LogicalAgg { group_key: [first_value(t.x order_by(t.x ASC)), t.k], aggs: [sum(Unnest($0))] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(first_value(t.x order_by(t.x ASC)), first_value(t.x order_by(t.x ASC))) AND IsNotDistinctFrom(t.k, t.k), output: [first_value(t.x order_by(t.x ASC)), t.k, Unnest($0)] } - ├─LogicalAgg { group_key: [t.k], aggs: [first_value(t.x order_by(t.x ASC))] } + └─LogicalAgg { group_key: [internal_last_seen_value(t.x), t.k], aggs: [sum(Unnest($0))] } + └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(internal_last_seen_value(t.x), internal_last_seen_value(t.x)) AND IsNotDistinctFrom(t.k, t.k), output: [internal_last_seen_value(t.x), t.k, Unnest($0)] } + ├─LogicalAgg { group_key: [t.k], aggs: [internal_last_seen_value(t.x)] } │ └─LogicalScan { table: t, columns: [t.x, t.k] } - └─LogicalProject { exprs: [first_value(t.x order_by(t.x ASC)), t.k, Unnest($0)] } + └─LogicalProject { exprs: [internal_last_seen_value(t.x), t.k, Unnest($0)] } └─LogicalProjectSet { select_list: [$0, $1, Unnest($0)] } - └─LogicalJoin { type: Inner, on: true, output: [first_value(t.x order_by(t.x ASC)), t.k] } - ├─LogicalAgg { group_key: [t.k], aggs: [first_value(t.x order_by(t.x ASC))] } + └─LogicalJoin { type: Inner, on: true, output: [internal_last_seen_value(t.x), t.k] } + ├─LogicalAgg { group_key: [t.k], aggs: [internal_last_seen_value(t.x)] } │ └─LogicalScan { table: t, columns: [t.x, t.k], predicate: (t.k = 1:Int32) } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: CorrelatedInputRef in ProjectSet and apply on condition refers to table function. diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml index 6d216ad9c81c4..3b84def33abfa 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml @@ -466,14 +466,14 @@ └─LogicalScan { table: c, columns: [c.c1, c.c2, c.c3, c._row_id] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.a3, first_value(a.a3 order_by(a.a3 ASC))) AND IsNotDistinctFrom(b.b2, b.b2), output: [] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.a3, internal_last_seen_value(a.a3)) AND IsNotDistinctFrom(b.b2, b.b2), output: [] } ├─LogicalJoin { type: Inner, on: (a.a3 = b.b2), output: all } │ ├─LogicalScan { table: a, columns: [a.a3] } │ └─LogicalScan { table: b, columns: [b.b2] } └─LogicalFilter { predicate: (3:Int32 = count(1:Int32)) } - └─LogicalAgg { group_key: [first_value(a.a3 order_by(a.a3 ASC)), b.b2], aggs: [count(1:Int32)] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(first_value(a.a3 order_by(a.a3 ASC)), c.c3) AND IsNotDistinctFrom(b.b2, c.c2), output: [first_value(a.a3 order_by(a.a3 ASC)), b.b2, 1:Int32] } - ├─LogicalAgg { group_key: [b.b2], aggs: [first_value(a.a3 order_by(a.a3 ASC))] } + └─LogicalAgg { group_key: [internal_last_seen_value(a.a3), b.b2], aggs: [count(1:Int32)] } + └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(internal_last_seen_value(a.a3), c.c3) AND IsNotDistinctFrom(b.b2, c.c2), output: [internal_last_seen_value(a.a3), b.b2, 1:Int32] } + ├─LogicalAgg { group_key: [b.b2], aggs: [internal_last_seen_value(a.a3)] } │ └─LogicalJoin { type: Inner, on: (a.a3 = b.b2), output: all } │ ├─LogicalScan { table: a, columns: [a.a3] } │ └─LogicalScan { table: b, columns: [b.b2] } diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 2fb251ca89aa6..e0c7e339ee6a6 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -516,11 +516,8 @@ impl Agg { // we use materialized input state for non-retractable aggregate function. // for backward compatibility, the state type is same as the return type. // its values in the intermediate state table are always null. - } else { - field.data_type = sig - .state_type - .clone() - .unwrap_or(sig.ret_type.as_exact().clone()); + } else if let Some(state_type) = &sig.state_type { + field.data_type = state_type.clone(); } } let in_dist_key = self.input.distribution().dist_column_indices().to_vec(); diff --git a/src/frontend/src/optimizer/rule/agg_group_by_simplify_rule.rs b/src/frontend/src/optimizer/rule/agg_group_by_simplify_rule.rs index 34025eca43032..3e22348e27b49 100644 --- a/src/frontend/src/optimizer/rule/agg_group_by_simplify_rule.rs +++ b/src/frontend/src/optimizer/rule/agg_group_by_simplify_rule.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_expr::aggregate::AggKind; use super::super::plan_node::*; @@ -48,11 +47,11 @@ impl Rule for AggGroupBySimplifyRule { if !new_group_key.contains(i) { let data_type = agg_input.schema().fields[i].data_type(); new_agg_calls.push(PlanAggCall { - agg_kind: AggKind::FirstValue, + agg_kind: AggKind::InternalLastSeenValue, return_type: data_type.clone(), inputs: vec![InputRef::new(i, data_type)], distinct: false, - order_by: vec![ColumnOrder::new(i, OrderType::ascending())], + order_by: vec![], filter: Condition::true_cond(), direct_args: vec![], }); From 447e1422a2a0989d873ddd21f660460d0f0e26d2 Mon Sep 17 00:00:00 2001 From: Croxx Date: Tue, 24 Oct 2023 18:23:13 +0800 Subject: [PATCH 08/48] chore: bump foyer (#13029) Signed-off-by: MrCroxx --- Cargo.lock | 15 ++++++++------- src/storage/Cargo.toml | 2 +- src/storage/src/hummock/file_cache/store.rs | 1 + 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index bbc112fa4bb5b..ce71e2bf98950 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3015,7 +3015,7 @@ dependencies = [ [[package]] name = "foyer" version = "0.1.0" -source = "git+https://github.com/mrcroxx/foyer?rev=438eec8#438eec87e90c7a80cb53a06b711c6ea1ad7a0f41" +source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783" dependencies = [ "foyer-common", "foyer-intrusive", @@ -3026,10 +3026,11 @@ dependencies = [ [[package]] name = "foyer-common" version = "0.1.0" -source = "git+https://github.com/mrcroxx/foyer?rev=438eec8#438eec87e90c7a80cb53a06b711c6ea1ad7a0f41" +source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783" dependencies = [ "bytes", "foyer-workspace-hack", + "itertools 0.11.0", "madsim-tokio", "parking_lot 0.12.1", "paste", @@ -3040,13 +3041,13 @@ dependencies = [ [[package]] name = "foyer-intrusive" version = "0.1.0" -source = "git+https://github.com/mrcroxx/foyer?rev=438eec8#438eec87e90c7a80cb53a06b711c6ea1ad7a0f41" +source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783" dependencies = [ "bytes", "cmsketch", "foyer-common", "foyer-workspace-hack", - "itertools 0.10.5", + "itertools 0.11.0", "memoffset", "parking_lot 0.12.1", "paste", @@ -3057,7 +3058,7 @@ dependencies = [ [[package]] name = "foyer-storage" version = "0.1.0" -source = "git+https://github.com/mrcroxx/foyer?rev=438eec8#438eec87e90c7a80cb53a06b711c6ea1ad7a0f41" +source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783" dependencies = [ "anyhow", "async-channel", @@ -3086,7 +3087,7 @@ dependencies = [ [[package]] name = "foyer-workspace-hack" version = "0.1.0" -source = "git+https://github.com/mrcroxx/foyer?rev=438eec8#438eec87e90c7a80cb53a06b711c6ea1ad7a0f41" +source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783" dependencies = [ "crossbeam-utils", "either", @@ -3095,7 +3096,7 @@ dependencies = [ "futures-sink", "futures-util", "hyper", - "itertools 0.10.5", + "itertools 0.11.0", "libc", "memchr", "parking_lot 0.12.1", diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index f1022ab2fd935..c6fc5531acd33 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -25,7 +25,7 @@ dyn-clone = "1.0.14" either = "1" enum-as-inner = "0.6" fail = "0.5" -foyer = { git = "https://github.com/mrcroxx/foyer", rev = "438eec8" } +foyer = { git = "https://github.com/mrcroxx/foyer", rev = "5d0134b" } futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } hex = "0.4" diff --git a/src/storage/src/hummock/file_cache/store.rs b/src/storage/src/hummock/file_cache/store.rs index 9de54552ae077..222700c8376b2 100644 --- a/src/storage/src/hummock/file_cache/store.rs +++ b/src/storage/src/hummock/file_cache/store.rs @@ -256,6 +256,7 @@ where io_size: config.device_io_size, }, allocator_bits: config.allocator_bits, + catalog_bits: 6, admissions, reinsertions: config.reinsertions, buffer_pool_size: config.buffer_pool_size, From e81850870a8026fd41184344c01f71bc67a7cb5d Mon Sep 17 00:00:00 2001 From: Dylan Date: Tue, 24 Oct 2023 20:33:32 +0800 Subject: [PATCH 09/48] feat(optimizer): change stream join mv distribution key (#13022) --- .../tests/testdata/output/append_only.yaml | 11 +- .../tests/testdata/output/basic_query.yaml | 13 +- .../tests/testdata/output/ch_benchmark.yaml | 302 ++++++----- .../output/common_table_expressions.yaml | 22 +- .../testdata/output/distribution_derive.yaml | 110 ++-- .../tests/testdata/output/dynamic_filter.yaml | 21 +- .../tests/testdata/output/join.yaml | 133 ++--- .../tests/testdata/output/join_ordering.yaml | 108 ++-- .../testdata/output/lateral_subquery.yaml | 84 +-- .../tests/testdata/output/mv_on_mv.yaml | 11 +- .../tests/testdata/output/nexmark.yaml | 282 +++++----- .../tests/testdata/output/nexmark_source.yaml | 246 +++++---- .../output/nexmark_temporal_filter.yaml | 395 +++++++------- .../testdata/output/nexmark_watermark.yaml | 510 ++++++++++-------- .../testdata/output/over_window_function.yaml | 32 +- .../tests/testdata/output/pk_derive.yaml | 38 +- .../testdata/output/predicate_pushdown.yaml | 36 +- .../tests/testdata/output/project_set.yaml | 23 +- .../tests/testdata/output/select_except.yaml | 22 +- .../tests/testdata/output/share.yaml | 43 +- .../tests/testdata/output/shared_views.yaml | 33 +- .../tests/testdata/output/subquery.yaml | 179 +++--- .../output/subquery_expr_correlated.yaml | 74 +-- .../testdata/output/temporal_filter.yaml | 81 +-- .../tests/testdata/output/temporal_join.yaml | 171 +++--- .../tests/testdata/output/tpch_variant.yaml | 440 +++++++-------- .../tests/testdata/output/watermark.yaml | 52 +- .../tests/testdata/output/window_join.yaml | 38 +- .../optimizer/plan_node/stream_materialize.rs | 17 +- .../src/optimizer/property/distribution.rs | 2 +- .../scale/cascade_materialized_view.rs | 4 +- .../tests/integration_tests/scale/plan.rs | 8 +- 32 files changed, 1882 insertions(+), 1659 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/append_only.yaml b/src/frontend/planner_test/tests/testdata/output/append_only.yaml index 184abd564c32b..d693d3fc942df 100644 --- a/src/frontend/planner_test/tests/testdata/output/append_only.yaml +++ b/src/frontend/planner_test/tests/testdata/output/append_only.yaml @@ -14,11 +14,12 @@ select t1.v1 as id, v2, v3 from t1 join t2 on t1.v1=t2.v1; stream_plan: |- StreamMaterialize { columns: [id, v2, v3, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, id], pk_columns: [t1._row_id, t2._row_id, id], pk_conflict: NoCheck } - └─StreamHashJoin [append_only] { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v3, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.v1) } - └─StreamTableScan { table: t2, columns: [t2.v1, t2.v3, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) } + └─StreamHashJoin [append_only] { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v3, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v1) } + └─StreamTableScan { table: t2, columns: [t2.v1, t2.v3, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - sql: | create table t1 (v1 int, v2 int) append only; select v1 from t1 order by v1 limit 3 offset 3; diff --git a/src/frontend/planner_test/tests/testdata/output/basic_query.yaml b/src/frontend/planner_test/tests/testdata/output/basic_query.yaml index fde09972bb66b..ce6724dc91c37 100644 --- a/src/frontend/planner_test/tests/testdata/output/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/output/basic_query.yaml @@ -234,9 +234,10 @@ └─BatchValues { rows: [] } stream_plan: |- StreamMaterialize { columns: [v, t._row_id(hidden), t._row_id#1(hidden)], stream_key: [t._row_id, t._row_id#1, v], pk_columns: [t._row_id, t._row_id#1, v], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.v = t.v, output: [t.v, t._row_id, t._row_id] } - ├─StreamExchange { dist: HashShard(t.v) } - │ └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(t.v) } - └─StreamFilter { predicate: false:Boolean } - └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v, t._row_id, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.v = t.v, output: [t.v, t._row_id, t._row_id] } + ├─StreamExchange { dist: HashShard(t.v) } + │ └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v) } + └─StreamFilter { predicate: false:Boolean } + └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml index 6f4f8a673c996..e7196f7cf4fea 100644 --- a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml @@ -134,141 +134,145 @@ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } stream_plan: |- StreamMaterialize { columns: [s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id(hidden), stock.s_w_id(hidden), min(stock.s_quantity)(hidden), $expr2(hidden), region.r_regionkey(hidden), supplier.s_nationkey(hidden)], stream_key: [stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_columns: [n_name, s_name, i_id, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [item.i_id, item.i_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_i_id, stock.s_w_id, min(stock.s_quantity)] } - │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND min(stock.s_quantity) = stock.s_quantity AND stock.s_i_id = stock.s_i_id, output: [item.i_id, item.i_name, stock.s_i_id, stock.s_w_id, stock.s_i_id, min(stock.s_quantity)] } - │ ├─StreamProject { exprs: [stock.s_i_id, min(stock.s_quantity)] } - │ │ └─StreamHashAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity), count] } - │ │ └─StreamExchange { dist: HashShard(stock.s_i_id) } - │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } - │ │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] } - │ │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } - │ │ │ │ └─StreamProject { exprs: [region.r_regionkey] } - │ │ │ │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } - │ │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } - │ │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } - │ │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } - │ │ │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ │ │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ │ └─StreamExchange { dist: HashShard($expr1) } - │ │ └─StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } - │ │ └─StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) } - │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } - │ └─StreamHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all } - │ ├─StreamExchange { dist: HashShard(item.i_id) } - │ │ └─StreamProject { exprs: [item.i_id, item.i_name] } - │ │ └─StreamFilter { predicate: Like(item.i_data, '%b':Varchar) } - │ │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } - │ └─StreamExchange { dist: HashShard(stock.s_i_id) } - │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } - └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, supplier.s_nationkey] } - ├─StreamExchange { dist: HashShard(region.r_regionkey) } - │ └─StreamProject { exprs: [region.r_regionkey] } - │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } - │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } - └─StreamExchange { dist: HashShard(nation.n_regionkey) } - └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } - ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - └─StreamExchange { dist: HashShard(nation.n_nationkey) } - └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard(stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey) } + └─StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [item.i_id, item.i_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_i_id, stock.s_w_id, min(stock.s_quantity)] } + │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND min(stock.s_quantity) = stock.s_quantity AND stock.s_i_id = stock.s_i_id, output: [item.i_id, item.i_name, stock.s_i_id, stock.s_w_id, stock.s_i_id, min(stock.s_quantity)] } + │ ├─StreamProject { exprs: [stock.s_i_id, min(stock.s_quantity)] } + │ │ └─StreamHashAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity), count] } + │ │ └─StreamExchange { dist: HashShard(stock.s_i_id) } + │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } + │ │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] } + │ │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } + │ │ │ │ └─StreamProject { exprs: [region.r_regionkey] } + │ │ │ │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } + │ │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } + │ │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } + │ │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } + │ │ │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ │ │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ │ └─StreamExchange { dist: HashShard($expr1) } + │ │ └─StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } + │ │ └─StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) } + │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ └─StreamHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all } + │ ├─StreamExchange { dist: HashShard(item.i_id) } + │ │ └─StreamProject { exprs: [item.i_id, item.i_name] } + │ │ └─StreamFilter { predicate: Like(item.i_data, '%b':Varchar) } + │ │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } + │ └─StreamExchange { dist: HashShard(stock.s_i_id) } + │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, supplier.s_nationkey] } + ├─StreamExchange { dist: HashShard(region.r_regionkey) } + │ └─StreamProject { exprs: [region.r_regionkey] } + │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } + │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } + └─StreamExchange { dist: HashShard(nation.n_regionkey) } + └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } + ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(nation.n_nationkey) } + └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id(hidden), stock.s_w_id(hidden), min(stock.s_quantity)(hidden), $expr2(hidden), region.r_regionkey(hidden), supplier.s_nationkey(hidden)], stream_key: [stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_columns: [n_name, s_name, i_id, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([2]) from 1 - └── StreamExchange Hash([0]) from 11 + └── StreamExchange Hash([8, 9, 10, 11, 12, 13]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([2]) from 2 + └── StreamExchange Hash([0]) from 12 + + Fragment 2 StreamProject { exprs: [item.i_id, item.i_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_i_id, stock.s_w_id, min(stock.s_quantity)] } └── StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND min(stock.s_quantity) = stock.s_quantity AND stock.s_i_id = stock.s_i_id, output: [item.i_id, item.i_name, stock.s_i_id, stock.s_w_id, stock.s_i_id, min(stock.s_quantity)] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } ├── StreamProject { exprs: [stock.s_i_id, min(stock.s_quantity)] } │ └── StreamHashAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity), count] } { intermediate state table: 9, state tables: [ 8 ], distinct tables: [] } - │ └── StreamExchange Hash([0]) from 2 + │ └── StreamExchange Hash([0]) from 3 └── StreamHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all } { left table: 26, right table: 28, left degree table: 27, right degree table: 29 } - ├── StreamExchange Hash([0]) from 9 - └── StreamExchange Hash([0]) from 10 - - Fragment 2 - StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } { left table: 10, right table: 12, left degree table: 11, right degree table: 13 } - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([2]) from 8 + ├── StreamExchange Hash([0]) from 10 + └── StreamExchange Hash([0]) from 11 Fragment 3 - StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 } + StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } { left table: 10, right table: 12, left degree table: 11, right degree table: 13 } ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([1]) from 5 + └── StreamExchange Hash([2]) from 9 Fragment 4 + StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 } + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([1]) from 6 + + Fragment 5 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 18 } ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } { left table: 19, right table: 21, left degree table: 20, right degree table: 22 } - ├── StreamExchange Hash([1]) from 6 - └── StreamExchange Hash([0]) from 7 + ├── StreamExchange Hash([1]) from 7 + └── StreamExchange Hash([0]) from 8 - Fragment 6 + Fragment 7 Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 23 } ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 Chain { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 24 } ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } └── StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) } └── Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 25 } ├── Upstream └── BatchPlanNode - Fragment 9 + Fragment 10 StreamProject { exprs: [item.i_id, item.i_name] } └── StreamFilter { predicate: Like(item.i_data, '%b':Varchar) } └── Chain { table: item, columns: [item.i_id, item.i_name, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 30 } ├── Upstream └── BatchPlanNode - Fragment 10 + Fragment 11 Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 31 } ├── Upstream └── BatchPlanNode - Fragment 11 + Fragment 12 StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, supplier.s_nationkey] } { left table: 32, right table: 34, left degree table: 33, right degree table: 35 } - ├── StreamExchange Hash([0]) from 12 - └── StreamExchange Hash([6]) from 13 + ├── StreamExchange Hash([0]) from 13 + └── StreamExchange Hash([6]) from 14 - Fragment 12 + Fragment 13 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } └── Chain { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { state table: 36 } ├── Upstream └── BatchPlanNode - Fragment 13 + Fragment 14 StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, nation.n_regionkey, supplier.s_nationkey, nation.n_nationkey] } { left table: 37, right table: 39, left degree table: 38, right degree table: 40 } - ├── StreamExchange Hash([3]) from 14 - └── StreamExchange Hash([0]) from 15 + ├── StreamExchange Hash([3]) from 15 + └── StreamExchange Hash([0]) from 16 - Fragment 14 + Fragment 15 Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 41 } ├── Upstream └── BatchPlanNode - Fragment 15 + Fragment 16 Chain { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 42 } ├── Upstream └── BatchPlanNode @@ -359,7 +363,7 @@ Table 42 { columns: [ vnode, n_nationkey, nation_backfill_finished, nation_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $13 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 11 ], read pk prefix len hint: 9 } + Table 4294967294 { columns: [ s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $13 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 8, 9, 10, 11, 12, 13 ], read pk prefix len hint: 9 } - id: ch_q3 before: @@ -2496,59 +2500,63 @@ └─LogicalScan { table: revenue1, columns: [revenue1.total_revenue] } stream_plan: |- StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no(hidden)], stream_key: [s_suppkey, revenue1.supplier_no, total_revenue], pk_columns: [s_suppkey, revenue1.supplier_no, total_revenue], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } - ├─StreamExchange { dist: HashShard(revenue1.total_revenue) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } - │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard($expr1) } - │ └─StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no::Int64 as $expr1, revenue1.supplier_no] } - │ └─StreamTableScan { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } - └─StreamExchange { dist: HashShard(max(max(revenue1.total_revenue))) } - └─StreamProject { exprs: [max(max(revenue1.total_revenue))] } - └─StreamSimpleAgg { aggs: [max(max(revenue1.total_revenue)), count] } - └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr2], aggs: [max(revenue1.total_revenue), count] } - └─StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no, Vnode(revenue1.supplier_no) as $expr2] } - └─StreamTableScan { table: revenue1, columns: [revenue1.total_revenue, revenue1.supplier_no], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey, revenue1.total_revenue, revenue1.supplier_no) } + └─StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } + ├─StreamExchange { dist: HashShard(revenue1.total_revenue) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } + │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard($expr1) } + │ └─StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no::Int64 as $expr1, revenue1.supplier_no] } + │ └─StreamTableScan { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } + └─StreamExchange { dist: HashShard(max(max(revenue1.total_revenue))) } + └─StreamProject { exprs: [max(max(revenue1.total_revenue))] } + └─StreamSimpleAgg { aggs: [max(max(revenue1.total_revenue)), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [$expr2], aggs: [max(revenue1.total_revenue), count] } + └─StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no, Vnode(revenue1.supplier_no) as $expr2] } + └─StreamTableScan { table: revenue1, columns: [revenue1.total_revenue, revenue1.supplier_no], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no(hidden)], stream_key: [s_suppkey, revenue1.supplier_no, total_revenue], pk_columns: [s_suppkey, revenue1.supplier_no, total_revenue], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([4]) from 1 - └── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0, 4, 5]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([4]) from 2 + └── StreamExchange Hash([0]) from 5 + + Fragment 2 StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } ├── left table: 4 ├── right table: 6 ├── left degree table: 5 ├── right degree table: 7 - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([1]) from 3 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([1]) from 4 - Fragment 2 + Fragment 3 Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 8 } ├── Upstream └── BatchPlanNode - Fragment 3 + Fragment 4 StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no::Int64 as $expr1, revenue1.supplier_no] } └── Chain { table: revenue1, columns: [revenue1.supplier_no, revenue1.total_revenue], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } { state table: 9 } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [max(max(revenue1.total_revenue))] } └── StreamSimpleAgg { aggs: [max(max(revenue1.total_revenue)), count] } { intermediate state table: 11, state tables: [ 10 ], distinct tables: [] } - └── StreamExchange Single from 5 + └── StreamExchange Single from 6 - Fragment 5 + Fragment 6 StreamHashAgg { group_key: [$expr2], aggs: [max(revenue1.total_revenue), count] } { intermediate state table: 13, state tables: [ 12 ], distinct tables: [] } └── StreamProject { exprs: [revenue1.total_revenue, revenue1.supplier_no, Vnode(revenue1.supplier_no) as $expr2] } └── Chain { table: revenue1, columns: [revenue1.total_revenue, revenue1.supplier_no], pk: [revenue1.supplier_no], dist: UpstreamHashShard(revenue1.supplier_no) } { state table: 14 } @@ -2590,7 +2598,7 @@ Table 14 { columns: [ vnode, supplier_no, revenue1_backfill_finished, revenue1_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no ], primary key: [ $0 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 4 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no ], primary key: [ $0 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 4, 5 ], read pk prefix len hint: 3 } - id: ch_q16 before: @@ -3174,58 +3182,62 @@ └─BatchScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_delivery_d], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [supplier.s_suppkey, supplier.s_nationkey], pk_columns: [s_name, supplier.s_suppkey, supplier.s_nationkey], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } - ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } - │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ └─StreamProject { exprs: [nation.n_nationkey] } - │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamProject { exprs: [((stock.s_i_id * stock.s_w_id) % 10000:Int32)::Int64 as $expr1, stock.s_i_id, stock.s_w_id, stock.s_quantity] } - └─StreamFilter { predicate: ((2:Int32 * stock.s_quantity) > sum(order_line.ol_quantity)) } - └─StreamProject { exprs: [stock.s_i_id, stock.s_w_id, stock.s_quantity, sum(order_line.ol_quantity)] } - └─StreamHashAgg { group_key: [stock.s_i_id, stock.s_w_id, stock.s_quantity], aggs: [sum(order_line.ol_quantity), count] } - └─StreamHashJoin { type: LeftSemi, predicate: stock.s_i_id = item.i_id, output: all } - ├─StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, stock.s_quantity, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - │ ├─StreamExchange { dist: HashShard(stock.s_i_id) } - │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } - │ └─StreamExchange { dist: HashShard(order_line.ol_i_id) } - │ └─StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - │ └─StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } - │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } - └─StreamExchange { dist: HashShard(item.i_id) } - └─StreamProject { exprs: [item.i_id] } - └─StreamFilter { predicate: Like(item.i_data, 'co%':Varchar) } - └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey, supplier.s_nationkey) } + └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } + ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } + │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ └─StreamProject { exprs: [nation.n_nationkey] } + │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [((stock.s_i_id * stock.s_w_id) % 10000:Int32)::Int64 as $expr1, stock.s_i_id, stock.s_w_id, stock.s_quantity] } + └─StreamFilter { predicate: ((2:Int32 * stock.s_quantity) > sum(order_line.ol_quantity)) } + └─StreamProject { exprs: [stock.s_i_id, stock.s_w_id, stock.s_quantity, sum(order_line.ol_quantity)] } + └─StreamHashAgg { group_key: [stock.s_i_id, stock.s_w_id, stock.s_quantity], aggs: [sum(order_line.ol_quantity), count] } + └─StreamHashJoin { type: LeftSemi, predicate: stock.s_i_id = item.i_id, output: all } + ├─StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, stock.s_quantity, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + │ ├─StreamExchange { dist: HashShard(stock.s_i_id) } + │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ └─StreamExchange { dist: HashShard(order_line.ol_i_id) } + │ └─StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + │ └─StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } + │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └─StreamExchange { dist: HashShard(item.i_id) } + └─StreamProject { exprs: [item.i_id] } + └─StreamFilter { predicate: Like(item.i_data, 'co%':Varchar) } + └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [supplier.s_suppkey, supplier.s_nationkey], pk_columns: [s_name, supplier.s_suppkey, supplier.s_nationkey], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([2, 3]) from 1 Fragment 1 - StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } - ├── StreamExchange Hash([3]) from 2 - └── StreamExchange Hash([0]) from 3 + StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 5 Fragment 2 + StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } + ├── StreamExchange Hash([3]) from 3 + └── StreamExchange Hash([0]) from 4 + + Fragment 3 Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 8 } ├── Upstream └── BatchPlanNode - Fragment 3 + Fragment 4 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } └── Chain { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { state table: 9 } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [((stock.s_i_id * stock.s_w_id) % 10000:Int32)::Int64 as $expr1, stock.s_i_id, stock.s_w_id, stock.s_quantity] } └── StreamFilter { predicate: ((2:Int32 * stock.s_quantity) > sum(order_line.ol_quantity)) } └── StreamProject { exprs: [stock.s_i_id, stock.s_w_id, stock.s_quantity, sum(order_line.ol_quantity)] } @@ -3236,16 +3248,16 @@ │ ├── right table: 17 │ ├── left degree table: 16 │ ├── right degree table: 18 - │ ├── StreamExchange Hash([0]) from 5 - │ └── StreamExchange Hash([0]) from 6 - └── StreamExchange Hash([0]) from 7 + │ ├── StreamExchange Hash([0]) from 6 + │ └── StreamExchange Hash([0]) from 7 + └── StreamExchange Hash([0]) from 8 - Fragment 5 + Fragment 6 Chain { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 19 } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } └── Chain { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } @@ -3253,7 +3265,7 @@ ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamProject { exprs: [item.i_id] } └── StreamFilter { predicate: Like(item.i_data, 'co%':Varchar) } └── Chain { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } { state table: 21 } @@ -3304,7 +3316,7 @@ Table 21 { columns: [ vnode, i_id, item_backfill_finished, item_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2, 3 ], read pk prefix len hint: 3 } - id: ch_q21 before: diff --git a/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml b/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml index 9c961429276a3..dece27002b19b 100644 --- a/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml +++ b/src/frontend/planner_test/tests/testdata/output/common_table_expressions.yaml @@ -23,11 +23,12 @@ └─LogicalScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id] } stream_plan: |- StreamMaterialize { columns: [v3, v4, v1, t2._row_id(hidden), t1._row_id(hidden)], stream_key: [t2._row_id, t1._row_id, v3], pk_columns: [t2._row_id, t1._row_id, v3], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t2._row_id, t1._row_id] } - ├─StreamExchange { dist: HashShard(t2.v3) } - │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - └─StreamExchange { dist: HashShard(t1.v1) } - └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v3, t2._row_id, t1._row_id) } + └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t2._row_id, t1._row_id] } + ├─StreamExchange { dist: HashShard(t2.v3) } + │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1) } + └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | create table t1 (v1 int, v2 int); create table t2 (v3 int, v4 int); @@ -79,8 +80,9 @@ └─LogicalValues { rows: [['cn':Varchar, 'China':Varchar], ['us':Varchar, 'United States':Varchar]], schema: Schema { fields: [*VALUES*_0.column_0:Varchar, *VALUES*_0.column_1:Varchar] } } stream_plan: |- StreamMaterialize { columns: [v, c, abbr, real, t._row_id(hidden), _row_id(hidden)], stream_key: [t._row_id, _row_id, c], pk_columns: [t._row_id, _row_id, c], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.c = *VALUES*_0.column_0, output: [t.v, t.c, *VALUES*_0.column_0, *VALUES*_0.column_1, t._row_id, _row_id] } - ├─StreamExchange { dist: HashShard(t.c) } - │ └─StreamTableScan { table: t, columns: [t.v, t.c, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(*VALUES*_0.column_0) } - └─StreamValues { rows: [['cn':Varchar, 'China':Varchar, 0:Int64], ['us':Varchar, 'United States':Varchar, 1:Int64]] } + └─StreamExchange { dist: HashShard(t.c, t._row_id, _row_id) } + └─StreamHashJoin { type: Inner, predicate: t.c = *VALUES*_0.column_0, output: [t.v, t.c, *VALUES*_0.column_0, *VALUES*_0.column_1, t._row_id, _row_id] } + ├─StreamExchange { dist: HashShard(t.c) } + │ └─StreamTableScan { table: t, columns: [t.v, t.c, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(*VALUES*_0.column_0) } + └─StreamValues { rows: [['cn':Varchar, 'China':Varchar, 0:Int64], ['us':Varchar, 'United States':Varchar, 1:Int64]] } diff --git a/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml index 85d76188f3e76..818fd88b30a20 100644 --- a/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml @@ -963,31 +963,35 @@ └─BatchScan { table: a, columns: [a.k1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], stream_key: [ak1.a._row_id, ak1.k1], pk_columns: [ak1.a._row_id, ak1.k1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] } - ├─StreamExchange { dist: HashShard(ak1.k1) } - │ └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } - └─StreamHashAgg { group_key: [a.k1], aggs: [count] } - └─StreamExchange { dist: HashShard(a.k1) } - └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + └─StreamExchange { dist: HashShard(ak1.a._row_id, ak1.k1) } + └─StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] } + ├─StreamExchange { dist: HashShard(ak1.k1) } + │ └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } + └─StreamHashAgg { group_key: [a.k1], aggs: [count] } + └─StreamExchange { dist: HashShard(a.k1) } + └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], stream_key: [ak1.a._row_id, ak1.k1], pk_columns: [ak1.a._row_id, ak1.k1], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([2, 3]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 4 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -1022,7 +1026,7 @@ ├── columns: [ v, bv, ak1.a._row_id, ak1.k1, a.k1 ] ├── primary key: [ $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] - ├── distribution key: [ 3 ] + ├── distribution key: [ 2, 3 ] └── read pk prefix len hint: 2 - id: aggk1_join_Ak1_onk1 @@ -1054,31 +1058,35 @@ └─BatchScan { table: a, columns: [a.k1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden)], stream_key: [a.k1, ak1.a._row_id], pk_columns: [a.k1, ak1.a._row_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] } - ├─StreamHashAgg { group_key: [a.k1], aggs: [count] } - │ └─StreamExchange { dist: HashShard(a.k1) } - │ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } - └─StreamExchange { dist: HashShard(ak1.k1) } - └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } + └─StreamExchange { dist: HashShard(a.k1, ak1.a._row_id) } + └─StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] } + ├─StreamHashAgg { group_key: [a.k1], aggs: [count] } + │ └─StreamExchange { dist: HashShard(a.k1) } + │ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + └─StreamExchange { dist: HashShard(ak1.k1) } + └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [v, bv, a.k1(hidden), ak1.a._row_id(hidden)], stream_key: [a.k1, ak1.a._row_id], pk_columns: [a.k1, ak1.a._row_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 4, state tables: [], distinct tables: [] } - │ └── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([2, 3]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: a.k1 = ak1.k1, output: [ak1.v, count, a.k1, ak1.a._row_id] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 4, state tables: [], distinct tables: [] } + │ └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 5 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -1113,7 +1121,7 @@ ├── columns: [ v, bv, a.k1, ak1.a._row_id ] ├── primary key: [ $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3 ] - ├── distribution key: [ 2 ] + ├── distribution key: [ 2, 3 ] └── read pk prefix len hint: 2 - id: aggk1_join_aggk1_onk1 @@ -1156,33 +1164,37 @@ └─BatchScan { table: b, columns: [b.k1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], stream_key: [a.k1], pk_columns: [a.k1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } - ├─StreamHashAgg { group_key: [a.k1], aggs: [count] } - │ └─StreamExchange { dist: HashShard(a.k1) } - │ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } - └─StreamHashAgg { group_key: [b.k1], aggs: [count] } - └─StreamExchange { dist: HashShard(b.k1) } - └─StreamTableScan { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } + └─StreamExchange { dist: HashShard(a.k1) } + └─StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } + ├─StreamHashAgg { group_key: [a.k1], aggs: [count] } + │ └─StreamExchange { dist: HashShard(a.k1) } + │ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } + └─StreamHashAgg { group_key: [b.k1], aggs: [count] } + └─StreamExchange { dist: HashShard(b.k1) } + └─StreamTableScan { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], stream_key: [a.k1], pk_columns: [a.k1], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 4, state tables: [], distinct tables: [] } - │ └── StreamExchange Hash([0]) from 1 - └── StreamHashAgg { group_key: [b.k1], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 4, state tables: [], distinct tables: [] } + │ └── StreamExchange Hash([0]) from 2 + └── StreamHashAgg { group_key: [b.k1], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } { state table: 5 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } { state table: 7 } ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml b/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml index a340014298c47..ab282ebe3858a 100644 --- a/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/dynamic_filter.yaml @@ -124,16 +124,17 @@ └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: |- StreamMaterialize { columns: [v1, max, t1._row_id(hidden)], stream_key: [t1._row_id, v1], pk_columns: [t1._row_id, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v1 = max(max(t2.v2)), output: [t1.v1, max(max(t2.v2)), t1._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(max(max(t2.v2))) } - └─StreamProject { exprs: [max(max(t2.v2))] } - └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] } - └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [max(t2.v2), count] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } - └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v1 = max(max(t2.v2)), output: [t1.v1, max(max(t2.v2)), t1._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(max(max(t2.v2))) } + └─StreamProject { exprs: [max(max(t2.v2))] } + └─StreamSimpleAgg { aggs: [max(max(t2.v2)), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [$expr1], aggs: [max(t2.v2), count] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } + └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Dynamic filter join on unequal types sql: | create table t1 (v1 int); diff --git a/src/frontend/planner_test/tests/testdata/output/join.yaml b/src/frontend/planner_test/tests/testdata/output/join.yaml index 4ef01cc84ab47..a61d2a0d73327 100644 --- a/src/frontend/planner_test/tests/testdata/output/join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/join.yaml @@ -14,14 +14,15 @@ └─LogicalScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id] } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1, t3._row_id], pk_columns: [t1._row_id, t2._row_id, v1, t3._row_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v1 = t3.v5, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] } - ├─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } - │ ├─StreamExchange { dist: HashShard(t1.v1) } - │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ └─StreamExchange { dist: HashShard(t2.v3) } - │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - └─StreamExchange { dist: HashShard(t3.v5) } - └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id, t3._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v1 = t3.v5, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] } + ├─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } + │ ├─StreamExchange { dist: HashShard(t1.v1) } + │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: HashShard(t2.v3) } + │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t3.v5) } + └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } - name: self join sql: | create table t (v1 int, v2 int); @@ -33,11 +34,12 @@ └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } stream_plan: |- StreamMaterialize { columns: [t1v1, t2v1, t._row_id(hidden), t._row_id#1(hidden)], stream_key: [t._row_id, t._row_id#1, t1v1], pk_columns: [t._row_id, t._row_id#1, t1v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v1, t._row_id, t._row_id] } - ├─StreamExchange { dist: HashShard(t.v1) } - │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(t.v1) } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1, t._row_id, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v1, t._row_id, t._row_id] } + ├─StreamExchange { dist: HashShard(t.v1) } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1) } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t1 (v1 int, v2 int); create table t2 (v1 int, v2 int); @@ -65,15 +67,16 @@ └─BatchScan { table: t3, columns: [t3.v1, t3.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [t1_v1, t1_v2, t2_v1, t2_v2, t3_v1, t3_v2, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1, t3._row_id, t2_v2], pk_columns: [t1._row_id, t2._row_id, t1_v1, t3._row_id, t2_v2], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t2.v2 = t3.v2, output: [t1.v1, t1.v2, t2.v1, t2.v2, t3.v1, t3.v2, t1._row_id, t2._row_id, t3._row_id] } - ├─StreamExchange { dist: HashShard(t2.v2) } - │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } - │ ├─StreamExchange { dist: HashShard(t1.v1) } - │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ └─StreamExchange { dist: HashShard(t2.v1) } - │ └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - └─StreamExchange { dist: HashShard(t3.v2) } - └─StreamTableScan { table: t3, columns: [t3.v1, t3.v2, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t2.v2, t1._row_id, t2._row_id, t3._row_id) } + └─StreamHashJoin { type: Inner, predicate: t2.v2 = t3.v2, output: [t1.v1, t1.v2, t2.v1, t2.v2, t3.v1, t3.v2, t1._row_id, t2._row_id, t3._row_id] } + ├─StreamExchange { dist: HashShard(t2.v2) } + │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } + │ ├─StreamExchange { dist: HashShard(t1.v1) } + │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: HashShard(t2.v1) } + │ └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t3.v2) } + └─StreamTableScan { table: t3, columns: [t3.v1, t3.v2, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } - sql: | create table t1 (v1 int, v2 int); create table t2 (v1 int, v2 int); @@ -93,11 +96,12 @@ └─BatchScan { table: t2, columns: [t2.v1, t2.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [t1_v2, t2_v2, t1._row_id(hidden), t1.v1(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1.v1], pk_columns: [t1._row_id, t2._row_id, t1.v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v2, t2.v2, t1._row_id, t1.v1, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.v1) } - └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1._row_id, t1.v1, t2._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v1, output: [t1.v2, t2.v2, t1._row_id, t1.v1, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v1) } + └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - sql: | create table t1 (v1 int, v2 int); create table t2 (v1 int, v2 int); @@ -154,11 +158,12 @@ └─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) } stream_plan: |- StreamMaterialize { columns: [ix, iix, i.t._row_id(hidden), i.t._row_id#1(hidden)], stream_key: [i.t._row_id, i.t._row_id#1, ix], pk_columns: [i.t._row_id, i.t._row_id#1, ix], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id] } - ├─StreamExchange { dist: HashShard(i.x) } - │ └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } - └─StreamExchange { dist: HashShard(i.x) } - └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } + └─StreamExchange { dist: HashShard(i.x, i.t._row_id, i.t._row_id) } + └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id] } + ├─StreamExchange { dist: HashShard(i.x) } + │ └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } + └─StreamExchange { dist: HashShard(i.x) } + └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } - name: Left & right has same SomeShard distribution. There should still be exchanges below hash join sql: | create table t(x int); @@ -170,11 +175,12 @@ └─BatchScan { table: i, columns: [i.x], distribution: UpstreamHashShard(i.x) } stream_plan: |- StreamMaterialize { columns: [ix, tx, i.t._row_id(hidden), t._row_id(hidden)], stream_key: [i.t._row_id, t._row_id, ix], pk_columns: [i.t._row_id, t._row_id, ix], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: i.x = t.x, output: [i.x, t.x, i.t._row_id, t._row_id] } - ├─StreamExchange { dist: HashShard(i.x) } - │ └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } - └─StreamExchange { dist: HashShard(t.x) } - └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(i.x, i.t._row_id, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: i.x = t.x, output: [i.x, t.x, i.t._row_id, t._row_id] } + ├─StreamExchange { dist: HashShard(i.x) } + │ └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) } + └─StreamExchange { dist: HashShard(t.x) } + └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: Left & right has same HashShard distribution. There should be no exchange below hash join sql: | create table t(x int); @@ -628,12 +634,13 @@ └─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr1(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, $expr1], pk_columns: [t1._row_id, t2._row_id, $expr1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: $expr1 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr1, t2._row_id] } - ├─StreamExchange { dist: HashShard($expr1) } - │ └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.v2) } - └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1._row_id, $expr1, t2._row_id) } + └─StreamHashJoin { type: Inner, predicate: $expr1 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr1, t2._row_id] } + ├─StreamExchange { dist: HashShard($expr1) } + │ └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v2) } + └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Repeated columns in project should not interfere with join result (https://github.com/risingwavelabs/risingwave/issues/8216) sql: | create table t(x int); @@ -652,39 +659,43 @@ select t1.src p1, t1.dst p2, t2.dst p3 from t t1, t t2, t t3 where t1.dst = t2.src and t2.src = t3.dst and t3.dst = t1.src; stream_plan: |- StreamMaterialize { columns: [p1, p2, p3, t._row_id(hidden), t._row_id#1(hidden), t.src(hidden), t._row_id#2(hidden)], stream_key: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_columns: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] } - ├─StreamExchange { dist: HashShard(t.src) } - │ └─StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] } - │ ├─StreamExchange { dist: HashShard(t.dst) } - │ │ └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - │ └─StreamExchange { dist: HashShard(t.src) } - │ └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(t.dst) } - └─StreamTableScan { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.src, t.dst, t._row_id, t._row_id, t.src, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] } + ├─StreamExchange { dist: HashShard(t.src) } + │ └─StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] } + │ ├─StreamExchange { dist: HashShard(t.dst) } + │ │ └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + │ └─StreamExchange { dist: HashShard(t.src) } + │ └─StreamTableScan { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.dst) } + └─StreamTableScan { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [p1, p2, p3, t._row_id(hidden), t._row_id#1(hidden), t.src(hidden), t._row_id#2(hidden)], stream_key: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_columns: [t._row_id, t._row_id#1, p2, t._row_id#2, t.src, p1], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0, 1, 3, 4, 5, 6]) from 1 Fragment 1 - StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } - ├── StreamExchange Hash([1]) from 2 - └── StreamExchange Hash([0]) from 3 + StreamHashJoin { type: Inner, predicate: t.src = t.dst AND t.src = t.dst, output: [t.src, t.dst, t.dst, t._row_id, t._row_id, t.src, t._row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 5 Fragment 2 + StreamHashJoin { type: Inner, predicate: t.dst = t.src, output: [t.src, t.dst, t.src, t.dst, t._row_id, t._row_id] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } + ├── StreamExchange Hash([1]) from 3 + └── StreamExchange Hash([0]) from 4 + + Fragment 3 Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 8 } ├── Upstream └── BatchPlanNode - Fragment 3 + Fragment 4 Chain { table: t, columns: [t.src, t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 9 } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 Chain { table: t, columns: [t.dst, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } { state table: 10 } ├── Upstream └── BatchPlanNode @@ -711,5 +722,5 @@ Table 10 { columns: [ vnode, _row_id, t_backfill_finished, t_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ p1, p2, p3, t._row_id, t._row_id#1, t.src, t._row_id#2 ], primary key: [ $3 ASC, $4 ASC, $1 ASC, $6 ASC, $5 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 6 } + Table 4294967294 { columns: [ p1, p2, p3, t._row_id, t._row_id#1, t.src, t._row_id#2 ], primary key: [ $3 ASC, $4 ASC, $1 ASC, $6 ASC, $5 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 1, 3, 4, 5, 6 ], read pk prefix len hint: 6 } diff --git a/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml b/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml index b24fc18c6b513..31c53d02a9a18 100644 --- a/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml +++ b/src/frontend/planner_test/tests/testdata/output/join_ordering.yaml @@ -34,19 +34,20 @@ └─BatchScan { table: t4, columns: [t4.v7, t4.v8], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, v7, v8, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden), t4._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1, t3._row_id, v2, t4._row_id, v5], pk_columns: [t1._row_id, t2._row_id, v1, t3._row_id, v2, t4._row_id, v5], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] } - ├─StreamExchange { dist: HashShard(t3.v5) } - │ └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] } - │ ├─StreamExchange { dist: HashShard(t1.v2) } - │ │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } - │ │ ├─StreamExchange { dist: HashShard(t1.v1) } - │ │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ │ └─StreamExchange { dist: HashShard(t2.v3) } - │ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - │ └─StreamExchange { dist: HashShard(t3.v6) } - │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } - └─StreamExchange { dist: HashShard(t4.v7) } - └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1.v2, t3.v5, t1._row_id, t2._row_id, t3._row_id, t4._row_id) } + └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] } + ├─StreamExchange { dist: HashShard(t3.v5) } + │ └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t1._row_id, t2._row_id, t3._row_id] } + │ ├─StreamExchange { dist: HashShard(t1.v2) } + │ │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } + │ │ ├─StreamExchange { dist: HashShard(t1.v1) } + │ │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ └─StreamExchange { dist: HashShard(t2.v3) } + │ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + │ └─StreamExchange { dist: HashShard(t3.v6) } + │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } + └─StreamExchange { dist: HashShard(t4.v7) } + └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } - name: bushy tree join ordering sql: | create table t1 (v1 int, v2 int); @@ -81,19 +82,20 @@ └─BatchScan { table: t4, columns: [t4.v7, t4.v8], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, v7, v8, t2._row_id(hidden), t1._row_id(hidden), t4._row_id(hidden), t3._row_id(hidden)], stream_key: [t2._row_id, t1._row_id, v3, t4._row_id, t3._row_id, v7, v2], pk_columns: [t2._row_id, t1._row_id, v3, t4._row_id, t3._row_id, v7, v2], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t2._row_id, t1._row_id, t4._row_id, t3._row_id] } - ├─StreamExchange { dist: HashShard(t1.v2) } - │ └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t1.v2, t2._row_id, t1._row_id] } - │ ├─StreamExchange { dist: HashShard(t2.v3) } - │ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - │ └─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t3.v6) } - └─StreamHashJoin { type: Inner, predicate: t4.v7 = t3.v5, output: [t4.v7, t4.v8, t3.v5, t3.v6, t4._row_id, t3._row_id] } - ├─StreamExchange { dist: HashShard(t4.v7) } - │ └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } - └─StreamExchange { dist: HashShard(t3.v5) } - └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } + └─StreamExchange { dist: HashShard(t1.v2, t2.v3, t4.v7, t2._row_id, t1._row_id, t4._row_id, t3._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t2._row_id, t1._row_id, t4._row_id, t3._row_id] } + ├─StreamExchange { dist: HashShard(t1.v2) } + │ └─StreamHashJoin { type: Inner, predicate: t2.v3 = t1.v1, output: [t2.v3, t2.v4, t1.v1, t1.v2, t2._row_id, t1._row_id] } + │ ├─StreamExchange { dist: HashShard(t2.v3) } + │ │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + │ └─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t3.v6) } + └─StreamHashJoin { type: Inner, predicate: t4.v7 = t3.v5, output: [t4.v7, t4.v8, t3.v5, t3.v6, t4._row_id, t3._row_id] } + ├─StreamExchange { dist: HashShard(t4.v7) } + │ └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } + └─StreamExchange { dist: HashShard(t3.v5) } + └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } - name: bushy tree join ordering manually sql: | set rw_enable_join_ordering = false; @@ -128,19 +130,20 @@ └─BatchScan { table: t4, columns: [t4.v7, t4.v8], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, v7, v8, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden), t4._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1, t3._row_id, t4._row_id, v5, v2], pk_columns: [t1._row_id, t2._row_id, v1, t3._row_id, t4._row_id, v5, v2], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] } - ├─StreamExchange { dist: HashShard(t1.v2) } - │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } - │ ├─StreamExchange { dist: HashShard(t1.v1) } - │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ └─StreamExchange { dist: HashShard(t2.v3) } - │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - └─StreamExchange { dist: HashShard(t3.v6) } - └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t3.v5, t3.v6, t4.v7, t4.v8, t3._row_id, t4._row_id] } - ├─StreamExchange { dist: HashShard(t3.v5) } - │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } - └─StreamExchange { dist: HashShard(t4.v7) } - └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1.v2, t3.v5, t1._row_id, t2._row_id, t3._row_id, t4._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v2 = t3.v6, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] } + ├─StreamExchange { dist: HashShard(t1.v2) } + │ └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t1._row_id, t2._row_id] } + │ ├─StreamExchange { dist: HashShard(t1.v1) } + │ │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: HashShard(t2.v3) } + │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t3.v6) } + └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t3.v5, t3.v6, t4.v7, t4.v8, t3._row_id, t4._row_id] } + ├─StreamExchange { dist: HashShard(t3.v5) } + │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } + └─StreamExchange { dist: HashShard(t4.v7) } + └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } - name: right deep tree join ordering manually sql: | set rw_enable_join_ordering = false; @@ -175,16 +178,17 @@ └─BatchScan { table: t4, columns: [t4.v7, t4.v8], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, v7, v8, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden), t4._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t3._row_id, t4._row_id, v5, v4, v1], pk_columns: [t1._row_id, t2._row_id, t3._row_id, t4._row_id, v5, v4, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.v3) } - └─StreamHashJoin { type: Inner, predicate: t2.v4 = t3.v6, output: [t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t2._row_id, t3._row_id, t4._row_id] } - ├─StreamExchange { dist: HashShard(t2.v4) } - │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - └─StreamExchange { dist: HashShard(t3.v6) } - └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t3.v5, t3.v6, t4.v7, t4.v8, t3._row_id, t4._row_id] } - ├─StreamExchange { dist: HashShard(t3.v5) } - │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } - └─StreamExchange { dist: HashShard(t4.v7) } - └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t2.v4, t3.v5, t1._row_id, t2._row_id, t3._row_id, t4._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v3, output: [t1.v1, t1.v2, t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t1._row_id, t2._row_id, t3._row_id, t4._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v3) } + └─StreamHashJoin { type: Inner, predicate: t2.v4 = t3.v6, output: [t2.v3, t2.v4, t3.v5, t3.v6, t4.v7, t4.v8, t2._row_id, t3._row_id, t4._row_id] } + ├─StreamExchange { dist: HashShard(t2.v4) } + │ └─StreamTableScan { table: t2, columns: [t2.v3, t2.v4, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t3.v6) } + └─StreamHashJoin { type: Inner, predicate: t3.v5 = t4.v7, output: [t3.v5, t3.v6, t4.v7, t4.v8, t3._row_id, t4._row_id] } + ├─StreamExchange { dist: HashShard(t3.v5) } + │ └─StreamTableScan { table: t3, columns: [t3.v5, t3.v6, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } + └─StreamExchange { dist: HashShard(t4.v7) } + └─StreamTableScan { table: t4, columns: [t4.v7, t4.v8, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml b/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml index 8e63beb9798c1..85bfb1a6cda36 100644 --- a/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml @@ -42,22 +42,23 @@ └─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), all_sales._row_id(hidden), salesperson.id(hidden), all_sales.amount(hidden), salesperson.id#1(hidden)], stream_key: [salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount], pk_columns: [salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM salesperson.id AND all_sales.amount = max(all_sales.amount), output: [salesperson.name, max(all_sales.amount), all_sales.customer_name, salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount, salesperson.id] } - ├─StreamHashJoin { type: Inner, predicate: salesperson.id = all_sales.salesperson_id, output: [salesperson.id, salesperson.name, all_sales.customer_name, all_sales.amount, salesperson._row_id, all_sales._row_id] } - │ ├─StreamExchange { dist: HashShard(salesperson.id) } - │ │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } - │ └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } - │ └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } - └─StreamProject { exprs: [salesperson.id, max(all_sales.amount)] } - └─StreamHashAgg { group_key: [salesperson.id], aggs: [max(all_sales.amount), count] } - └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.id, all_sales.amount, all_sales._row_id] } - ├─StreamProject { exprs: [salesperson.id] } - │ └─StreamHashAgg { group_key: [salesperson.id], aggs: [count] } - │ └─StreamExchange { dist: HashShard(salesperson.id) } - │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } - └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } - └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) } - └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } + └─StreamExchange { dist: HashShard(salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount) } + └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM salesperson.id AND all_sales.amount = max(all_sales.amount), output: [salesperson.name, max(all_sales.amount), all_sales.customer_name, salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount, salesperson.id] } + ├─StreamHashJoin { type: Inner, predicate: salesperson.id = all_sales.salesperson_id, output: [salesperson.id, salesperson.name, all_sales.customer_name, all_sales.amount, salesperson._row_id, all_sales._row_id] } + │ ├─StreamExchange { dist: HashShard(salesperson.id) } + │ │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } + │ └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } + │ └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } + └─StreamProject { exprs: [salesperson.id, max(all_sales.amount)] } + └─StreamHashAgg { group_key: [salesperson.id], aggs: [max(all_sales.amount), count] } + └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.id, all_sales.amount, all_sales._row_id] } + ├─StreamProject { exprs: [salesperson.id] } + │ └─StreamHashAgg { group_key: [salesperson.id], aggs: [count] } + │ └─StreamExchange { dist: HashShard(salesperson.id) } + │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } + └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } + └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) } + └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } - name: lateral join 2 sql: | create table all_sales (salesperson_id int, customer_name varchar, amount int ); @@ -87,14 +88,15 @@ └─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), salesperson.id(hidden), all_sales.salesperson_id(hidden)], stream_key: [salesperson._row_id, salesperson.id], pk_columns: [salesperson._row_id, salesperson.id], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] } - ├─StreamExchange { dist: HashShard(salesperson.id) } - │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } - └─StreamGroupTopN { order: [all_sales.amount DESC], limit: 1, offset: 0, group_key: [all_sales.salesperson_id] } - └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } - └─StreamProject { exprs: [all_sales.salesperson_id, all_sales.amount, all_sales.customer_name, all_sales._row_id] } - └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) } - └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } + └─StreamExchange { dist: HashShard(salesperson._row_id, salesperson.id) } + └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] } + ├─StreamExchange { dist: HashShard(salesperson.id) } + │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } + └─StreamGroupTopN { order: [all_sales.amount DESC], limit: 1, offset: 0, group_key: [all_sales.salesperson_id] } + └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } + └─StreamProject { exprs: [all_sales.salesperson_id, all_sales.amount, all_sales.customer_name, all_sales._row_id] } + └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) } + └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } - name: lateral join 2 (left join) sql: | create table all_sales (salesperson_id int, customer_name varchar, amount int ); @@ -124,14 +126,15 @@ └─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), salesperson.id(hidden), all_sales.salesperson_id(hidden)], stream_key: [salesperson._row_id, salesperson.id], pk_columns: [salesperson._row_id, salesperson.id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] } - ├─StreamExchange { dist: HashShard(salesperson.id) } - │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } - └─StreamGroupTopN { order: [all_sales.amount DESC], limit: 1, offset: 0, group_key: [all_sales.salesperson_id] } - └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } - └─StreamProject { exprs: [all_sales.salesperson_id, all_sales.amount, all_sales.customer_name, all_sales._row_id] } - └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) } - └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } + └─StreamExchange { dist: HashShard(salesperson._row_id, salesperson.id) } + └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] } + ├─StreamExchange { dist: HashShard(salesperson.id) } + │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } + └─StreamGroupTopN { order: [all_sales.amount DESC], limit: 1, offset: 0, group_key: [all_sales.salesperson_id] } + └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } + └─StreamProject { exprs: [all_sales.salesperson_id, all_sales.amount, all_sales.customer_name, all_sales._row_id] } + └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) } + └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } - name: lateral join 2 (right join) should throw an error sql: | create table all_sales (salesperson_id int, customer_name varchar, amount int ); @@ -165,14 +168,15 @@ └─BatchScan { table: t, columns: [t.arr], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, arr, unnest, t._row_id(hidden), t.arr(hidden), projected_row_id(hidden)], stream_key: [t._row_id, projected_row_id, arr], pk_columns: [t._row_id, projected_row_id, arr], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, Unnest($0), t._row_id, t.arr, projected_row_id] } - ├─StreamExchange { dist: HashShard(t.arr) } - │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamProjectSet { select_list: [$0, Unnest($0)] } - └─StreamProject { exprs: [t.arr] } - └─StreamHashAgg { group_key: [t.arr], aggs: [count] } - └─StreamExchange { dist: HashShard(t.arr) } - └─StreamTableScan { table: t, columns: [t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.arr, t._row_id, projected_row_id) } + └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, Unnest($0), t._row_id, t.arr, projected_row_id] } + ├─StreamExchange { dist: HashShard(t.arr) } + │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProjectSet { select_list: [$0, Unnest($0)] } + └─StreamProject { exprs: [t.arr] } + └─StreamHashAgg { group_key: [t.arr], aggs: [count] } + └─StreamExchange { dist: HashShard(t.arr) } + └─StreamTableScan { table: t, columns: [t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: https://github.com/risingwavelabs/risingwave/issues/12298 sql: | create table t1(c varchar, n varchar, id varchar, d varchar); diff --git a/src/frontend/planner_test/tests/testdata/output/mv_on_mv.yaml b/src/frontend/planner_test/tests/testdata/output/mv_on_mv.yaml index 2f7d9e5e75b3b..6838ddb331939 100644 --- a/src/frontend/planner_test/tests/testdata/output/mv_on_mv.yaml +++ b/src/frontend/planner_test/tests/testdata/output/mv_on_mv.yaml @@ -12,8 +12,9 @@ select m1.v1 as m1v1, m1.v2 as m1v2, m2.v1 as m2v1, m2.v2 as m2v2 from m1 join m2 on m1.v1 = m2.v1; stream_plan: |- StreamMaterialize { columns: [m1v1, m1v2, m2v1, m2v2, m1.t1._row_id(hidden), m2.t1._row_id(hidden)], stream_key: [m1.t1._row_id, m2.t1._row_id, m1v1], pk_columns: [m1.t1._row_id, m2.t1._row_id, m1v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: m1.v1 = m2.v1, output: [m1.v1, m1.v2, m2.v1, m2.v2, m1.t1._row_id, m2.t1._row_id] } - ├─StreamExchange { dist: HashShard(m1.v1) } - │ └─StreamTableScan { table: m1, columns: [m1.v1, m1.v2, m1.t1._row_id], pk: [m1.t1._row_id], dist: UpstreamHashShard(m1.t1._row_id) } - └─StreamExchange { dist: HashShard(m2.v1) } - └─StreamTableScan { table: m2, columns: [m2.v1, m2.v2, m2.t1._row_id], pk: [m2.t1._row_id], dist: UpstreamHashShard(m2.t1._row_id) } + └─StreamExchange { dist: HashShard(m1.v1, m1.t1._row_id, m2.t1._row_id) } + └─StreamHashJoin { type: Inner, predicate: m1.v1 = m2.v1, output: [m1.v1, m1.v2, m2.v1, m2.v2, m1.t1._row_id, m2.t1._row_id] } + ├─StreamExchange { dist: HashShard(m1.v1) } + │ └─StreamTableScan { table: m1, columns: [m1.v1, m1.v2, m1.t1._row_id], pk: [m1.t1._row_id], dist: UpstreamHashShard(m1.t1._row_id) } + └─StreamExchange { dist: HashShard(m2.v1) } + └─StreamTableScan { table: m2, columns: [m2.v1, m2.v2, m2.t1._row_id], pk: [m2.t1._row_id], dist: UpstreamHashShard(m2.t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index 2f23b0674fa84..8d452bf45bc36 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -181,34 +181,38 @@ └─BatchScan { table: auction, columns: [auction.id, auction.seller, auction.category], distribution: UpstreamHashShard(auction.id) } stream_plan: |- StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], stream_key: [id, auction.seller], pk_columns: [id, auction.seller], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] } - ├─StreamExchange { dist: HashShard(auction.seller) } - │ └─StreamProject { exprs: [auction.id, auction.seller] } - │ └─StreamFilter { predicate: (auction.category = 10:Int32) } - │ └─StreamTableScan { table: auction, columns: [auction.id, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } - └─StreamExchange { dist: HashShard(person.id) } - └─StreamFilter { predicate: (((person.state = 'or':Varchar) OR (person.state = 'id':Varchar)) OR (person.state = 'ca':Varchar)) } - └─StreamTableScan { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) } + └─StreamExchange { dist: HashShard(auction.id, auction.seller) } + └─StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] } + ├─StreamExchange { dist: HashShard(auction.seller) } + │ └─StreamProject { exprs: [auction.id, auction.seller] } + │ └─StreamFilter { predicate: (auction.category = 10:Int32) } + │ └─StreamTableScan { table: auction, columns: [auction.id, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + └─StreamExchange { dist: HashShard(person.id) } + └─StreamFilter { predicate: (((person.state = 'or':Varchar) OR (person.state = 'id':Varchar)) OR (person.state = 'ca':Varchar)) } + └─StreamTableScan { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], stream_key: [id, auction.seller], pk_columns: [id, auction.seller], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([1]) from 1 - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([3, 4]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([1]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 StreamProject { exprs: [auction.id, auction.seller] } └── StreamFilter { predicate: (auction.category = 10:Int32) } └── Chain { table: auction, columns: [auction.id, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 4 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 StreamFilter { predicate: (((person.state = 'or':Varchar) OR (person.state = 'id':Varchar)) OR (person.state = 'ca':Varchar)) } └── Chain { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) } { state table: 5 } ├── Upstream @@ -242,7 +246,7 @@ ├── columns: [ name, city, state, id, auction.seller, person.id ] ├── primary key: [ $3 ASC, $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5 ] - ├── distribution key: [ 4 ] + ├── distribution key: [ 3, 4 ] └── read pk prefix len hint: 2 - id: nexmark_q4 @@ -847,34 +851,38 @@ └─BatchScan { table: auction, columns: [auction.date_time, auction.seller], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2], pk_columns: [id, starttime, $expr2], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] } - ├─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) } - │ └─StreamProject { exprs: [person.id, $expr1, $expr2, internal_last_seen_value(person.name)] } - │ └─StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name), count] } - │ └─StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } - │ └─StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] } - │ └─StreamTableScan { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } - └─StreamProject { exprs: [auction.seller, $expr3, $expr4] } - └─StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } - └─StreamExchange { dist: HashShard(auction.seller, $expr3, $expr4) } - └─StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] } - └─StreamProject { exprs: [auction.date_time, auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, auction.id] } - └─StreamTableScan { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + └─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) } + └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] } + ├─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) } + │ └─StreamProject { exprs: [person.id, $expr1, $expr2, internal_last_seen_value(person.name)] } + │ └─StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name), count] } + │ └─StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } + │ └─StreamProject { exprs: [person.id, person.name, person.date_time, TumbleStart(person.date_time, '00:00:10':Interval) as $expr1] } + │ └─StreamTableScan { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } + └─StreamProject { exprs: [auction.seller, $expr3, $expr4] } + └─StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } + └─StreamExchange { dist: HashShard(auction.seller, $expr3, $expr4) } + └─StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] } + └─StreamProject { exprs: [auction.date_time, auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, auction.id] } + └─StreamTableScan { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2], pk_columns: [id, starttime, $expr2], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0, 1, 2]) from 1 - └── StreamProject { exprs: [auction.seller, $expr3, $expr4] } - └── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0, 1, 2]) from 2 + └── StreamExchange Hash([0, 2, 3]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, internal_last_seen_value(person.name), $expr1, $expr2, auction.seller, $expr3, $expr4] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0, 1, 2]) from 2 + └── StreamProject { exprs: [auction.seller, $expr3, $expr4] } + └── StreamHashAgg { group_key: [auction.seller, $expr3, $expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0, 1, 2]) from 3 + + Fragment 2 StreamProject { exprs: [person.id, $expr1, $expr2, internal_last_seen_value(person.name)] } └── StreamHashAgg { group_key: [person.id, $expr1, $expr2], aggs: [internal_last_seen_value(person.name), count] } { intermediate state table: 4, state tables: [], distinct tables: [] } └── StreamProject { exprs: [person.id, person.name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } @@ -883,7 +891,7 @@ ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 StreamProject { exprs: [auction.seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4, auction.id] } └── StreamProject { exprs: [auction.date_time, auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, auction.id] } └── Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 7 } @@ -1133,27 +1141,31 @@ └─StreamTableScan { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr1(hidden), side_input.key(hidden)], stream_key: [bid._row_id, $expr1], pk_columns: [bid._row_id, $expr1], pk_conflict: NoCheck } - └─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] } - │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(side_input.key) } - └─StreamTableScan { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) } + └─StreamExchange { dist: HashShard(bid._row_id, $expr1) } + └─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] } + │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(side_input.key) } + └─StreamTableScan { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr1(hidden), side_input.key(hidden)], stream_key: [bid._row_id, $expr1], pk_columns: [bid._row_id, $expr1], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── 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 Hash([4]) from 1 - └── StreamExchange NoShuffle from 2 + └── StreamExchange Hash([5, 6]) from 1 Fragment 1 + 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 Hash([4]) from 2 + └── StreamExchange NoShuffle from 3 + + Fragment 2 StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, (bid.auction % 10000:Int32) as $expr1, bid._row_id] } └── Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 0 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) } { state table: 1 } ├── Upstream └── BatchPlanNode @@ -1166,7 +1178,7 @@ ├── columns: [ auction, bidder, price, date_time, value, bid._row_id, $expr1, side_input.key ] ├── primary key: [ $5 ASC, $6 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] - ├── distribution key: [ 6 ] + ├── distribution key: [ 5, 6 ] └── read pk prefix len hint: 2 - id: nexmark_q14 @@ -1795,30 +1807,34 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], stream_key: [bid._row_id, auction], pk_columns: [bid._row_id, auction], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] } - ├─StreamExchange { dist: HashShard(bid.auction) } - │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } - └─StreamExchange { dist: HashShard(auction.id) } - └─StreamFilter { predicate: (auction.category = 10:Int32) } - └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + └─StreamExchange { dist: HashShard(bid.auction, bid._row_id) } + └─StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] } + ├─StreamExchange { dist: HashShard(bid.auction) } + │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamExchange { dist: HashShard(auction.id) } + └─StreamFilter { predicate: (auction.category = 10:Int32) } + └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], stream_key: [bid._row_id, auction], pk_columns: [bid._row_id, auction], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0, 14]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 4 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 StreamFilter { predicate: (auction.category = 10:Int32) } └── Chain { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 5 } ├── Upstream @@ -1840,7 +1856,7 @@ ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id, auction.id ] ├── primary key: [ $14 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 14 ] └── read pk prefix len hint: 2 - id: nexmark_q21 @@ -1946,33 +1962,37 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.price], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] } - ├─StreamExchange { dist: HashShard(auction.id) } - │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } - └─StreamProject { exprs: [bid.auction, max(bid.price)] } - └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [max(bid.price), count] } - └─StreamExchange { dist: HashShard(bid.auction) } - └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamExchange { dist: HashShard(auction.id) } + └─StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] } + ├─StreamExchange { dist: HashShard(auction.id) } + │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + └─StreamProject { exprs: [bid.auction, max(bid.price)] } + └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [max(bid.price), count] } + └─StreamExchange { dist: HashShard(bid.auction) } + └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [bid.auction, max(bid.price)] } - └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [max(bid.price), count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [bid.auction, max(bid.price)] } + └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [max(bid.price), count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } { state table: 4 } ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -2215,39 +2235,43 @@ └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all } - ├─StreamExchange { dist: HashShard(auction.id) } - │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } - └─StreamProject { exprs: [bid.auction] } - └─StreamFilter { predicate: (count >= 20:Int32) } - └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } - └─StreamExchange { dist: HashShard(bid.auction) } - └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamExchange { dist: HashShard(auction.id) } + └─StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all } + ├─StreamExchange { dist: HashShard(auction.id) } + │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + └─StreamProject { exprs: [bid.auction] } + └─StreamFilter { predicate: (count >= 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } + └─StreamExchange { dist: HashShard(bid.auction) } + └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [bid.auction] } - └── StreamFilter { predicate: (count >= 20:Int32) } - └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } - ├── intermediate state table: 5 - ├── state tables: [] - ├── distinct tables: [] - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamHashJoin { type: LeftSemi, predicate: auction.id = bid.auction, output: all } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [bid.auction] } + └── StreamFilter { predicate: (count >= 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } + ├── intermediate state table: 5 + ├── state tables: [] + ├── distinct tables: [] + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── state table: 4 ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode @@ -2333,39 +2357,43 @@ └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all } - ├─StreamExchange { dist: HashShard(auction.id) } - │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } - └─StreamProject { exprs: [bid.auction] } - └─StreamFilter { predicate: (count < 20:Int32) } - └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } - └─StreamExchange { dist: HashShard(bid.auction) } - └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamExchange { dist: HashShard(auction.id) } + └─StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all } + ├─StreamExchange { dist: HashShard(auction.id) } + │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } + └─StreamProject { exprs: [bid.auction] } + └─StreamFilter { predicate: (count < 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } + └─StreamExchange { dist: HashShard(bid.auction) } + └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [bid.auction] } - └── StreamFilter { predicate: (count < 20:Int32) } - └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } - ├── intermediate state table: 5 - ├── state tables: [] - ├── distinct tables: [] - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamHashJoin { type: LeftAnti, predicate: auction.id = bid.auction, output: all } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [bid.auction] } + └── StreamFilter { predicate: (count < 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [bid.auction], aggs: [count] } + ├── intermediate state table: 5 + ├── state tables: [] + ├── distinct tables: [] + └── StreamExchange Hash([0]) from 3 + + Fragment 2 Chain { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } ├── state table: 4 ├── Upstream └── BatchPlanNode - Fragment 2 + Fragment 3 Chain { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } { state table: 6 } ├── Upstream └── BatchPlanNode 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 7c694fad1fa67..31be64b2c480a 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml @@ -158,29 +158,33 @@ └─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 } - └─StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } - ├─StreamExchange { dist: HashShard(seller) } - │ └─StreamFilter { predicate: (category = 10:Int32) } - │ └─StreamRowIdGen { row_id_index: 10 } - │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } - └─StreamExchange { dist: HashShard(id) } - └─StreamFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } - └─StreamRowIdGen { row_id_index: 8 } - └─StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } + └─StreamExchange { dist: HashShard(_row_id, seller, _row_id) } + └─StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } + ├─StreamExchange { dist: HashShard(seller) } + │ └─StreamFilter { predicate: (category = 10:Int32) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + └─StreamExchange { dist: HashShard(id) } + └─StreamFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } + └─StreamRowIdGen { row_id_index: 8 } + └─StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } stream_dist_plan: |+ Fragment 0 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 } ├── materialized table: 4294967294 - └── StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([7]) from 1 - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([4, 5, 6]) from 1 Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([7]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 StreamFilter { predicate: (category = 10:Int32) } └── StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } { source state table: 4 } - Fragment 2 + Fragment 3 StreamFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } └── StreamRowIdGen { row_id_index: 8 } └── StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } { source state table: 5 } @@ -211,7 +215,7 @@ ├── columns: [ name, city, state, id, _row_id, seller, _row_id#1 ] ├── primary key: [ $4 ASC, $6 ASC, $5 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] - ├── distribution key: [ 5 ] + ├── distribution key: [ 4, 5, 6 ] └── read pk prefix len hint: 3 - id: nexmark_q4 @@ -737,40 +741,44 @@ └─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: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, name, starttime, $expr2], pk_columns: [id, name, starttime, $expr2], pk_conflict: NoCheck } - └─StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } - ├─StreamExchange { dist: HashShard(id, $expr1, $expr2) } - │ └─StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } - │ └─StreamExchange { dist: HashShard(id, name, $expr1, $expr2) } - │ └─StreamProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } - │ └─StreamProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } - │ └─StreamRowIdGen { row_id_index: 8 } - │ └─StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } - └─StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } - └─StreamExchange { dist: HashShard(seller, $expr3, $expr4) } - └─StreamProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] } - └─StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } - └─StreamRowIdGen { row_id_index: 10 } - └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + └─StreamExchange { dist: HashShard(id, name, $expr1, $expr2) } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } + ├─StreamExchange { dist: HashShard(id, $expr1, $expr2) } + │ └─StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } + │ └─StreamExchange { dist: HashShard(id, name, $expr1, $expr2) } + │ └─StreamProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } + │ └─StreamProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } + │ └─StreamRowIdGen { row_id_index: 8 } + │ └─StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } + └─StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } + └─StreamExchange { dist: HashShard(seller, $expr3, $expr4) } + └─StreamProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] } + └─StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, name, starttime, $expr2], pk_columns: [id, name, starttime, $expr2], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0, 2, 3]) from 1 - └── StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } { state table: 6 } - └── StreamExchange Hash([0, 1, 2]) from 3 + └── StreamExchange Hash([0, 1, 2, 3]) from 1 Fragment 1 - StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } { state table: 4 } - └── StreamExchange Hash([0, 1, 2, 3]) from 2 + StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0, 2, 3]) from 2 + └── StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } { state table: 6 } + └── StreamExchange Hash([0, 1, 2]) from 4 Fragment 2 + StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } { state table: 4 } + └── StreamExchange Hash([0, 1, 2, 3]) from 3 + + Fragment 3 StreamProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } └── StreamProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } └── StreamRowIdGen { row_id_index: 8 } └── StreamSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] } └── StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } └── StreamRowIdGen { row_id_index: 10 } @@ -796,7 +804,7 @@ ├── columns: [ id, name, starttime, $expr2, seller, $expr3, $expr4 ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] - ├── distribution key: [ 0, 2, 3 ] + ├── distribution key: [ 0, 1, 2, 3 ] └── read pk prefix len hint: 4 - id: nexmark_q9 @@ -1629,31 +1637,31 @@ └─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 } - └─StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(auction) } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } - └─StreamExchange { dist: HashShard(id) } - └─StreamFilter { predicate: (category = 10:Int32) } - └─StreamRowIdGen { row_id_index: 10 } - └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + └─StreamExchange { dist: HashShard(auction, _row_id, _row_id) } + └─StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(auction) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamExchange { dist: HashShard(id) } + └─StreamFilter { predicate: (category = 10:Int32) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_dist_plan: |+ Fragment 0 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 } ├── materialized table: 4294967294 - └── StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0, 14, 15]) from 1 Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 4 } - Fragment 2 + Fragment 3 StreamFilter { predicate: (category = 10:Int32) } └── StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } { source state table: 5 } @@ -1674,7 +1682,7 @@ ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ] ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 14, 15 ] └── read pk prefix len hint: 3 - id: nexmark_q21 @@ -1775,30 +1783,34 @@ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } - ├─StreamExchange { dist: HashShard(id) } - │ └─StreamRowIdGen { row_id_index: 10 } - │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } - └─StreamProject { exprs: [auction, max(price)] } - └─StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } - └─StreamExchange { dist: HashShard(auction) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamExchange { dist: HashShard(id, _row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + └─StreamProject { exprs: [auction, max(price)] } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [auction, max(price)] } - └── StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0, 3]) from 1 Fragment 1 + StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [auction, max(price)] } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } { source state table: 4 } - Fragment 2 + Fragment 3 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 6 } @@ -1825,7 +1837,7 @@ ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, auction ] ├── primary key: [ $3 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 3 ] └── read pk prefix len hint: 2 - id: nexmark_q102 @@ -1992,37 +2004,41 @@ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] } - ├─StreamExchange { dist: HashShard(id) } - │ └─StreamRowIdGen { row_id_index: 10 } - │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } - └─StreamProject { exprs: [auction] } - └─StreamFilter { predicate: (count >= 20:Int32) } - └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } - └─StreamExchange { dist: HashShard(auction) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamExchange { dist: HashShard(id, _row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + └─StreamProject { exprs: [auction] } + └─StreamFilter { predicate: (count >= 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [auction] } - └── StreamFilter { predicate: (count >= 20:Int32) } - └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0, 2]) from 1 Fragment 1 + StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [auction] } + └── StreamFilter { predicate: (count >= 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └── source state table: 4 - Fragment 2 + Fragment 3 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 6 } @@ -2049,7 +2065,7 @@ ├── columns: [ auction_id, auction_item_name, _row_id ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 2 ] └── read pk prefix len hint: 2 - id: nexmark_q104 @@ -2080,37 +2096,41 @@ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] } - ├─StreamExchange { dist: HashShard(id) } - │ └─StreamRowIdGen { row_id_index: 10 } - │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } - └─StreamProject { exprs: [auction] } - └─StreamFilter { predicate: (count < 20:Int32) } - └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } - └─StreamExchange { dist: HashShard(auction) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamExchange { dist: HashShard(id, _row_id) } + └─StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + └─StreamProject { exprs: [auction] } + └─StreamFilter { predicate: (count < 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [auction] } - └── StreamFilter { predicate: (count < 20:Int32) } - └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0, 2]) from 1 Fragment 1 + StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [auction] } + └── StreamFilter { predicate: (count < 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └── source state table: 4 - Fragment 2 + Fragment 3 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 6 } @@ -2137,7 +2157,7 @@ ├── columns: [ auction_id, auction_item_name, _row_id ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 2 ] └── read pk prefix len hint: 2 - id: nexmark_q105 diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml index ccdde39e76764..c6c3ffd4f5ad6 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml @@ -717,65 +717,69 @@ AND P.endtime = A.endtime; stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr6(hidden), $expr8(hidden), $expr9(hidden), $expr10(hidden)], stream_key: [id, name, starttime, $expr6], pk_columns: [id, name, starttime, $expr6], pk_conflict: NoCheck } - └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all } - ├─StreamExchange { dist: HashShard($expr2, $expr5, $expr6) } - │ └─StreamAppendOnlyDedup { dedup_cols: [$expr2, $expr3, $expr5, $expr6] } - │ └─StreamExchange { dist: HashShard($expr2, $expr3, $expr5, $expr6) } - │ └─StreamProject { exprs: [$expr2, $expr3, $expr5, ($expr5 + '00:00:10':Interval) as $expr6] } - │ └─StreamProject { exprs: [$expr2, $expr3, $expr4, TumbleStart($expr4, '00:00:10':Interval) as $expr5, _row_id] } - │ └─StreamProject { exprs: [Field(person, 0:Int32) as $expr2, Field(person, 1:Int32) as $expr3, Field(person, 6:Int32) as $expr4, _row_id] } - │ └─StreamFilter { predicate: (event_type = 0:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, _row_id] } - │ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamAppendOnlyDedup { dedup_cols: [$expr8, $expr9, $expr10] } - └─StreamExchange { dist: HashShard($expr8, $expr9, $expr10) } - └─StreamProject { exprs: [$expr8, $expr9, ($expr9 + '00:00:10':Interval) as $expr10] } - └─StreamProject { exprs: [$expr7, $expr8, TumbleStart($expr7, '00:00:10':Interval) as $expr9, _row_id] } - └─StreamProject { exprs: [Field(auction, 5:Int32) as $expr7, Field(auction, 7:Int32) as $expr8, _row_id] } - └─StreamFilter { predicate: (event_type = 1:Int32) } - └─StreamShare { id: 5 } - └─StreamProject { exprs: [event_type, person, auction, _row_id] } - └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr2, $expr3, $expr5, $expr6) } + └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all } + ├─StreamExchange { dist: HashShard($expr2, $expr5, $expr6) } + │ └─StreamAppendOnlyDedup { dedup_cols: [$expr2, $expr3, $expr5, $expr6] } + │ └─StreamExchange { dist: HashShard($expr2, $expr3, $expr5, $expr6) } + │ └─StreamProject { exprs: [$expr2, $expr3, $expr5, ($expr5 + '00:00:10':Interval) as $expr6] } + │ └─StreamProject { exprs: [$expr2, $expr3, $expr4, TumbleStart($expr4, '00:00:10':Interval) as $expr5, _row_id] } + │ └─StreamProject { exprs: [Field(person, 0:Int32) as $expr2, Field(person, 1:Int32) as $expr3, Field(person, 6:Int32) as $expr4, _row_id] } + │ └─StreamFilter { predicate: (event_type = 0:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, _row_id] } + │ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamAppendOnlyDedup { dedup_cols: [$expr8, $expr9, $expr10] } + └─StreamExchange { dist: HashShard($expr8, $expr9, $expr10) } + └─StreamProject { exprs: [$expr8, $expr9, ($expr9 + '00:00:10':Interval) as $expr10] } + └─StreamProject { exprs: [$expr7, $expr8, TumbleStart($expr7, '00:00:10':Interval) as $expr9, _row_id] } + └─StreamProject { exprs: [Field(auction, 5:Int32) as $expr7, Field(auction, 7:Int32) as $expr8, _row_id] } + └─StreamFilter { predicate: (event_type = 1:Int32) } + └─StreamShare { id: 5 } + └─StreamProject { exprs: [event_type, person, auction, _row_id] } + └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [id, name, starttime, $expr6(hidden), $expr8(hidden), $expr9(hidden), $expr10(hidden)], stream_key: [id, name, starttime, $expr6], pk_columns: [id, name, starttime, $expr6], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0, 2, 3]) from 1 - └── StreamAppendOnlyDedup { dedup_cols: [$expr8, $expr9, $expr10] } { state table: 6 } - └── StreamExchange Hash([0, 1, 2]) from 4 + └── StreamExchange Hash([0, 1, 2, 3]) from 1 Fragment 1 - StreamAppendOnlyDedup { dedup_cols: [$expr2, $expr3, $expr5, $expr6] } { state table: 4 } - └── StreamExchange Hash([0, 1, 2, 3]) from 2 + StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0, 2, 3]) from 2 + └── StreamAppendOnlyDedup { dedup_cols: [$expr8, $expr9, $expr10] } { state table: 6 } + └── StreamExchange Hash([0, 1, 2]) from 5 Fragment 2 + StreamAppendOnlyDedup { dedup_cols: [$expr2, $expr3, $expr5, $expr6] } { state table: 4 } + └── StreamExchange Hash([0, 1, 2, 3]) from 3 + + Fragment 3 StreamProject { exprs: [$expr2, $expr3, $expr5, ($expr5 + '00:00:10':Interval) as $expr6] } └── StreamProject { exprs: [$expr2, $expr3, $expr4, TumbleStart($expr4, '00:00:10':Interval) as $expr5, _row_id] } └── StreamProject { exprs: [Field(person, 0:Int32) as $expr2, Field(person, 1:Int32) as $expr3, Field(person, 6:Int32) as $expr4, _row_id] } └── StreamFilter { predicate: (event_type = 0:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 - Fragment 3 + Fragment 4 StreamProject { exprs: [event_type, person, auction, _row_id] } └── StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } └── StreamRowIdGen { row_id_index: 5 } └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 4 + Fragment 5 StreamProject { exprs: [$expr8, $expr9, ($expr9 + '00:00:10':Interval) as $expr10] } └── StreamProject { exprs: [$expr7, $expr8, TumbleStart($expr7, '00:00:10':Interval) as $expr9, _row_id] } └── StreamProject { exprs: [Field(auction, 5:Int32) as $expr7, Field(auction, 7:Int32) as $expr8, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 Table 0 { columns: [ $expr2, $expr3, $expr5, $expr6 ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } @@ -795,7 +799,7 @@ ├── columns: [ id, name, starttime, $expr6, $expr8, $expr9, $expr10 ] ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] - ├── distribution key: [ 0, 2, 3 ] + ├── distribution key: [ 0, 1, 2, 3 ] └── read pk prefix len hint: 4 - id: nexmark_q9 @@ -1180,59 +1184,63 @@ WHERE A.category = 10; 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 } - └─StreamHashJoin { type: Inner, predicate: $expr3 = $expr9, output: [$expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard($expr3) } - │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr3, Field(bid, 1:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, Field(bid, 3:Int32) as $expr6, Field(bid, 4:Int32) as $expr7, Field(bid, 5:Int32) as $expr8, _row_id] } - │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } - │ ├─StreamFilter { predicate: (event_type = 2:Int32) } - │ │ └─StreamShare { id: 5 } - │ │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ │ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: HashShard($expr9) } - └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr9, Field(auction, 1:Int32) as $expr10, Field(auction, 2:Int32) as $expr11, Field(auction, 3:Int32) as $expr12, Field(auction, 4:Int32) as $expr13, Field(auction, 5:Int32) as $expr14, Field(auction, 6:Int32) as $expr15, Field(auction, 7:Int32) as $expr16, Field(auction, 8:Int32) as $expr17, _row_id] } - └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - └─StreamShare { id: 5 } - └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr3, _row_id, _row_id) } + └─StreamHashJoin { type: Inner, predicate: $expr3 = $expr9, output: [$expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard($expr3) } + │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr3, Field(bid, 1:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, Field(bid, 3:Int32) as $expr6, Field(bid, 4:Int32) as $expr7, Field(bid, 5:Int32) as $expr8, _row_id] } + │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } + │ ├─StreamFilter { predicate: (event_type = 2:Int32) } + │ │ └─StreamShare { id: 5 } + │ │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ │ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } + │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard($expr9) } + └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr9, Field(auction, 1:Int32) as $expr10, Field(auction, 2:Int32) as $expr11, Field(auction, 3:Int32) as $expr12, Field(auction, 4:Int32) as $expr13, Field(auction, 5:Int32) as $expr14, Field(auction, 6:Int32) as $expr15, Field(auction, 7:Int32) as $expr16, Field(auction, 8:Int32) as $expr17, _row_id] } + └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } + └─StreamShare { id: 5 } + └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 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 } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: $expr3 = $expr9, output: [$expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0, 14, 15]) from 1 Fragment 1 + StreamHashJoin { type: Inner, predicate: $expr3 = $expr9, output: [$expr3, $expr4, $expr5, $expr6, $expr7, $expr8, $expr10, $expr11, $expr12, $expr13, $expr14, $expr15, $expr16, $expr17, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 5 + + Fragment 2 StreamProject { exprs: [Field(bid, 0:Int32) as $expr3, Field(bid, 1:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, Field(bid, 3:Int32) as $expr6, Field(bid, 4:Int32) as $expr7, Field(bid, 5:Int32) as $expr8, _row_id] } └── StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } { left table: 4, right table: 5 } ├── StreamFilter { predicate: (event_type = 2:Int32) } - │ └── StreamExchange NoShuffle from 2 - └── StreamExchange Broadcast from 3 + │ └── StreamExchange NoShuffle from 3 + └── StreamExchange Broadcast from 4 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } └── StreamRowIdGen { row_id_index: 5 } └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 6 } - Fragment 3 + Fragment 4 StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └── StreamNow { output: [now] } { state table: 7 } - Fragment 4 + Fragment 5 StreamProject { exprs: [Field(auction, 0:Int32) as $expr9, Field(auction, 1:Int32) as $expr10, Field(auction, 2:Int32) as $expr11, Field(auction, 3:Int32) as $expr12, Field(auction, 4:Int32) as $expr13, Field(auction, 5:Int32) as $expr14, Field(auction, 6:Int32) as $expr15, Field(auction, 7:Int32) as $expr16, Field(auction, 8:Int32) as $expr17, _row_id] } └── StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr3, $expr4, $expr5, $expr6, $expr7, $expr8, _row_id ], primary key: [ $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1250,7 +1258,12 @@ Table 7 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ], primary key: [ $14 ASC, $15 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ], distribution key: [ 0 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ] + ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] + ├── distribution key: [ 0, 14, 15 ] + └── read pk prefix len hint: 3 - id: nexmark_q21 before: @@ -1375,61 +1388,65 @@ ) b ON a.id = b.auction; stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamProject { exprs: [$expr5, max($expr6)] } - └─StreamHashAgg { group_key: [$expr5], aggs: [max($expr6), count] } - └─StreamExchange { dist: HashShard($expr5) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, Field(bid, 2:Int32) as $expr6, _row_id] } - └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } - ├─StreamFilter { predicate: (event_type = 2:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } - └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard($expr2, _row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamProject { exprs: [$expr5, max($expr6)] } + └─StreamHashAgg { group_key: [$expr5], aggs: [max($expr6), count] } + └─StreamExchange { dist: HashShard($expr5) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, Field(bid, 2:Int32) as $expr6, _row_id] } + └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } + ├─StreamFilter { predicate: (event_type = 2:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } + └─StreamNow { output: [now] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [$expr5, max($expr6)] } - └── StreamHashAgg { group_key: [$expr5], aggs: [max($expr6), count] } { intermediate state table: 6, state tables: [ 5 ], distinct tables: [] } - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 3]) from 1 Fragment 1 + StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [$expr5, max($expr6)] } + └── StreamHashAgg { group_key: [$expr5], aggs: [max($expr6), count] } { intermediate state table: 6, state tables: [ 5 ], distinct tables: [] } + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 4 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, Field(bid, 2:Int32) as $expr6, _row_id] } └── StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } { left table: 7, right table: 8 } ├── StreamFilter { predicate: (event_type = 2:Int32) } - │ └── StreamExchange NoShuffle from 2 - └── StreamExchange Broadcast from 4 + │ └── StreamExchange NoShuffle from 3 + └── StreamExchange Broadcast from 5 - Fragment 4 + Fragment 5 StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } └── StreamNow { output: [now] } { state table: 9 } @@ -1457,7 +1474,7 @@ ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, $expr5 ] ├── primary key: [ $3 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 3 ] └── read pk prefix len hint: 2 - id: nexmark_q102 @@ -1642,65 +1659,69 @@ ); stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr5, output: all } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamProject { exprs: [$expr5] } - └─StreamFilter { predicate: (count >= 20:Int32) } - └─StreamHashAgg { group_key: [$expr5], aggs: [count] } - └─StreamExchange { dist: HashShard($expr5) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } - └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } - ├─StreamFilter { predicate: (event_type = 2:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } - └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard($expr2, _row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr5, output: all } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamProject { exprs: [$expr5] } + └─StreamFilter { predicate: (count >= 20:Int32) } + └─StreamHashAgg { group_key: [$expr5], aggs: [count] } + └─StreamExchange { dist: HashShard($expr5) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } + └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } + ├─StreamFilter { predicate: (event_type = 2:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } + └─StreamNow { output: [now] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr5, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [$expr5] } - └── StreamFilter { predicate: (count >= 20:Int32) } - └── StreamHashAgg { group_key: [$expr5], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 2]) from 1 Fragment 1 + StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr5, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [$expr5] } + └── StreamFilter { predicate: (count >= 20:Int32) } + └── StreamHashAgg { group_key: [$expr5], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 4 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } └── StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } ├── left table: 6 ├── right table: 7 ├── StreamFilter { predicate: (event_type = 2:Int32) } - │ └── StreamExchange NoShuffle from 2 - └── StreamExchange Broadcast from 4 + │ └── StreamExchange NoShuffle from 3 + └── StreamExchange Broadcast from 5 - Fragment 4 + Fragment 5 StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } └── StreamNow { output: [now] } { state table: 8 } @@ -1731,7 +1752,7 @@ ├── columns: [ auction_id, auction_item_name, _row_id ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 2 ] └── read pk prefix len hint: 2 - id: nexmark_q104 @@ -1752,65 +1773,69 @@ ); stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr5, output: all } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamProject { exprs: [$expr5] } - └─StreamFilter { predicate: (count < 20:Int32) } - └─StreamHashAgg { group_key: [$expr5], aggs: [count] } - └─StreamExchange { dist: HashShard($expr5) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } - └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } - ├─StreamFilter { predicate: (event_type = 2:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } - └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard($expr2, _row_id) } + └─StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr5, output: all } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamProject { exprs: [$expr5] } + └─StreamFilter { predicate: (count < 20:Int32) } + └─StreamHashAgg { group_key: [$expr5], aggs: [count] } + └─StreamExchange { dist: HashShard($expr5) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } + └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } + ├─StreamFilter { predicate: (event_type = 2:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } + └─StreamNow { output: [now] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr5, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [$expr5] } - └── StreamFilter { predicate: (count < 20:Int32) } - └── StreamHashAgg { group_key: [$expr5], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 2]) from 1 Fragment 1 + StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr5, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [$expr5] } + └── StreamFilter { predicate: (count < 20:Int32) } + └── StreamHashAgg { group_key: [$expr5], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 4 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } └── StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [$expr1], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } ├── left table: 6 ├── right table: 7 ├── StreamFilter { predicate: (event_type = 2:Int32) } - │ └── StreamExchange NoShuffle from 2 - └── StreamExchange Broadcast from 4 + │ └── StreamExchange NoShuffle from 3 + └── StreamExchange Broadcast from 5 - Fragment 4 + Fragment 5 StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [$expr4] } └── StreamNow { output: [now] } { state table: 8 } @@ -1841,7 +1866,7 @@ ├── columns: [ auction_id, auction_item_name, _row_id ] ├── primary key: [ $2 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 2 ] └── read pk prefix len hint: 2 - id: nexmark_q105 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 3554e31d281ec..39adc39a16653 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml @@ -131,45 +131,49 @@ └─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 } - └─StreamHashJoin [append_only] { type: Inner, predicate: $expr3 = $expr4, output: [$expr5, $expr6, $expr7, $expr2, _row_id, $expr3, _row_id] } - ├─StreamExchange { dist: HashShard($expr3) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 7:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, person, auction, _row_id] } - │ └─StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((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))) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { 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] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: HashShard($expr4) } - └─StreamProject { exprs: [Field(person, 0:Int32) as $expr4, Field(person, 1:Int32) as $expr5, Field(person, 4:Int32) as $expr6, Field(person, 5:Int32) as $expr7, _row_id] } - └─StreamFilter { 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) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, person, auction, _row_id] } - └─StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((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))) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { 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] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard(_row_id, $expr3, _row_id) } + └─StreamHashJoin [append_only] { type: Inner, predicate: $expr3 = $expr4, output: [$expr5, $expr6, $expr7, $expr2, _row_id, $expr3, _row_id] } + ├─StreamExchange { dist: HashShard($expr3) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 7:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, person, auction, _row_id] } + │ └─StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((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))) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { 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] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr4) } + └─StreamProject { exprs: [Field(person, 0:Int32) as $expr4, Field(person, 1:Int32) as $expr5, Field(person, 4:Int32) as $expr6, Field(person, 5:Int32) as $expr7, _row_id] } + └─StreamFilter { 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) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, person, auction, _row_id] } + └─StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((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))) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { 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] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 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 } ├── materialized table: 4294967294 - └── StreamHashJoin [append_only] { type: Inner, predicate: $expr3 = $expr4, output: [$expr5, $expr6, $expr7, $expr2, _row_id, $expr3, _row_id] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([1]) from 1 - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([4, 5, 6]) from 1 Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: $expr3 = $expr4, output: [$expr5, $expr6, $expr7, $expr2, _row_id, $expr3, _row_id] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([1]) from 2 + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 7:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, person, auction, _row_id] } └── StreamFilter { predicate: (((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32)) OR ((((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))) } └── StreamRowIdGen { row_id_index: 5 } @@ -177,10 +181,10 @@ └── StreamProject { 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] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(person, 0:Int32) as $expr4, Field(person, 1:Int32) as $expr5, Field(person, 4:Int32) as $expr6, Field(person, 5:Int32) as $expr7, _row_id] } └── StreamFilter { 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) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } @@ -194,7 +198,7 @@ Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ name, city, state, id, _row_id, $expr3, _row_id#1 ], primary key: [ $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 5 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ name, city, state, id, _row_id, $expr3, _row_id#1 ], primary key: [ $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 4, 5, 6 ], read pk prefix len hint: 3 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -696,43 +700,48 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time, $expr5(hidden)] } - └─StreamHashJoin [interval] { type: Inner, predicate: $expr4 = max($expr4) AND ($expr1 >= $expr6) AND ($expr1 <= $expr5), conditions_to_clean_left_state_table: ($expr1 >= $expr6), conditions_to_clean_right_state_table: ($expr1 <= $expr5), output_watermarks: [$expr1, $expr5], output: [$expr2, $expr4, $expr3, $expr1, _row_id, $expr5] } - ├─StreamExchange { dist: HashShard($expr4) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: (event_type = 2:Int32) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { 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] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: HashShard(max($expr4)) } - └─StreamProject { exprs: [$expr5, max($expr4), ($expr5 - '00:00:10':Interval) as $expr6], output_watermarks: [$expr5, $expr6] } - └─StreamHashAgg [append_only] { group_key: [$expr5], aggs: [max($expr4), count], output_watermarks: [$expr5] } - └─StreamExchange { dist: HashShard($expr5) } - └─StreamProject { exprs: [(TumbleStart($expr1, '00:00:10':Interval) + '00:00:10':Interval) as $expr5, $expr4, _row_id], output_watermarks: [$expr5] } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamFilter { predicate: (event_type = 2:Int32) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { 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] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr4, _row_id, $expr5) } + └─StreamHashJoin [interval] { type: Inner, predicate: $expr4 = max($expr4) AND ($expr1 >= $expr6) AND ($expr1 <= $expr5), conditions_to_clean_left_state_table: ($expr1 >= $expr6), conditions_to_clean_right_state_table: ($expr1 <= $expr5), output_watermarks: [$expr1, $expr5], output: [$expr2, $expr4, $expr3, $expr1, _row_id, $expr5] } + ├─StreamExchange { dist: HashShard($expr4) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: (event_type = 2:Int32) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { 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] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard(max($expr4)) } + └─StreamProject { exprs: [$expr5, max($expr4), ($expr5 - '00:00:10':Interval) as $expr6], output_watermarks: [$expr5, $expr6] } + └─StreamHashAgg [append_only] { group_key: [$expr5], aggs: [max($expr4), count], output_watermarks: [$expr5] } + └─StreamExchange { dist: HashShard($expr5) } + └─StreamProject { exprs: [(TumbleStart($expr1, '00:00:10':Interval) + '00:00:10':Interval) as $expr5, $expr4, _row_id], output_watermarks: [$expr5] } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamFilter { predicate: (event_type = 2:Int32) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { 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] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time, $expr5(hidden)] } { materialized table: 4294967294 } - └── StreamHashJoin [interval] { type: Inner, predicate: $expr4 = max($expr4) AND ($expr1 >= $expr6) AND ($expr1 <= $expr5), conditions_to_clean_left_state_table: ($expr1 >= $expr6), conditions_to_clean_right_state_table: ($expr1 <= $expr5), output_watermarks: [$expr1, $expr5], output: [$expr2, $expr4, $expr3, $expr1, _row_id, $expr5] } - ├── left table: 0 - ├── right table: 2 - ├── left degree table: 1 - ├── right degree table: 3 - ├── StreamExchange Hash([2]) from 1 - └── StreamExchange Hash([1]) from 3 + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time, $expr5(hidden)] } + ├── materialized table: 4294967294 + └── StreamExchange Hash([1, 4, 5]) from 1 Fragment 1 - StreamNoOp - └── StreamExchange NoShuffle from 2 + StreamHashJoin [interval] { type: Inner, predicate: $expr4 = max($expr4) AND ($expr1 >= $expr6) AND ($expr1 <= $expr5), conditions_to_clean_left_state_table: ($expr1 >= $expr6), conditions_to_clean_right_state_table: ($expr1 <= $expr5), output_watermarks: [$expr1, $expr5], output: [$expr2, $expr4, $expr3, $expr1, _row_id, $expr5] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 + ├── StreamExchange Hash([2]) from 2 + └── StreamExchange Hash([1]) from 4 Fragment 2 + StreamNoOp + └── StreamExchange NoShuffle from 3 + + Fragment 3 StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: (event_type = 2:Int32) } └── StreamRowIdGen { row_id_index: 5 } @@ -740,14 +749,14 @@ └── StreamProject { 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] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [$expr5, max($expr4), ($expr5 - '00:00:10':Interval) as $expr6], output_watermarks: [$expr5, $expr6] } └── StreamHashAgg [append_only] { group_key: [$expr5], aggs: [max($expr4), count], output_watermarks: [$expr5] } { intermediate state table: 6, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0]) from 5 - Fragment 4 + Fragment 5 StreamProject { exprs: [(TumbleStart($expr1, '00:00:10':Interval) + '00:00:10':Interval) as $expr5, $expr4, _row_id], output_watermarks: [$expr5] } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr2, $expr3, $expr4, $expr1, _row_id ], primary key: [ $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } @@ -763,7 +772,7 @@ Table 6 { columns: [ $expr5, max($expr4), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, price, bidder, date_time, _row_id, $expr5 ], primary key: [ $4 ASC, $5 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ auction, price, bidder, date_time, _row_id, $expr5 ], primary key: [ $4 ASC, $5 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 1, 4, 5 ], read pk prefix len hint: 3 } eowc_stream_plan: |- StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, price], pk_columns: [_row_id, $expr5, price], pk_conflict: NoCheck, watermark_columns: [date_time] } @@ -845,52 +854,56 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime, $expr5(hidden), $expr6(hidden), $expr8(hidden)] } - └─StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all } - ├─StreamExchange { dist: HashShard($expr3, $expr2, $expr5) } - │ └─StreamAppendOnlyDedup { dedup_cols: [$expr3, $expr4, $expr2, $expr5] } - │ └─StreamExchange { dist: HashShard($expr3, $expr4, $expr2, $expr5) } - │ └─StreamProject { exprs: [Field(person, 0:Int32) as $expr3, Field(person, 1:Int32) as $expr4, $expr2, ($expr2 + '00:00:10':Interval) as $expr5], output_watermarks: [$expr2, $expr5] } - │ └─StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr2, _row_id], output_watermarks: [$expr1, $expr2] } - │ └─StreamFilter { predicate: (event_type = 0:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { 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] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamAppendOnlyDedup { dedup_cols: [$expr7, $expr6, $expr8] } - └─StreamExchange { dist: HashShard($expr7, $expr6, $expr8) } - └─StreamProject { exprs: [Field(auction, 7:Int32) as $expr7, $expr6, ($expr6 + '00:00:10':Interval) as $expr8], output_watermarks: [$expr6, $expr8] } - └─StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr6, _row_id], output_watermarks: [$expr1, $expr6] } - └─StreamFilter { predicate: (event_type = 1:Int32) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { 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] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr3, $expr4, $expr2, $expr5) } + └─StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all } + ├─StreamExchange { dist: HashShard($expr3, $expr2, $expr5) } + │ └─StreamAppendOnlyDedup { dedup_cols: [$expr3, $expr4, $expr2, $expr5] } + │ └─StreamExchange { dist: HashShard($expr3, $expr4, $expr2, $expr5) } + │ └─StreamProject { exprs: [Field(person, 0:Int32) as $expr3, Field(person, 1:Int32) as $expr4, $expr2, ($expr2 + '00:00:10':Interval) as $expr5], output_watermarks: [$expr2, $expr5] } + │ └─StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr2, _row_id], output_watermarks: [$expr1, $expr2] } + │ └─StreamFilter { predicate: (event_type = 0:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { 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] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamAppendOnlyDedup { dedup_cols: [$expr7, $expr6, $expr8] } + └─StreamExchange { dist: HashShard($expr7, $expr6, $expr8) } + └─StreamProject { exprs: [Field(auction, 7:Int32) as $expr7, $expr6, ($expr6 + '00:00:10':Interval) as $expr8], output_watermarks: [$expr6, $expr8] } + └─StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr6, _row_id], output_watermarks: [$expr1, $expr6] } + └─StreamFilter { predicate: (event_type = 1:Int32) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { 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] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime, $expr5(hidden), $expr6(hidden), $expr8(hidden)] } ├── materialized table: 4294967294 - └── StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0, 2, 3]) from 1 - └── StreamAppendOnlyDedup { dedup_cols: [$expr7, $expr6, $expr8] } { state table: 7 } - └── StreamExchange Hash([0, 1, 2]) from 4 + └── StreamExchange Hash([0, 1, 2, 3]) from 1 Fragment 1 - StreamAppendOnlyDedup { dedup_cols: [$expr3, $expr4, $expr2, $expr5] } { state table: 4 } - └── StreamExchange Hash([0, 1, 2, 3]) from 2 + StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0, 2, 3]) from 2 + └── StreamAppendOnlyDedup { dedup_cols: [$expr7, $expr6, $expr8] } { state table: 7 } + └── StreamExchange Hash([0, 1, 2]) from 5 Fragment 2 + StreamAppendOnlyDedup { dedup_cols: [$expr3, $expr4, $expr2, $expr5] } { state table: 4 } + └── StreamExchange Hash([0, 1, 2, 3]) from 3 + + Fragment 3 StreamProject { exprs: [Field(person, 0:Int32) as $expr3, Field(person, 1:Int32) as $expr4, $expr2, ($expr2 + '00:00:10':Interval) as $expr5], output_watermarks: [$expr2, $expr5] } └── StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr2, _row_id], output_watermarks: [$expr1, $expr2] } └── StreamFilter { predicate: (event_type = 0:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 - Fragment 3 + Fragment 4 StreamProject { exprs: [event_type, person, auction, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: ((event_type = 0:Int32) OR (event_type = 1:Int32)) } └── StreamRowIdGen { row_id_index: 5 } @@ -898,11 +911,11 @@ └── StreamProject { 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] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 6 } - Fragment 4 + Fragment 5 StreamProject { exprs: [Field(auction, 7:Int32) as $expr7, $expr6, ($expr6 + '00:00:10':Interval) as $expr8], output_watermarks: [$expr6, $expr8] } └── StreamProject { exprs: [event_type, person, auction, $expr1, TumbleStart($expr1, '00:00:10':Interval) as $expr6, _row_id], output_watermarks: [$expr1, $expr6] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 Table 0 { columns: [ $expr3, $expr4, $expr2, $expr5 ], primary key: [ $2 ASC, $3 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } @@ -920,7 +933,7 @@ Table 7 { columns: [ $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4294967294 { columns: [ id, name, starttime, $expr5, $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 4 } + Table 4294967294 { columns: [ id, name, starttime, $expr5, $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } eowc_stream_plan: |- StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime] } @@ -1715,41 +1728,45 @@ └─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 } - └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr7, output: [$expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { 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, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: (event_type = 2:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - │ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { 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] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: HashShard($expr7) } - └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr7, Field(auction, 1:Int32) as $expr8, Field(auction, 2:Int32) as $expr9, Field(auction, 3:Int32) as $expr10, Field(auction, 4:Int32) as $expr11, $expr1, Field(auction, 6:Int32) as $expr12, Field(auction, 7:Int32) as $expr13, Field(auction, 8:Int32) as $expr14, _row_id], output_watermarks: [$expr1] } - └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { 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] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr2, _row_id, _row_id) } + └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr7, output: [$expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { 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, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: (event_type = 2:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + │ └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { 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] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr7) } + └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr7, Field(auction, 1:Int32) as $expr8, Field(auction, 2:Int32) as $expr9, Field(auction, 3:Int32) as $expr10, Field(auction, 4:Int32) as $expr11, $expr1, Field(auction, 6:Int32) as $expr12, Field(auction, 7:Int32) as $expr13, Field(auction, 8:Int32) as $expr14, _row_id], output_watermarks: [$expr1] } + └─StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { 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] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 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 } ├── materialized table: 4294967294 - └── StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr7, output: [$expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 14, 15]) from 1 Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr7, output: [$expr2, $expr3, $expr4, $expr5, $expr6, $expr1, $expr8, $expr9, $expr10, $expr11, $expr1, $expr12, $expr13, $expr14, _row_id, _row_id] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { 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, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: (event_type = 2:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: ((event_type = 2:Int32) OR ((Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32))) } └── StreamRowIdGen { row_id_index: 5 } @@ -1757,10 +1774,10 @@ └── StreamProject { 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] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(auction, 0:Int32) as $expr7, Field(auction, 1:Int32) as $expr8, Field(auction, 2:Int32) as $expr9, Field(auction, 3:Int32) as $expr10, Field(auction, 4:Int32) as $expr11, $expr1, Field(auction, 6:Int32) as $expr12, Field(auction, 7:Int32) as $expr13, Field(auction, 8:Int32) as $expr14, _row_id], output_watermarks: [$expr1] } └── StreamFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr2, $expr3, $expr4, $expr5, $expr6, $expr1, _row_id ], primary key: [ $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1774,7 +1791,12 @@ Table 5 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ], primary key: [ $14 ASC, $15 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ], distribution key: [ 0 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1 ] + ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] + ├── distribution key: [ 0, 14, 15 ] + └── read pk prefix len hint: 3 eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -1909,45 +1931,49 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr4(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { 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] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamProject { exprs: [$expr4, max($expr5)] } - └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [max($expr5), count] } - └─StreamExchange { dist: HashShard($expr4) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, _row_id] } - └─StreamFilter { predicate: (event_type = 2:Int32) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { 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] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr2, _row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { 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] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamProject { exprs: [$expr4, max($expr5)] } + └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [max($expr5), count] } + └─StreamExchange { dist: HashShard($expr4) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, _row_id] } + └─StreamFilter { predicate: (event_type = 2:Int32) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { 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] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr4(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [$expr4, max($expr5)] } - └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [max($expr5), count] } { intermediate state table: 6, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 3]) from 1 Fragment 1 + StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [$expr4, max($expr5)] } + └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [max($expr5), count] } { intermediate state table: 6, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, _row_id] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } @@ -1955,10 +1981,10 @@ └── StreamProject { 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] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, Field(bid, 2:Int32) as $expr5, _row_id] } └── StreamFilter { predicate: (event_type = 2:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1978,7 +2004,7 @@ ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, $expr4 ] ├── primary key: [ $3 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] - ├── distribution key: [ 0 ] + ├── distribution key: [ 0, 3 ] └── read pk prefix len hint: 2 eowc_stream_error: |- @@ -2184,47 +2210,51 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr4, output: all } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { 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] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamProject { exprs: [$expr4] } - └─StreamFilter { predicate: (count >= 20:Int32) } - └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } - └─StreamExchange { dist: HashShard($expr4) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } - └─StreamFilter { predicate: (event_type = 2:Int32) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { 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] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr2, _row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr4, output: all } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { 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] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamProject { exprs: [$expr4] } + └─StreamFilter { predicate: (count >= 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } + └─StreamExchange { dist: HashShard($expr4) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } + └─StreamFilter { predicate: (event_type = 2:Int32) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { 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] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [$expr4] } - └── StreamFilter { predicate: (count >= 20:Int32) } - └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 2]) from 1 Fragment 1 + StreamHashJoin { type: LeftSemi, predicate: $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [$expr4] } + └── StreamFilter { predicate: (count >= 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, _row_id] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } @@ -2232,10 +2262,10 @@ └── StreamProject { 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] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } └── StreamFilter { predicate: (event_type = 2:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2251,7 +2281,7 @@ Table 6 { columns: [ $expr4, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 2 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. @@ -2290,47 +2320,51 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr4, output: all } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - │ └─StreamProject { 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] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamProject { exprs: [$expr4] } - └─StreamFilter { predicate: (count < 20:Int32) } - └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } - └─StreamExchange { dist: HashShard($expr4) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } - └─StreamFilter { predicate: (event_type = 2:Int32) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } - └─StreamProject { 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] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr2, _row_id) } + └─StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr4, output: all } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + │ └─StreamProject { 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] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamProject { exprs: [$expr4] } + └─StreamFilter { predicate: (count < 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } + └─StreamExchange { dist: HashShard($expr4) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } + └─StreamFilter { predicate: (event_type = 2:Int32) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } + └─StreamProject { 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] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [$expr4] } - └── StreamFilter { predicate: (count < 20:Int32) } - └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } - └── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0, 2]) from 1 Fragment 1 + StreamHashJoin { type: LeftAnti, predicate: $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [$expr4] } + └── StreamFilter { predicate: (count < 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [$expr4], aggs: [count] } { intermediate state table: 6, state tables: [], distinct tables: [] } + └── StreamExchange Hash([0]) from 4 + + Fragment 2 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [event_type, auction, bid, _row_id] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } @@ -2338,10 +2372,10 @@ └── StreamProject { 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] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { source state table: 5 } - Fragment 3 + Fragment 4 StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } └── StreamFilter { predicate: (event_type = 2:Int32) } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ $expr2, $expr3, _row_id ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2357,7 +2391,7 @@ Table 6 { columns: [ $expr4, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ auction_id, auction_item_name, _row_id ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 2 } eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. diff --git a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml index 5cc81578f829c..733a19f4ba05c 100644 --- a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml +++ b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml @@ -185,15 +185,16 @@ └─BatchScan { table: t, columns: [t.x, t.y, t.w], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, y, sum, max, min, t._row_id(hidden), t.y(hidden)], stream_key: [t._row_id, y], pk_columns: [t._row_id, y], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.y = t.y, output: [t.x, t.y, sum(t.x), max(t.x), min(t.w), t._row_id, t.y] } - ├─StreamExchange { dist: HashShard(t.y) } - │ └─StreamShare { id: 1 } - │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.w, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamProject { exprs: [t.y, sum(t.x), max(t.x), min(t.w)] } - └─StreamHashAgg { group_key: [t.y], aggs: [sum(t.x), max(t.x), min(t.w), count] } - └─StreamExchange { dist: HashShard(t.y) } - └─StreamShare { id: 1 } - └─StreamTableScan { table: t, columns: [t.x, t.y, t.w, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.y, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.y = t.y, output: [t.x, t.y, sum(t.x), max(t.x), min(t.w), t._row_id, t.y] } + ├─StreamExchange { dist: HashShard(t.y) } + │ └─StreamShare { id: 1 } + │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.w, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProject { exprs: [t.y, sum(t.x), max(t.x), min(t.w)] } + └─StreamHashAgg { group_key: [t.y], aggs: [sum(t.x), max(t.x), min(t.w), count] } + └─StreamExchange { dist: HashShard(t.y) } + └─StreamShare { id: 1 } + └─StreamTableScan { table: t, columns: [t.x, t.y, t.w, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - id: aggregate with over clause, rows frame definition with implicit current row, without ORDER BY sql: | create table t(x int, y int); @@ -913,12 +914,13 @@ └─BatchScan { table: t, columns: [t.x, t.y, t.z], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [t1x, t2x, t1z, t2y, t2z, t._row_id(hidden)], stream_key: [t1x, t._row_id], pk_columns: [t1x, t._row_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.x = t.x, output: [t.x, t.x, t.z, t.y, t.z, t._row_id] } - ├─StreamGroupTopN { order: [t.y ASC], limit: 1, offset: 0, group_key: [t.x] } - │ └─StreamExchange { dist: HashShard(t.x) } - │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(t.x) } - └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.x, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.x = t.x, output: [t.x, t.x, t.z, t.y, t.z, t._row_id] } + ├─StreamGroupTopN { order: [t.y ASC], limit: 1, offset: 0, group_key: [t.x] } + │ └─StreamExchange { dist: HashShard(t.x) } + │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.x) } + └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - id: split calls with different ORDER BY or PARTITION BY sql: | create table t(x int, y int, z int); diff --git a/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml b/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml index 55131ed1614cd..65469e7754e6b 100644 --- a/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml @@ -21,15 +21,16 @@ Tone.id = Ttwo.id; stream_plan: |- StreamMaterialize { columns: [max_v1, max_v2, t1.id(hidden), t2.id(hidden)], stream_key: [t1.id], pk_columns: [t1.id], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.id = t2.id, output: [max(t1.v1), max(t2.v2), t1.id, t2.id] } - ├─StreamProject { exprs: [t1.id, max(t1.v1)] } - │ └─StreamHashAgg { group_key: [t1.id], aggs: [max(t1.v1), count] } - │ └─StreamExchange { dist: HashShard(t1.id) } - │ └─StreamTableScan { table: t1, columns: [t1.id, t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamProject { exprs: [t2.id, max(t2.v2)] } - └─StreamHashAgg { group_key: [t2.id], aggs: [max(t2.v2), count] } - └─StreamExchange { dist: HashShard(t2.id) } - └─StreamTableScan { table: t2, columns: [t2.id, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.id) } + └─StreamHashJoin { type: Inner, predicate: t1.id = t2.id, output: [max(t1.v1), max(t2.v2), t1.id, t2.id] } + ├─StreamProject { exprs: [t1.id, max(t1.v1)] } + │ └─StreamHashAgg { group_key: [t1.id], aggs: [max(t1.v1), count] } + │ └─StreamExchange { dist: HashShard(t1.id) } + │ └─StreamTableScan { table: t1, columns: [t1.id, t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamProject { exprs: [t2.id, max(t2.v2)] } + └─StreamHashAgg { group_key: [t2.id], aggs: [max(t2.v2), count] } + └─StreamExchange { dist: HashShard(t2.id) } + └─StreamTableScan { table: t2, columns: [t2.id, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - sql: | create table t (id int, v int); SELECT Tone.max_v, Ttwo.min_v @@ -51,15 +52,16 @@ Tone.id = Ttwo.id; stream_plan: |- StreamMaterialize { columns: [max_v, min_v, t.id(hidden), t.id#1(hidden)], stream_key: [t.id], pk_columns: [t.id], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.id = t.id, output: [max(t.v), min(t.v), t.id, t.id] } - ├─StreamProject { exprs: [t.id, max(t.v)] } - │ └─StreamHashAgg { group_key: [t.id], aggs: [max(t.v), count] } - │ └─StreamExchange { dist: HashShard(t.id) } - │ └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamProject { exprs: [t.id, min(t.v)] } - └─StreamHashAgg { group_key: [t.id], aggs: [min(t.v), count] } - └─StreamExchange { dist: HashShard(t.id) } - └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.id) } + └─StreamHashJoin { type: Inner, predicate: t.id = t.id, output: [max(t.v), min(t.v), t.id, t.id] } + ├─StreamProject { exprs: [t.id, max(t.v)] } + │ └─StreamHashAgg { group_key: [t.id], aggs: [max(t.v), count] } + │ └─StreamExchange { dist: HashShard(t.id) } + │ └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProject { exprs: [t.id, min(t.v)] } + └─StreamHashAgg { group_key: [t.id], aggs: [min(t.v), count] } + └─StreamExchange { dist: HashShard(t.id) } + └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 varchar, v2 varchar, v3 varchar); select diff --git a/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml index ae37459ef7bed..91dff73df0e6a 100644 --- a/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/output/predicate_pushdown.yaml @@ -260,15 +260,16 @@ └─LogicalScan { table: t2, columns: [t2.v2], predicate: (t2.v2 > ('2021-04-01 00:00:00+00:00':Timestamptz + '01:00:00':Interval)) } stream_plan: |- StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1], pk_columns: [t1._row_id, t2._row_id, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id], cleaned_by_watermark: true } - │ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [AddWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: HashShard(t2.v2) } - └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id], cleaned_by_watermark: true } + │ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [AddWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(t2.v2) } + └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: now() in a complex cmp expr does not get pushed down sql: | create table t1(v1 timestamp with time zone); @@ -343,14 +344,15 @@ └─LogicalScan { table: t2, columns: [t2.v2], predicate: (t2.v2 > '2021-04-01 00:00:00+00:00':Timestamptz) } stream_plan: |- StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, v1], pk_columns: [t1._row_id, t2._row_id, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamDynamicFilter { predicate: (t1.v1 > now), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id], cleaned_by_watermark: true } - │ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: HashShard(t2.v2) } - └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamDynamicFilter { predicate: (t1.v1 > now), output_watermarks: [t1.v1], output: [t1.v1, t1._row_id], cleaned_by_watermark: true } + │ ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(t2.v2) } + └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: eq-predicate derived condition is banned for mismatching types sql: | create table t1(v1 int, v2 int); diff --git a/src/frontend/planner_test/tests/testdata/output/project_set.yaml b/src/frontend/planner_test/tests/testdata/output/project_set.yaml index 23db668a070df..676772d99d72e 100644 --- a/src/frontend/planner_test/tests/testdata/output/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/output/project_set.yaml @@ -155,17 +155,18 @@ └─BatchScan { table: t, columns: [t.x], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [unnest, t._row_id(hidden), projected_row_id(hidden), t._row_id#1(hidden), projected_row_id#1(hidden)], stream_key: [t._row_id, projected_row_id, t._row_id#1, projected_row_id#1, unnest], pk_columns: [t._row_id, projected_row_id, t._row_id#1, projected_row_id#1, unnest], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: Unnest($0) = Unnest($0), output: [Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id] } - ├─StreamExchange { dist: HashShard(Unnest($0)) } - │ └─StreamShare { id: 3 } - │ └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } - │ └─StreamProjectSet { select_list: [Unnest($0), $1] } - │ └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(Unnest($0)) } - └─StreamShare { id: 3 } - └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } - └─StreamProjectSet { select_list: [Unnest($0), $1] } - └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id) } + └─StreamHashJoin { type: Inner, predicate: Unnest($0) = Unnest($0), output: [Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id] } + ├─StreamExchange { dist: HashShard(Unnest($0)) } + │ └─StreamShare { id: 3 } + │ └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } + │ └─StreamProjectSet { select_list: [Unnest($0), $1] } + │ └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(Unnest($0)) } + └─StreamShare { id: 3 } + └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] } + └─StreamProjectSet { select_list: [Unnest($0), $1] } + └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: issue-10080 sql: | with cte as (SELECT 1 as v1, unnest(array[1,2,3,4,5]) AS v2) select v1 from cte; diff --git a/src/frontend/planner_test/tests/testdata/output/select_except.yaml b/src/frontend/planner_test/tests/testdata/output/select_except.yaml index 2193524b7076f..ffd6da30b90bc 100644 --- a/src/frontend/planner_test/tests/testdata/output/select_except.yaml +++ b/src/frontend/planner_test/tests/testdata/output/select_except.yaml @@ -34,11 +34,12 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v3, v2, t._row_id(hidden), t._row_id#1(hidden)], stream_key: [t._row_id, t._row_id#1, v1], pk_columns: [t._row_id, t._row_id#1, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v3, t.v2, t._row_id, t._row_id] } - ├─StreamExchange { dist: HashShard(t.v1) } - │ └─StreamTableScan { table: t, columns: [t.v1, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(t.v1) } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1, t._row_id, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v3, t.v2, t._row_id, t._row_id] } + ├─StreamExchange { dist: HashShard(t.v1) } + │ └─StreamTableScan { table: t, columns: [t.v1, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1) } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: qualified wildcard sql: | create table t (v1 int, v2 int, v3 int); @@ -52,11 +53,12 @@ └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, v3, t._row_id(hidden), t._row_id#1(hidden)], stream_key: [t._row_id, t._row_id#1, v1], pk_columns: [t._row_id, t._row_id#1, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v2, t.v3, t._row_id, t._row_id] } - ├─StreamExchange { dist: HashShard(t.v1) } - │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamExchange { dist: HashShard(t.v1) } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1, t._row_id, t._row_id) } + └─StreamHashJoin { type: Inner, predicate: t.v1 = t.v1, output: [t.v1, t.v2, t.v3, t._row_id, t._row_id] } + ├─StreamExchange { dist: HashShard(t.v1) } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1) } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: except with unknown column sql: | create table t (v1 int, v2 int, v3 int); diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index 2815b00784b1d..15404d6d863ab 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -343,24 +343,25 @@ └─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 } - └─StreamHashJoin { type: Inner, predicate: id = id, output: [id, id, date_time, date_time, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(id) } - │ └─StreamProject { exprs: [id, date_time, _row_id] } - │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [id, date_time, $expr1, _row_id], cleaned_by_watermark: true } - │ ├─StreamProject { exprs: [id, date_time, AtTimeZone(date_time, 'UTC':Varchar) as $expr1, _row_id] } - │ │ └─StreamFilter { predicate: (initial_bid = 1:Int32) } - │ │ └─StreamShare { id: 4 } - │ │ └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } - │ │ └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:00:01':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: HashShard(id) } - └─StreamFilter { predicate: (initial_bid = 2:Int32) } - └─StreamShare { id: 4 } - └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } - └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } - └─StreamRowIdGen { row_id_index: 10 } - └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + └─StreamExchange { dist: HashShard(id, _row_id, _row_id) } + └─StreamHashJoin { type: Inner, predicate: id = id, output: [id, id, date_time, date_time, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamProject { exprs: [id, date_time, _row_id] } + │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [id, date_time, $expr1, _row_id], cleaned_by_watermark: true } + │ ├─StreamProject { exprs: [id, date_time, AtTimeZone(date_time, 'UTC':Varchar) as $expr1, _row_id] } + │ │ └─StreamFilter { predicate: (initial_bid = 1:Int32) } + │ │ └─StreamShare { id: 4 } + │ │ └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } + │ │ └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:00:01':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(id) } + └─StreamFilter { predicate: (initial_bid = 2:Int32) } + └─StreamShare { id: 4 } + └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } + └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/shared_views.yaml b/src/frontend/planner_test/tests/testdata/output/shared_views.yaml index 775812f77b59c..3777705c97ced 100644 --- a/src/frontend/planner_test/tests/testdata/output/shared_views.yaml +++ b/src/frontend/planner_test/tests/testdata/output/shared_views.yaml @@ -23,19 +23,20 @@ └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } stream_plan: |- StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden)], stream_key: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, z], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, z], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: $expr1 = $expr2, output: [$expr1, $expr2, $expr3, t1._row_id, t1._row_id, t1._row_id, t1.x] } - ├─StreamExchange { dist: HashShard($expr1) } - │ └─StreamShare { id: 3 } - │ └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } - │ └─StreamFilter { predicate: (t1.y > 0:Int32) } - │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard($expr2) } - └─StreamProject { exprs: [(t1.x * $expr1) as $expr2, (t1.y * $expr1) as $expr3, t1._row_id, t1._row_id, t1.x] } - └─StreamHashJoin { type: Inner, predicate: t1.x = $expr1, output: [t1.x, t1.y, $expr1, t1._row_id, t1._row_id] } - ├─StreamExchange { dist: HashShard(t1.x) } - │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamShare { id: 3 } - └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } - └─StreamFilter { predicate: (t1.y > 0:Int32) } - └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard($expr1, t1._row_id, t1._row_id, t1._row_id, t1.x) } + └─StreamHashJoin { type: Inner, predicate: $expr1 = $expr2, output: [$expr1, $expr2, $expr3, t1._row_id, t1._row_id, t1._row_id, t1.x] } + ├─StreamExchange { dist: HashShard($expr1) } + │ └─StreamShare { id: 3 } + │ └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } + │ └─StreamFilter { predicate: (t1.y > 0:Int32) } + │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard($expr2) } + └─StreamProject { exprs: [(t1.x * $expr1) as $expr2, (t1.y * $expr1) as $expr3, t1._row_id, t1._row_id, t1.x] } + └─StreamHashJoin { type: Inner, predicate: t1.x = $expr1, output: [t1.x, t1.y, $expr1, t1._row_id, t1._row_id] } + ├─StreamExchange { dist: HashShard(t1.x) } + │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamShare { id: 3 } + └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] } + └─StreamFilter { predicate: (t1.y > 0:Int32) } + └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 003e4185ada81..e07e84e040929 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -346,21 +346,22 @@ └─BatchScan { table: auction, columns: [auction.date_time], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [date_time, window_start, window_end, auction._row_id(hidden)], stream_key: [auction._row_id, window_start, window_end, date_time], pk_columns: [auction._row_id, window_start, window_end, date_time], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: auction.date_time IS NOT DISTINCT FROM auction.date_time, output: all } - ├─StreamExchange { dist: HashShard(auction.date_time) } - │ └─StreamShare { id: 3 } - │ └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] } - │ └─StreamFilter { predicate: IsNotNull(auction.date_time) } - │ └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], pk: [auction._row_id], dist: UpstreamHashShard(auction._row_id) } - └─StreamProject { exprs: [auction.date_time] } - └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] } - └─StreamProject { exprs: [auction.date_time] } - └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] } - └─StreamExchange { dist: HashShard(auction.date_time) } - └─StreamShare { id: 3 } - └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] } - └─StreamFilter { predicate: IsNotNull(auction.date_time) } - └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], pk: [auction._row_id], dist: UpstreamHashShard(auction._row_id) } + └─StreamExchange { dist: HashShard(auction.date_time, window_start, window_end, auction._row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: auction.date_time IS NOT DISTINCT FROM auction.date_time, output: all } + ├─StreamExchange { dist: HashShard(auction.date_time) } + │ └─StreamShare { id: 3 } + │ └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] } + │ └─StreamFilter { predicate: IsNotNull(auction.date_time) } + │ └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], pk: [auction._row_id], dist: UpstreamHashShard(auction._row_id) } + └─StreamProject { exprs: [auction.date_time] } + └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] } + └─StreamProject { exprs: [auction.date_time] } + └─StreamHashAgg { group_key: [auction.date_time], aggs: [count] } + └─StreamExchange { dist: HashShard(auction.date_time) } + └─StreamShare { id: 3 } + └─StreamHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time, window_start, window_end, auction._row_id] } + └─StreamFilter { predicate: IsNotNull(auction.date_time) } + └─StreamTableScan { table: auction, columns: [auction.date_time, auction._row_id], pk: [auction._row_id], dist: UpstreamHashShard(auction._row_id) } - sql: | CREATE TABLE t (v int); SELECT 1 FROM t AS t_inner WHERE EXISTS ( SELECT 1 HAVING t_inner.v > 1); @@ -535,22 +536,23 @@ └─BatchScan { table: t, columns: [t.x], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, y, k, sum_x, t.x(hidden)], stream_key: [k, x], pk_columns: [k, x], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, t.y, t.k, sum(Unnest($0)), t.x] } - ├─StreamExchange { dist: HashShard(t.x) } - │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) } - └─StreamProject { exprs: [t.x, sum(Unnest($0))] } - └─StreamHashAgg { group_key: [t.x], aggs: [sum(Unnest($0)), count] } - └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, Unnest($0), t.x, projected_row_id] } - ├─StreamProject { exprs: [t.x] } - │ └─StreamHashAgg { group_key: [t.x], aggs: [count] } - │ └─StreamExchange { dist: HashShard(t.x) } - │ └─StreamTableScan { table: t, columns: [t.x, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) } - └─StreamProject { exprs: [t.x, Unnest($0), projected_row_id] } - └─StreamProjectSet { select_list: [$0, Unnest($0)] } - └─StreamProject { exprs: [t.x] } - └─StreamHashAgg { group_key: [t.x], aggs: [count] } - └─StreamExchange { dist: HashShard(t.x) } - └─StreamTableScan { table: t, columns: [t.x, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) } + └─StreamExchange { dist: HashShard(t.x, t.k) } + └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, t.y, t.k, sum(Unnest($0)), t.x] } + ├─StreamExchange { dist: HashShard(t.x) } + │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) } + └─StreamProject { exprs: [t.x, sum(Unnest($0))] } + └─StreamHashAgg { group_key: [t.x], aggs: [sum(Unnest($0)), count] } + └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, Unnest($0), t.x, projected_row_id] } + ├─StreamProject { exprs: [t.x] } + │ └─StreamHashAgg { group_key: [t.x], aggs: [count] } + │ └─StreamExchange { dist: HashShard(t.x) } + │ └─StreamTableScan { table: t, columns: [t.x, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) } + └─StreamProject { exprs: [t.x, Unnest($0), projected_row_id] } + └─StreamProjectSet { select_list: [$0, Unnest($0)] } + └─StreamProject { exprs: [t.x] } + └─StreamHashAgg { group_key: [t.x], aggs: [count] } + └─StreamExchange { dist: HashShard(t.x) } + └─StreamTableScan { table: t, columns: [t.x, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) } - name: CorrelatedInputRef in ProjectSet and apply on condition is true. sql: | create table t(x int[], y int[], k int primary key); @@ -632,16 +634,17 @@ └─BatchScan { table: integers, columns: [integers.i], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, col, integers._row_id(hidden), integers.i(hidden)], stream_key: [integers._row_id, i], pk_columns: [i, integers._row_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, row_number, integers._row_id, integers.i] } - ├─StreamExchange { dist: HashShard(integers.i) } - │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } - └─StreamProject { exprs: [integers.i, row_number, integers._row_id] } - └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─StreamExchange { dist: HashShard(integers.i) } - └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } - └─StreamFilter { predicate: IsNotNull(integers.i) } - └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamExchange { dist: HashShard(integers.i, integers._row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, row_number, integers._row_id, integers.i] } + ├─StreamExchange { dist: HashShard(integers.i) } + │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } + └─StreamProject { exprs: [integers.i, row_number, integers._row_id] } + └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(integers.i) } + └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } + └─StreamFilter { predicate: IsNotNull(integers.i) } + └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - name: test over window subquery 2 (with nested loop join so cannot be transformed into a stream plan) sql: | CREATE TABLE integers(i INTEGER); @@ -690,16 +693,17 @@ └─BatchScan { table: integers, columns: [integers.i], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, col, integers._row_id(hidden), integers.i(hidden)], stream_key: [integers._row_id, i], pk_columns: [i, integers._row_id], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, sum, integers._row_id, integers.i] } - ├─StreamExchange { dist: HashShard(integers.i) } - │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } - └─StreamProject { exprs: [integers.i, sum, integers._row_id] } - └─StreamOverWindow { window_functions: [sum(integers.i) OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─StreamExchange { dist: HashShard(integers.i) } - └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } - └─StreamFilter { predicate: IsNotNull(integers.i) } - └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamExchange { dist: HashShard(integers.i, integers._row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, sum, integers._row_id, integers.i] } + ├─StreamExchange { dist: HashShard(integers.i) } + │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } + └─StreamProject { exprs: [integers.i, sum, integers._row_id] } + └─StreamOverWindow { window_functions: [sum(integers.i) OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(integers.i) } + └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } + └─StreamFilter { predicate: IsNotNull(integers.i) } + └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - name: test over window subquery 4 (with nested loop join so cannot be transformed into a stream plan) sql: | CREATE TABLE integers(i INTEGER); @@ -747,17 +751,18 @@ └─BatchScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, integers._row_id(hidden), $expr1(hidden), integers.correlated_col(hidden)], stream_key: [integers._row_id, $expr1, integers.correlated_col], pk_columns: [integers._row_id, $expr1, integers.correlated_col], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: $expr1 = sum AND integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.i, integers._row_id, $expr1, integers.correlated_col] } - ├─StreamExchange { dist: HashShard(integers.correlated_col, $expr1) } - │ └─StreamProject { exprs: [integers.i, integers.correlated_col, integers.i::Int64 as $expr1, integers._row_id] } - │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamExchange { dist: HashShard(rows.correlated_col, sum) } - └─StreamProject { exprs: [rows.correlated_col, sum, rows._row_id, rows.k] } - └─StreamOverWindow { window_functions: [sum(rows.v) OVER(PARTITION BY rows.correlated_col, rows.k ORDER BY rows.v ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─StreamExchange { dist: HashShard(rows.correlated_col, rows.k) } - └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] } - └─StreamFilter { predicate: IsNotNull(rows.correlated_col) } - └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) } + └─StreamExchange { dist: HashShard(integers._row_id, $expr1, integers.correlated_col) } + └─StreamHashJoin { type: LeftSemi, predicate: $expr1 = sum AND integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.i, integers._row_id, $expr1, integers.correlated_col] } + ├─StreamExchange { dist: HashShard(integers.correlated_col, $expr1) } + │ └─StreamProject { exprs: [integers.i, integers.correlated_col, integers.i::Int64 as $expr1, integers._row_id] } + │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamExchange { dist: HashShard(rows.correlated_col, sum) } + └─StreamProject { exprs: [rows.correlated_col, sum, rows._row_id, rows.k] } + └─StreamOverWindow { window_functions: [sum(rows.v) OVER(PARTITION BY rows.correlated_col, rows.k ORDER BY rows.v ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(rows.correlated_col, rows.k) } + └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] } + └─StreamFilter { predicate: IsNotNull(rows.correlated_col) } + └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) } - name: test cardinality visitor with correlated filter sql: | CREATE TABLE t1(i INT); @@ -818,21 +823,22 @@ └─BatchScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, correlated_col, integers._row_id(hidden), 2:Int64(hidden)], stream_key: [integers._row_id, correlated_col, 2:Int64], pk_columns: [integers._row_id, correlated_col, 2:Int64], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: integers.correlated_col IS NOT DISTINCT FROM integers.correlated_col AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col, integers._row_id, 2:Int64] } - ├─StreamExchange { dist: HashShard(integers.correlated_col) } - │ └─StreamProject { exprs: [integers.i, integers.correlated_col, 2:Int64, integers._row_id] } - │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamProject { exprs: [integers.correlated_col, (count(distinct rows.k) + count(distinct rows.v)) as $expr1] } - └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count(distinct rows.k), count(distinct rows.v), count] } - └─StreamHashJoin { type: LeftOuter, predicate: integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.correlated_col, rows.k, rows.v, rows._row_id] } - ├─StreamProject { exprs: [integers.correlated_col] } - │ └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count] } - │ └─StreamExchange { dist: HashShard(integers.correlated_col) } - │ └─StreamTableScan { table: integers, columns: [integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamExchange { dist: HashShard(rows.correlated_col) } - └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] } - └─StreamFilter { predicate: IsNotNull(rows.correlated_col) } - └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) } + └─StreamExchange { dist: HashShard(integers.correlated_col, integers._row_id, 2:Int64) } + └─StreamHashJoin { type: LeftSemi, predicate: integers.correlated_col IS NOT DISTINCT FROM integers.correlated_col AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col, integers._row_id, 2:Int64] } + ├─StreamExchange { dist: HashShard(integers.correlated_col) } + │ └─StreamProject { exprs: [integers.i, integers.correlated_col, 2:Int64, integers._row_id] } + │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamProject { exprs: [integers.correlated_col, (count(distinct rows.k) + count(distinct rows.v)) as $expr1] } + └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count(distinct rows.k), count(distinct rows.v), count] } + └─StreamHashJoin { type: LeftOuter, predicate: integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.correlated_col, rows.k, rows.v, rows._row_id] } + ├─StreamProject { exprs: [integers.correlated_col] } + │ └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count] } + │ └─StreamExchange { dist: HashShard(integers.correlated_col) } + │ └─StreamTableScan { table: integers, columns: [integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamExchange { dist: HashShard(rows.correlated_col) } + └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] } + └─StreamFilter { predicate: IsNotNull(rows.correlated_col) } + └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) } - name: test hop window subquery 1 sql: | create table t1 (k int primary key, ts timestamp); @@ -848,12 +854,13 @@ └─BatchValues { rows: [[1:Int32], [2:Int32]] } stream_plan: |- StreamMaterialize { columns: [col, k, ts, window_start, window_end], stream_key: [col, window_start, window_end], pk_columns: [col, window_start, window_end], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: 1:Int32 = t1.k, output: all } - ├─StreamAppendOnlyDedup { dedup_cols: [1:Int32] } - │ └─StreamExchange { dist: HashShard(1:Int32) } - │ └─StreamProject { exprs: [1:Int32] } - │ └─StreamValues { rows: [[1:Int32, 0:Int64], [2:Int32, 1:Int64]] } - └─StreamExchange { dist: HashShard(t1.k) } - └─StreamHopWindow { time_col: t1.ts, slide: 00:10:00, size: 00:30:00, output: all } - └─StreamFilter { predicate: IsNotNull(t1.ts) } - └─StreamTableScan { table: t1, columns: [t1.k, t1.ts], pk: [t1.k], dist: UpstreamHashShard(t1.k) } + └─StreamExchange { dist: HashShard(1:Int32, window_start, window_end) } + └─StreamHashJoin { type: Inner, predicate: 1:Int32 = t1.k, output: all } + ├─StreamAppendOnlyDedup { dedup_cols: [1:Int32] } + │ └─StreamExchange { dist: HashShard(1:Int32) } + │ └─StreamProject { exprs: [1:Int32] } + │ └─StreamValues { rows: [[1:Int32, 0:Int64], [2:Int32, 1:Int64]] } + └─StreamExchange { dist: HashShard(t1.k) } + └─StreamHopWindow { time_col: t1.ts, slide: 00:10:00, size: 00:30:00, output: all } + └─StreamFilter { predicate: IsNotNull(t1.ts) } + └─StreamTableScan { table: t1, columns: [t1.k, t1.ts], pk: [t1.k], dist: UpstreamHashShard(t1.k) } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml index 3b84def33abfa..0d393c378ff85 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml @@ -717,15 +717,16 @@ └─BatchScan { table: t2, columns: [t2.x], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, y, t1._row_id(hidden)], stream_key: [t1._row_id, x], pk_columns: [t1._row_id, x], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: t1.x IS NOT DISTINCT FROM t2.x, output: all } - ├─StreamExchange { dist: HashShard(t1.x) } - │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamProject { exprs: [t2.x] } - └─StreamGroupTopN { order: [t2.x ASC], limit: 1, offset: 0, group_key: [t2.x] } - └─StreamExchange { dist: HashShard(t2.x) } - └─StreamProject { exprs: [t2.x, t2.x, t2._row_id] } - └─StreamFilter { predicate: IsNotNull(t2.x) } - └─StreamTableScan { table: t2, columns: [t2.x, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.x, t1._row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: t1.x IS NOT DISTINCT FROM t2.x, output: all } + ├─StreamExchange { dist: HashShard(t1.x) } + │ └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamProject { exprs: [t2.x] } + └─StreamGroupTopN { order: [t2.x ASC], limit: 1, offset: 0, group_key: [t2.x] } + └─StreamExchange { dist: HashShard(t2.x) } + └─StreamProject { exprs: [t2.x, t2.x, t2._row_id] } + └─StreamFilter { predicate: IsNotNull(t2.x) } + └─StreamTableScan { table: t2, columns: [t2.x, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -883,14 +884,15 @@ └─BatchScan { table: t2, columns: [t2.v2, t2.k2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, k1, t1._row_id(hidden)], stream_key: [t1._row_id, v1, k1], pk_columns: [t1._row_id, v1, k1], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2 AND t1.k1 IS NOT DISTINCT FROM t2.k2, output: all } - ├─StreamExchange { dist: HashShard(t1.k1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamGroupTopN { order: [t2.v2 ASC], limit: 1, offset: 0, group_key: [t2.k2] } - └─StreamExchange { dist: HashShard(t2.k2) } - └─StreamProject { exprs: [t2.k2, t2.v2, t2._row_id] } - └─StreamFilter { predicate: IsNotNull(t2.k2) } - └─StreamTableScan { table: t2, columns: [t2.v2, t2.k2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1.k1, t1._row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2 AND t1.k1 IS NOT DISTINCT FROM t2.k2, output: all } + ├─StreamExchange { dist: HashShard(t1.k1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamGroupTopN { order: [t2.v2 ASC], limit: 1, offset: 0, group_key: [t2.k2] } + └─StreamExchange { dist: HashShard(t2.k2) } + └─StreamProject { exprs: [t2.k2, t2.v2, t2._row_id] } + └─StreamFilter { predicate: IsNotNull(t2.k2) } + └─StreamTableScan { table: t2, columns: [t2.v2, t2.k2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: test ApplyTopNTransposeRule case 2 sql: | create table t1 (v1 int, k1 int); @@ -908,16 +910,17 @@ └─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, k1, t1._row_id(hidden)], stream_key: [t1._row_id, v1], pk_columns: [t1._row_id, v1], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2, output: all } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.v2) } - └─StreamProject { exprs: [t2.v2, t2._row_id] } - └─StreamTopN { order: [t2.v2 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [t2.v2 ASC], limit: 1, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } - └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2, output: all } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v2) } + └─StreamProject { exprs: [t2.v2, t2._row_id] } + └─StreamTopN { order: [t2.v2 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [t2.v2 ASC], limit: 1, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } + └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: test ApplyLimitTransposeRule case 1 sql: | create table t1 (v1 int, k1 int); @@ -935,11 +938,12 @@ └─BatchScan { table: t2, columns: [t2.v2, t2.k2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, k1, t1._row_id(hidden)], stream_key: [t1._row_id, v1, k1], pk_columns: [t1._row_id, v1, k1], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2 AND t1.k1 IS NOT DISTINCT FROM t2.k2, output: all } - ├─StreamExchange { dist: HashShard(t1.k1) } - │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamGroupTopN { order: [t2.k2 ASC], limit: 1, offset: 0, group_key: [t2.k2] } - └─StreamExchange { dist: HashShard(t2.k2) } - └─StreamProject { exprs: [t2.k2, t2.v2, t2._row_id] } - └─StreamFilter { predicate: IsNotNull(t2.k2) } - └─StreamTableScan { table: t2, columns: [t2.v2, t2.k2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1.k1, t1._row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: t1.v1 = t2.v2 AND t1.k1 IS NOT DISTINCT FROM t2.k2, output: all } + ├─StreamExchange { dist: HashShard(t1.k1) } + │ └─StreamTableScan { table: t1, columns: [t1.v1, t1.k1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamGroupTopN { order: [t2.k2 ASC], limit: 1, offset: 0, group_key: [t2.k2] } + └─StreamExchange { dist: HashShard(t2.k2) } + └─StreamProject { exprs: [t2.k2, t2.v2, t2._row_id] } + └─StreamFilter { predicate: IsNotNull(t2.k2) } + └─StreamTableScan { table: t2, columns: [t2.v2, t2.k2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml index 6673d86fd9745..29e391853cf8a 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml @@ -122,19 +122,20 @@ select * from t1 join t2 on a = b AND ta < now() - interval '1 hour' and ta >= now() - interval '2 hour'; stream_plan: |- StreamMaterialize { columns: [a, ta, b, tb, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, a], pk_columns: [t1._row_id, t2._row_id, a], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: t1.a = t2.b, output: [t1.a, t1.ta, t2.b, t2.tb, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.a) } - │ └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id] } - │ ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true } - │ │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ │ └─StreamExchange { dist: Broadcast } - │ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } - │ │ └─StreamNow { output: [now] } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: HashShard(t2.b) } - └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.a, t1._row_id, t2._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.a = t2.b, output: [t1.a, t1.ta, t2.b, t2.tb, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.a) } + │ └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id] } + │ ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true } + │ │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ └─StreamExchange { dist: Broadcast } + │ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ │ └─StreamNow { output: [now] } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(t2.b) } + └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Temporal filter in on clause for left join's left side sql: | create table t1 (a int, ta timestamp with time zone); @@ -150,19 +151,20 @@ select * from t1 right join t2 on a = b AND ta < now() - interval '1 hour' and ta >= now() - interval '2 hour'; stream_plan: |- StreamMaterialize { columns: [a, ta, b, tb, t2._row_id(hidden), t1._row_id(hidden)], stream_key: [t2._row_id, t1._row_id, b], pk_columns: [t2._row_id, t1._row_id, b], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: t2.b = t1.a, output: [t1.a, t1.ta, t2.b, t2.tb, t2._row_id, t1._row_id] } - ├─StreamExchange { dist: HashShard(t2.b) } - │ └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - └─StreamExchange { dist: HashShard(t1.a) } - └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id] } - ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true } - │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } - └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(t2.b, t2._row_id, t1._row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: t2.b = t1.a, output: [t1.a, t1.ta, t2.b, t2.tb, t2._row_id, t1._row_id] } + ├─StreamExchange { dist: HashShard(t2.b) } + │ └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.a) } + └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id] } + ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true } + │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + └─StreamNow { output: [now] } - name: Temporal filter in on clause for full join's left side sql: | create table t1 (a int, ta timestamp with time zone); @@ -178,19 +180,20 @@ select * from t1 left join t2 on a = b AND tb < now() - interval '1 hour' and tb >= now() - interval '2 hour'; stream_plan: |- StreamMaterialize { columns: [a, ta, b, tb, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, a], pk_columns: [t1._row_id, t2._row_id, a], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftOuter, predicate: t1.a = t2.b, output: [t1.a, t1.ta, t2.b, t2.tb, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.a) } - │ └─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.b) } - └─StreamDynamicFilter { predicate: (t2.tb < $expr2), output: [t2.b, t2.tb, t2._row_id] } - ├─StreamDynamicFilter { predicate: (t2.tb >= $expr1), output_watermarks: [t2.tb], output: [t2.b, t2.tb, t2._row_id], cleaned_by_watermark: true } - │ ├─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } - │ └─StreamNow { output: [now] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } - └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(t1.a, t1._row_id, t2._row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: t1.a = t2.b, output: [t1.a, t1.ta, t2.b, t2.tb, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.a) } + │ └─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.b) } + └─StreamDynamicFilter { predicate: (t2.tb < $expr2), output: [t2.b, t2.tb, t2._row_id] } + ├─StreamDynamicFilter { predicate: (t2.tb >= $expr1), output_watermarks: [t2.tb], output: [t2.b, t2.tb, t2._row_id], cleaned_by_watermark: true } + │ ├─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + └─StreamNow { output: [now] } - name: Temporal filter in on clause for right join's right side sql: | create table t1 (a int, ta timestamp with time zone); diff --git a/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml b/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml index a1020b8d16ee5..f49a82be2dd78 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml @@ -6,11 +6,12 @@ select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on id1= id2 stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2, output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } - ├─StreamExchange { dist: HashShard(stream.id1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } - └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } + └─StreamExchange { dist: HashShard(stream.id1, stream._row_id) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2, output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } + ├─StreamExchange { dist: HashShard(stream.id1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } + └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } batch_error: |- Not supported: do not support temporal join for batch queries HINT: please use temporal join in streaming queries @@ -21,11 +22,12 @@ select id1, a1, id2, a2 from stream join version FOR SYSTEM_TIME AS OF PROCTIME() on id1 = id2 where a2 < 10; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } - ├─StreamExchange { dist: HashShard(stream.id1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } - └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } + └─StreamExchange { dist: HashShard(stream.id1, stream._row_id) } + └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } + ├─StreamExchange { dist: HashShard(stream.id1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } + └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } - name: implicit join with temporal tables sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -33,11 +35,12 @@ select id1, a1, id2, a2 from stream, version FOR SYSTEM_TIME AS OF PROCTIME() where id1 = id2 AND a2 < 10; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } - ├─StreamExchange { dist: HashShard(stream.id1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } - └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } + └─StreamExchange { dist: HashShard(stream.id1, stream._row_id) } + └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } + ├─StreamExchange { dist: HashShard(stream.id1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } + └─StreamTableScan { table: version, columns: [version.id2, version.a2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } - name: Multi join key for temporal join sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -45,11 +48,12 @@ select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and id1 = id2 where b2 != a2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1, a1], pk_columns: [stream._row_id, id1, a1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND stream.a1 = version.a2 AND (version.b2 <> version.a2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } - ├─StreamExchange { dist: HashShard(stream.id1, stream.a1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2, version.a2) } - └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2], pk: [version.id2, version.a2], dist: UpstreamHashShard(version.id2, version.a2) } + └─StreamExchange { dist: HashShard(stream.id1, stream.a1, stream._row_id) } + └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND stream.a1 = version.a2 AND (version.b2 <> version.a2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } + ├─StreamExchange { dist: HashShard(stream.id1, stream.a1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2, version.a2) } + └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2], pk: [version.id2, version.a2], dist: UpstreamHashShard(version.id2, version.a2) } - name: Temporal join with Aggregation sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -101,15 +105,16 @@ join version2 FOR SYSTEM_TIME AS OF PROCTIME() on stream.k = version2.k where a1 < 10; stream_plan: |- StreamMaterialize { columns: [k, x1, x2, a1, b1, stream._row_id(hidden), version2.k(hidden)], stream_key: [stream._row_id, k], pk_columns: [stream._row_id, k], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.k = version2.k, output: [stream.k, version1.x1, version2.x2, stream.a1, stream.b1, stream._row_id, version2.k] } - ├─StreamTemporalJoin { type: Inner, predicate: stream.k = version1.k, output: [stream.k, stream.a1, stream.b1, version1.x1, stream._row_id, version1.k] } - │ ├─StreamExchange { dist: HashShard(stream.k) } - │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } - │ │ └─StreamTableScan { table: stream, columns: [stream.k, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.k) } - │ └─StreamTableScan { table: version1, columns: [version1.k, version1.x1], pk: [version1.k], dist: UpstreamHashShard(version1.k) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.k) } - └─StreamTableScan { table: version2, columns: [version2.k, version2.x2], pk: [version2.k], dist: UpstreamHashShard(version2.k) } + └─StreamExchange { dist: HashShard(stream.k, stream._row_id) } + └─StreamTemporalJoin { type: Inner, predicate: stream.k = version2.k, output: [stream.k, version1.x1, version2.x2, stream.a1, stream.b1, stream._row_id, version2.k] } + ├─StreamTemporalJoin { type: Inner, predicate: stream.k = version1.k, output: [stream.k, stream.a1, stream.b1, version1.x1, stream._row_id, version1.k] } + │ ├─StreamExchange { dist: HashShard(stream.k) } + │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } + │ │ └─StreamTableScan { table: stream, columns: [stream.k, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.k) } + │ └─StreamTableScan { table: version1, columns: [version1.k, version1.x1], pk: [version1.k], dist: UpstreamHashShard(version1.k) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.k) } + └─StreamTableScan { table: version2, columns: [version2.k, version2.x2], pk: [version2.k], dist: UpstreamHashShard(version2.k) } - name: multi-way temporal join with different keys sql: | create table stream(id1 int, id2 int, a1 int, b1 int) APPEND ONLY; @@ -121,16 +126,17 @@ join version2 FOR SYSTEM_TIME AS OF PROCTIME() on stream.id2 = version2.id2 where a1 < 10; stream_plan: |- StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version2.id2(hidden)], stream_key: [stream._row_id, id1, id2], pk_columns: [stream._row_id, id1, id2], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } - ├─StreamExchange { dist: HashShard(stream.id2) } - │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } - │ ├─StreamExchange { dist: HashShard(stream.id1) } - │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } - │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.id1) } - │ └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.id2) } - └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) } + └─StreamExchange { dist: HashShard(stream.id1, stream.id2, stream._row_id) } + └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } + ├─StreamExchange { dist: HashShard(stream.id2) } + │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } + │ ├─StreamExchange { dist: HashShard(stream.id1) } + │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } + │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.id1) } + │ └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.id2) } + └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) } - name: multi-way temporal join with different keys sql: | create table stream(id1 int, id2 int, a1 int, b1 int) APPEND ONLY; @@ -142,16 +148,17 @@ join version2 FOR SYSTEM_TIME AS OF PROCTIME() on stream.id2 = version2.id2 where a1 < 10; stream_plan: |- StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version2.id2(hidden)], stream_key: [stream._row_id, id1, id2], pk_columns: [stream._row_id, id1, id2], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } - ├─StreamExchange { dist: HashShard(stream.id2) } - │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } - │ ├─StreamExchange { dist: HashShard(stream.id1) } - │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } - │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.id1) } - │ └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.id2) } - └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) } + └─StreamExchange { dist: HashShard(stream.id1, stream.id2, stream._row_id) } + └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } + ├─StreamExchange { dist: HashShard(stream.id2) } + │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } + │ ├─StreamExchange { dist: HashShard(stream.id1) } + │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } + │ │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.id2, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + │ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version1.id1) } + │ └─StreamTableScan { table: version1, columns: [version1.id1, version1.x1], pk: [version1.id1], dist: UpstreamHashShard(version1.id1) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version2.id2) } + └─StreamTableScan { table: version2, columns: [version2.id2, version2.x2], pk: [version2.id2], dist: UpstreamHashShard(version2.id2) } - name: temporal join with an index (distribution key size = 1) sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -160,11 +167,12 @@ select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } - ├─StreamExchange { dist: HashShard(stream.a1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } - └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) } + └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } + ├─StreamExchange { dist: HashShard(stream.a1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } + └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) } - name: temporal join with an index (distribution key size = 2) sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -173,11 +181,12 @@ select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } - ├─StreamExchange { dist: HashShard(stream.a1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } - └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) } + └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } + ├─StreamExchange { dist: HashShard(stream.a1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } + └─StreamTableScan { table: idx2, columns: [idx2.a2, idx2.b2, idx2.id2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) } - name: temporal join with an index (index column size = 1) sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -186,11 +195,12 @@ select id1, a1, id2, a2 from stream left join idx2 FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, stream.b1, a1], pk_columns: [stream._row_id, id2, stream.b1, a1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.b1 = idx2.b2 AND (stream.a1 = idx2.a2), output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } - ├─StreamExchange { dist: HashShard(stream.b1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.b2) } - └─StreamTableScan { table: idx2, columns: [idx2.b2, idx2.id2, idx2.a2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.b2) } + └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.b1 = idx2.b2 AND (stream.a1 = idx2.a2), output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } + ├─StreamExchange { dist: HashShard(stream.b1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.b2) } + └─StreamTableScan { table: idx2, columns: [idx2.b2, idx2.id2, idx2.a2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.b2) } - name: temporal join with singleton table sql: | create table t (a int) append only; @@ -212,11 +222,12 @@ select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx.a2 AND stream.b1 = idx.b2, output: [stream.id1, stream.a1, idx.id2, idx.a2, stream._row_id, stream.b1] } - ├─StreamExchange { dist: HashShard(stream.a1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx.a2) } - └─StreamTableScan { table: idx, columns: [idx.id2, idx.a2, idx.b2], pk: [idx.id2], dist: UpstreamHashShard(idx.a2) } + └─StreamExchange { dist: HashShard(stream.a1, idx.id2, stream._row_id, stream.b1) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx.a2 AND stream.b1 = idx.b2, output: [stream.id1, stream.a1, idx.id2, idx.a2, stream._row_id, stream.b1] } + ├─StreamExchange { dist: HashShard(stream.a1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx.a2) } + └─StreamTableScan { table: idx, columns: [idx.id2, idx.a2, idx.b2], pk: [idx.id2], dist: UpstreamHashShard(idx.a2) } - name: index selection for temporal join (with two indexes) and should choose the index with a longer prefix.. sql: | create table stream(id1 int, a1 int, b1 int) APPEND ONLY; @@ -226,11 +237,12 @@ select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden)], stream_key: [stream._row_id, id2, a1, stream.b1], pk_columns: [stream._row_id, id2, a1, stream.b1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } - ├─StreamExchange { dist: HashShard(stream.a1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } - └─StreamTableScan { table: idx2, columns: [idx2.id2, idx2.a2, idx2.b2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) } + └─StreamExchange { dist: HashShard(stream.a1, idx2.id2, stream._row_id, stream.b1) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.a1 = idx2.a2 AND stream.b1 = idx2.b2, output: [stream.id1, stream.a1, idx2.id2, idx2.a2, stream._row_id, stream.b1] } + ├─StreamExchange { dist: HashShard(stream.a1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(idx2.a2) } + └─StreamTableScan { table: idx2, columns: [idx2.id2, idx2.a2, idx2.b2], pk: [idx2.id2], dist: UpstreamHashShard(idx2.a2) } - name: index selection for temporal join (with three indexes) and should choose primary table. sql: | create table stream(id1 int, a1 int, b1 int, c1 int) APPEND ONLY; @@ -241,11 +253,12 @@ select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2 and c1 = c2 and id1 = id2; stream_plan: |- StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden), stream.c1(hidden)], stream_key: [stream._row_id, id1, a1, stream.b1, stream.c1], pk_columns: [stream._row_id, id1, a1, stream.b1, stream.c1], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2 AND (stream.a1 = version.a2) AND (stream.b1 = version.b2) AND (stream.c1 = version.c2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id, stream.b1, stream.c1] } - ├─StreamExchange { dist: HashShard(stream.id1) } - │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream.c1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } - └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } - └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2, version.c2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } + └─StreamExchange { dist: HashShard(stream.id1, stream.a1, stream._row_id, stream.b1, stream.c1) } + └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2 AND (stream.a1 = version.a2) AND (stream.b1 = version.b2) AND (stream.c1 = version.c2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id, stream.b1, stream.c1] } + ├─StreamExchange { dist: HashShard(stream.id1) } + │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream.c1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } + └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(version.id2) } + └─StreamTableScan { table: version, columns: [version.id2, version.a2, version.b2, version.c2], pk: [version.id2], dist: UpstreamHashShard(version.id2) } - name: index selection for temporal join (two index) and no one matches. sql: | create table stream(id1 int, a1 int, b1 int, c1 int) APPEND ONLY; diff --git a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml index eaaa1f8e5c8d5..fdf928a0c9c84 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml @@ -243,214 +243,218 @@ └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), _row_id#1(hidden), r_regionkey(hidden), _row_id#2(hidden), _row_id#3(hidden), _row_id#4(hidden), ps_suppkey(hidden), n_nationkey(hidden), ps_supplycost(hidden), p_partkey#1(hidden)], stream_key: [_row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, p_partkey, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_conflict: NoCheck } - └─StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } - ├─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamShare { id: 26 } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } - │ │ │ └─StreamShare { id: 3 } - │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] } - │ │ │ └─StreamRowIdGen { row_id_index: 3 } - │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ │ └─StreamShare { id: 7 } - │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 4 } - │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(p_partkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 9 } - │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamShare { id: 15 } - │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamShare { id: 21 } - │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - └─StreamProject { exprs: [p_partkey, min(ps_supplycost)] } - └─StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] } - └─StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] } - ├─StreamAppendOnlyDedup { dedup_cols: [p_partkey] } - │ └─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamProject { exprs: [p_partkey] } - │ └─StreamShare { id: 26 } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } - │ │ │ └─StreamShare { id: 3 } - │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] } - │ │ │ └─StreamRowIdGen { row_id_index: 3 } - │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ │ └─StreamShare { id: 7 } - │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 4 } - │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(p_partkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 9 } - │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamShare { id: 15 } - │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamShare { id: 21 } - │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - └─StreamExchange { dist: HashShard(ps_partkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } - ├─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ └─StreamFilter { predicate: IsNotNull(ps_partkey) } - │ │ └─StreamShare { id: 15 } - │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamShare { id: 21 } - │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - └─StreamExchange { dist: HashShard(n_nationkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } - ├─StreamExchange { dist: HashShard(r_regionkey) } - │ └─StreamShare { id: 3 } - │ └─StreamProject { exprs: [r_regionkey, _row_id] } - │ └─StreamRowIdGen { row_id_index: 3 } - │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } - └─StreamExchange { dist: HashShard(n_regionkey) } - └─StreamShare { id: 7 } - └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - └─StreamRowIdGen { row_id_index: 4 } - └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─StreamExchange { dist: HashShard(p_partkey, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost) } + └─StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } + ├─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamShare { id: 26 } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ │ └─StreamShare { id: 3 } + │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 3 } + │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } + │ │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ │ └─StreamShare { id: 7 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 4 } + │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 9 } + │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ │ └─StreamShare { id: 15 } + │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamShare { id: 21 } + │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + └─StreamProject { exprs: [p_partkey, min(ps_supplycost)] } + └─StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] } + └─StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] } + ├─StreamAppendOnlyDedup { dedup_cols: [p_partkey] } + │ └─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamProject { exprs: [p_partkey] } + │ └─StreamShare { id: 26 } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ │ └─StreamShare { id: 3 } + │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 3 } + │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } + │ │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ │ └─StreamShare { id: 7 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 4 } + │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 9 } + │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ │ └─StreamShare { id: 15 } + │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamShare { id: 21 } + │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + └─StreamExchange { dist: HashShard(ps_partkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } + ├─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ └─StreamFilter { predicate: IsNotNull(ps_partkey) } + │ │ └─StreamShare { id: 15 } + │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamShare { id: 21 } + │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + └─StreamExchange { dist: HashShard(n_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } + ├─StreamExchange { dist: HashShard(r_regionkey) } + │ └─StreamShare { id: 3 } + │ └─StreamProject { exprs: [r_regionkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 3 } + │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } + └─StreamExchange { dist: HashShard(n_regionkey) } + └─StreamShare { id: 7 } + └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + └─StreamRowIdGen { row_id_index: 4 } + └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), _row_id#1(hidden), r_regionkey(hidden), _row_id#2(hidden), _row_id#3(hidden), _row_id#4(hidden), ps_suppkey(hidden), n_nationkey(hidden), ps_supplycost(hidden), p_partkey#1(hidden)], stream_key: [_row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, p_partkey, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamProject { exprs: [p_partkey, min(ps_supplycost)] } - └── StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] } { intermediate state table: 26, state tables: [ 25 ], distinct tables: [] } - └── StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] } { left table: 27, right table: 29, left degree table: 28, right degree table: 30 } - ├── StreamAppendOnlyDedup { dedup_cols: [p_partkey] } { state table: 31 } - │ └── StreamExchange Hash([0]) from 15 - └── StreamExchange Hash([0]) from 16 + └── StreamExchange Hash([3, 8, 9, 10, 11, 12, 13, 14, 15, 16]) from 1 Fragment 1 - StreamNoOp - └── StreamExchange NoShuffle from 2 + StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [p_partkey, min(ps_supplycost)] } + └── StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] } { intermediate state table: 26, state tables: [ 25 ], distinct tables: [] } + └── StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] } { left table: 27, right table: 29, left degree table: 28, right degree table: 30 } + ├── StreamAppendOnlyDedup { dedup_cols: [p_partkey] } { state table: 31 } + │ └── StreamExchange Hash([0]) from 16 + └── StreamExchange Hash([0]) from 17 Fragment 2 - StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([5]) from 8 + StreamNoOp + └── StreamExchange NoShuffle from 3 Fragment 3 - StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } { left table: 8, right table: 10, left degree table: 9, right degree table: 11 } + StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([2]) from 6 + └── StreamExchange Hash([5]) from 9 Fragment 4 - StreamNoOp - └── StreamExchange NoShuffle from 5 + StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } { left table: 8, right table: 10, left degree table: 9, right degree table: 11 } + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([2]) from 7 Fragment 5 + StreamNoOp + └── StreamExchange NoShuffle from 6 + + Fragment 6 StreamProject { exprs: [r_regionkey, _row_id] } └── StreamRowIdGen { row_id_index: 3 } └── StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } { source state table: 12 } - Fragment 6 + Fragment 7 StreamNoOp - └── StreamExchange NoShuffle from 7 + └── StreamExchange NoShuffle from 8 - Fragment 7 + Fragment 8 StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } └── StreamRowIdGen { row_id_index: 4 } └── StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } { source state table: 13 } - Fragment 8 + Fragment 9 StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 } - ├── StreamExchange Hash([2]) from 9 - └── StreamExchange Hash([0]) from 13 + ├── StreamExchange Hash([2]) from 10 + └── StreamExchange Hash([0]) from 14 - Fragment 9 + Fragment 10 StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } { left table: 18, right table: 20, left degree table: 19, right degree table: 21 } - ├── StreamExchange Hash([0]) from 10 - └── StreamExchange Hash([0]) from 11 + ├── StreamExchange Hash([0]) from 11 + └── StreamExchange Hash([0]) from 12 - Fragment 10 + Fragment 11 StreamRowIdGen { row_id_index: 9 } └── StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } { source state table: 22 } - Fragment 11 + Fragment 12 StreamNoOp - └── StreamExchange NoShuffle from 12 + └── StreamExchange NoShuffle from 13 - Fragment 12 + Fragment 13 StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } └── StreamRowIdGen { row_id_index: 5 } └── StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } { source state table: 23 } - Fragment 13 + Fragment 14 StreamNoOp - └── StreamExchange NoShuffle from 14 + └── StreamExchange NoShuffle from 15 - Fragment 14 + Fragment 15 StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } └── StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } { source state table: 24 } - Fragment 15 + Fragment 16 StreamProject { exprs: [p_partkey] } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 - Fragment 16 + Fragment 17 StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } { left table: 32, right table: 34, left degree table: 33, right degree table: 35 } - ├── StreamExchange Hash([2]) from 17 - └── StreamExchange Hash([0]) from 20 + ├── StreamExchange Hash([2]) from 18 + └── StreamExchange Hash([0]) from 21 - Fragment 17 + Fragment 18 StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } { left table: 36, right table: 38, left degree table: 37, right degree table: 39 } - ├── StreamExchange Hash([1]) from 18 - └── StreamExchange Hash([0]) from 19 + ├── StreamExchange Hash([1]) from 19 + └── StreamExchange Hash([0]) from 20 - Fragment 18 + Fragment 19 StreamFilter { predicate: IsNotNull(ps_partkey) } - └── StreamExchange NoShuffle from 12 + └── StreamExchange NoShuffle from 13 - Fragment 19 + Fragment 20 StreamNoOp - └── StreamExchange NoShuffle from 14 + └── StreamExchange NoShuffle from 15 - Fragment 20 + Fragment 21 StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } { left table: 40, right table: 42, left degree table: 41, right degree table: 43 } - ├── StreamExchange Hash([0]) from 21 - └── StreamExchange Hash([2]) from 22 + ├── StreamExchange Hash([0]) from 22 + └── StreamExchange Hash([2]) from 23 - Fragment 21 + Fragment 22 StreamNoOp - └── StreamExchange NoShuffle from 5 + └── StreamExchange NoShuffle from 6 - Fragment 22 + Fragment 23 StreamNoOp - └── StreamExchange NoShuffle from 7 + └── StreamExchange NoShuffle from 8 Table 0 { columns: [ p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id_0, r_regionkey, n_nationkey, _row_id_1, _row_id_2, _row_id_3, ps_suppkey ], primary key: [ $0 ASC, $7 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC, $12 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16 ], distribution key: [ 0 ], read pk prefix len hint: 2 } @@ -540,7 +544,7 @@ Table 43 { columns: [ n_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost, p_partkey#1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 3 ], read pk prefix len hint: 13 } + Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost, p_partkey#1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 3, 8, 9, 10, 11, 12, 13, 14, 15, 16 ], read pk prefix len hint: 13 } - id: tpch_q5 before: @@ -1797,76 +1801,80 @@ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [_row_id, _row_id#1, s_nationkey, s_suppkey], pk_columns: [s_name, _row_id, _row_id#1, s_nationkey, s_suppkey], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } - ├─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(s_nationkey) } - │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(n_nationkey) } - │ └─StreamRowIdGen { row_id_index: 4 } - │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } - └─StreamExchange { dist: HashShard(ps_suppkey) } - └─StreamProject { exprs: [ps_suppkey, _row_id, ps_partkey, ps_partkey, ps_suppkey] } - └─StreamFilter { predicate: ($expr1 > $expr2) } - └─StreamHashJoin { type: Inner, predicate: ps_partkey IS NOT DISTINCT FROM ps_partkey AND ps_suppkey IS NOT DISTINCT FROM ps_suppkey, output: all } - ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } - │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] } - │ └─StreamShare { id: 13 } - │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamProject { exprs: [p_partkey, _row_id] } - │ └─StreamRowIdGen { row_id_index: 9 } - │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } - └─StreamProject { exprs: [ps_partkey, ps_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } - └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [sum(l_quantity), count] } - └─StreamHashJoin { type: LeftOuter, predicate: ps_partkey IS NOT DISTINCT FROM l_partkey AND ps_suppkey IS NOT DISTINCT FROM l_suppkey, output: [ps_partkey, ps_suppkey, l_quantity, _row_id] } - ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } - │ └─StreamProject { exprs: [ps_partkey, ps_suppkey] } - │ └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [count] } - │ └─StreamShare { id: 13 } - │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamProject { exprs: [p_partkey, _row_id] } - │ └─StreamRowIdGen { row_id_index: 9 } - │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } - └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) } - └─StreamProject { exprs: [l_partkey, l_suppkey, l_quantity, _row_id] } - └─StreamFilter { predicate: IsNotNull(l_partkey) AND IsNotNull(l_suppkey) } - └─StreamRowIdGen { row_id_index: 16 } - └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + └─StreamExchange { dist: HashShard(_row_id, _row_id, s_nationkey, s_suppkey) } + └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamRowIdGen { row_id_index: 4 } + │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─StreamExchange { dist: HashShard(ps_suppkey) } + └─StreamProject { exprs: [ps_suppkey, _row_id, ps_partkey, ps_partkey, ps_suppkey] } + └─StreamFilter { predicate: ($expr1 > $expr2) } + └─StreamHashJoin { type: Inner, predicate: ps_partkey IS NOT DISTINCT FROM ps_partkey AND ps_suppkey IS NOT DISTINCT FROM ps_suppkey, output: all } + ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } + │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] } + │ └─StreamShare { id: 13 } + │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_partkey) } + │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamProject { exprs: [p_partkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 9 } + │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + └─StreamProject { exprs: [ps_partkey, ps_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } + └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [sum(l_quantity), count] } + └─StreamHashJoin { type: LeftOuter, predicate: ps_partkey IS NOT DISTINCT FROM l_partkey AND ps_suppkey IS NOT DISTINCT FROM l_suppkey, output: [ps_partkey, ps_suppkey, l_quantity, _row_id] } + ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } + │ └─StreamProject { exprs: [ps_partkey, ps_suppkey] } + │ └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [count] } + │ └─StreamShare { id: 13 } + │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_partkey) } + │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamProject { exprs: [p_partkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 9 } + │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) } + └─StreamProject { exprs: [l_partkey, l_suppkey, l_quantity, _row_id] } + └─StreamFilter { predicate: IsNotNull(l_partkey) AND IsNotNull(l_suppkey) } + └─StreamRowIdGen { row_id_index: 16 } + └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [_row_id, _row_id#1, s_nationkey, s_suppkey], pk_columns: [s_name, _row_id, _row_id#1, s_nationkey, s_suppkey], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } - ├── StreamExchange Hash([0]) from 1 - └── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([2, 3, 4, 5]) from 1 Fragment 1 - StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } - ├── StreamExchange Hash([3]) from 2 - └── StreamExchange Hash([0]) from 3 + StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 5 Fragment 2 + StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } + ├── StreamExchange Hash([3]) from 3 + └── StreamExchange Hash([0]) from 4 + + Fragment 3 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } { source state table: 8 } - Fragment 3 + Fragment 4 StreamRowIdGen { row_id_index: 4 } └── StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } { source state table: 9 } - Fragment 4 + Fragment 5 StreamProject { exprs: [ps_suppkey, _row_id, ps_partkey, ps_partkey, ps_suppkey] } └── StreamFilter { predicate: ($expr1 > $expr2) } └── StreamHashJoin { type: Inner, predicate: ps_partkey IS NOT DISTINCT FROM ps_partkey AND ps_suppkey IS NOT DISTINCT FROM ps_suppkey, output: all } { left table: 10, right table: 12, left degree table: 11, right degree table: 13 } - ├── StreamExchange Hash([0, 1]) from 5 + ├── StreamExchange Hash([0, 1]) from 6 └── StreamProject { exprs: [ps_partkey, ps_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } └── StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [sum(l_quantity), count] } { intermediate state table: 20, state tables: [], distinct tables: [] } └── StreamHashJoin { type: LeftOuter, predicate: ps_partkey IS NOT DISTINCT FROM l_partkey AND ps_suppkey IS NOT DISTINCT FROM l_suppkey, output: [ps_partkey, ps_suppkey, l_quantity, _row_id] } @@ -1874,33 +1882,33 @@ ├── right table: 23 ├── left degree table: 22 ├── right degree table: 24 - ├── StreamExchange Hash([0, 1]) from 9 - └── StreamExchange Hash([0, 1]) from 10 + ├── StreamExchange Hash([0, 1]) from 10 + └── StreamExchange Hash([0, 1]) from 11 - Fragment 5 + Fragment 6 StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] } - └── StreamExchange NoShuffle from 6 + └── StreamExchange NoShuffle from 7 - Fragment 6 + Fragment 7 StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 } - ├── StreamExchange Hash([0]) from 7 - └── StreamExchange Hash([0]) from 8 + ├── StreamExchange Hash([0]) from 8 + └── StreamExchange Hash([0]) from 9 - Fragment 7 + Fragment 8 StreamRowIdGen { row_id_index: 5 } └── StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } { source state table: 18 } - Fragment 8 + Fragment 9 StreamProject { exprs: [p_partkey, _row_id] } └── StreamRowIdGen { row_id_index: 9 } └── StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } { source state table: 19 } - Fragment 9 + Fragment 10 StreamProject { exprs: [ps_partkey, ps_suppkey] } └── StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [count] } { intermediate state table: 25, state tables: [], distinct tables: [] } - └── StreamExchange NoShuffle from 6 + └── StreamExchange NoShuffle from 7 - Fragment 10 + Fragment 11 StreamProject { exprs: [l_partkey, l_suppkey, l_quantity, _row_id] } └── StreamFilter { predicate: IsNotNull(l_partkey) AND IsNotNull(l_suppkey) } └── StreamRowIdGen { row_id_index: 16 } @@ -1961,7 +1969,7 @@ Table 26 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ s_name, s_address, _row_id, _row_id#1, s_nationkey, s_suppkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 5 } + Table 4294967294 { columns: [ s_name, s_address, _row_id, _row_id#1, s_nationkey, s_suppkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2, 3, 4, 5 ], read pk prefix len hint: 5 } - id: tpch_q21 before: diff --git a/src/frontend/planner_test/tests/testdata/output/watermark.yaml b/src/frontend/planner_test/tests/testdata/output/watermark.yaml index e4ef42b121528..d57d41fa76bc3 100644 --- a/src/frontend/planner_test/tests/testdata/output/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/watermark.yaml @@ -79,11 +79,12 @@ select t1.ts as t1_ts, t2.ts as ts2, t1.v1 as t1_v1, t1.v2 as t1_v2, t2.v1 as t2_v1, t2.v2 as t2_v2 from t1, t2 where t1.ts = t2.ts; stream_plan: |- StreamMaterialize { columns: [t1_ts, ts2, t1_v1, t1_v2, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_ts], pk_columns: [t1._row_id, t2._row_id, t1_ts], pk_conflict: NoCheck, watermark_columns: [t1_ts, ts2] } - └─StreamHashJoin [window, append_only] { type: Inner, predicate: t1.ts = t2.ts, output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t2.ts, t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.ts) } - │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.ts) } - └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.ts, t1._row_id, t2._row_id) } + └─StreamHashJoin [window, append_only] { type: Inner, predicate: t1.ts = t2.ts, output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t2.ts, t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.ts) } + │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.ts) } + └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: left semi window join sql: | create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; @@ -91,11 +92,12 @@ select t1.ts as t1_ts, t1.v1 as t1_v1, t1.v2 as t1_v2 from t1 where exists (select * from t2 where t1.ts = t2.ts); stream_plan: |- StreamMaterialize { columns: [t1_ts, t1_v1, t1_v2, t1._row_id(hidden)], stream_key: [t1._row_id, t1_ts], pk_columns: [t1._row_id, t1_ts], pk_conflict: NoCheck, watermark_columns: [t1_ts] } - └─StreamHashJoin [window] { type: LeftSemi, predicate: t1.ts = t2.ts, output_watermarks: [t1.ts], output: all } - ├─StreamExchange { dist: HashShard(t1.ts) } - │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.ts) } - └─StreamTableScan { table: t2, columns: [t2.ts, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.ts, t1._row_id) } + └─StreamHashJoin [window] { type: LeftSemi, predicate: t1.ts = t2.ts, output_watermarks: [t1.ts], output: all } + ├─StreamExchange { dist: HashShard(t1.ts) } + │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.ts) } + └─StreamTableScan { table: t2, columns: [t2.ts, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: interval join(left outer join) sql: | create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; @@ -108,13 +110,14 @@ └─LogicalScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id] } stream_plan: |- StreamMaterialize { columns: [t1_ts, t1_v1, t1_v2, t2_ts, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1], pk_columns: [t1._row_id, t2._row_id, t1_v1], pk_conflict: NoCheck, watermark_columns: [t1_ts, t2_ts] } - └─StreamHashJoin [interval] { type: LeftOuter, predicate: t1.v1 = t2.v1 AND (t1.ts >= $expr2) AND ($expr1 <= t2.ts), conditions_to_clean_left_state_table: (t1.ts >= $expr2), conditions_to_clean_right_state_table: ($expr1 <= t2.ts), output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, AddWithTimeZone(t1.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr1, t1._row_id], output_watermarks: [t1.ts, $expr1] } - │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.v1) } - └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, AddWithTimeZone(t2.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr2, t2._row_id], output_watermarks: [t2.ts, $expr2] } - └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) } + └─StreamHashJoin [interval] { type: LeftOuter, predicate: t1.v1 = t2.v1 AND (t1.ts >= $expr2) AND ($expr1 <= t2.ts), conditions_to_clean_left_state_table: (t1.ts >= $expr2), conditions_to_clean_right_state_table: ($expr1 <= t2.ts), output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, AddWithTimeZone(t1.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr1, t1._row_id], output_watermarks: [t1.ts, $expr1] } + │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v1) } + └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, AddWithTimeZone(t2.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr2, t2._row_id], output_watermarks: [t2.ts, $expr2] } + └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: interval join (inner join) sql: | create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; @@ -127,13 +130,14 @@ └─LogicalScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id] } stream_plan: |- StreamMaterialize { columns: [t1_ts, t1_v1, t1_v2, t2_ts, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1], pk_columns: [t1._row_id, t2._row_id, t1_v1], pk_conflict: NoCheck, watermark_columns: [t1_ts, t2_ts] } - └─StreamHashJoin [interval, append_only] { type: Inner, predicate: t1.v1 = t2.v1 AND (t1.ts >= $expr2) AND ($expr1 <= t2.ts), conditions_to_clean_left_state_table: (t1.ts >= $expr2), conditions_to_clean_right_state_table: ($expr1 <= t2.ts), output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2, t1._row_id, t2._row_id] } - ├─StreamExchange { dist: HashShard(t1.v1) } - │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, AddWithTimeZone(t1.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr1, t1._row_id], output_watermarks: [t1.ts, $expr1] } - │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.v1) } - └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, AddWithTimeZone(t2.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr2, t2._row_id], output_watermarks: [t2.ts, $expr2] } - └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamExchange { dist: HashShard(t1.v1, t1._row_id, t2._row_id) } + └─StreamHashJoin [interval, append_only] { type: Inner, predicate: t1.v1 = t2.v1 AND (t1.ts >= $expr2) AND ($expr1 <= t2.ts), conditions_to_clean_left_state_table: (t1.ts >= $expr2), conditions_to_clean_right_state_table: ($expr1 <= t2.ts), output_watermarks: [t1.ts, t2.ts], output: [t1.ts, t1.v1, t1.v2, t2.ts, t2.v1, t2.v2, t1._row_id, t2._row_id] } + ├─StreamExchange { dist: HashShard(t1.v1) } + │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, AddWithTimeZone(t1.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr1, t1._row_id], output_watermarks: [t1.ts, $expr1] } + │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.v1) } + └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, AddWithTimeZone(t2.ts, '00:00:01':Interval, 'UTC':Varchar) as $expr2, t2._row_id], output_watermarks: [t2.ts, $expr2] } + └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: union all sql: | create table t1 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; diff --git a/src/frontend/planner_test/tests/testdata/output/window_join.yaml b/src/frontend/planner_test/tests/testdata/output/window_join.yaml index 4113a6021e866..17c5e76f6e806 100644 --- a/src/frontend/planner_test/tests/testdata/output/window_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/window_join.yaml @@ -12,15 +12,16 @@ select * from t1, t2 where ts1 = ts2 and a1 = a2; stream_plan: |- StreamMaterialize { columns: [ts1, a1, b1, ts2, a2, b2, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, ts1, a1], pk_columns: [_row_id, _row_id#1, ts1, a1], pk_conflict: NoCheck, watermark_columns: [ts1, ts2] } - └─StreamHashJoin [window, append_only] { type: Inner, predicate: ts1 = ts2 AND a1 = a2, output_watermarks: [ts1, ts2], output: [ts1, a1, b1, ts2, a2, b2, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(ts1, a1) } - │ └─StreamRowIdGen { row_id_index: 3 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts1, expr: (ts1 - '00:00:01':Interval) }], output_watermarks: [ts1] } - │ └─StreamSource { source: t1, columns: [ts1, a1, b1, _row_id] } - └─StreamExchange { dist: HashShard(ts2, a2) } - └─StreamRowIdGen { row_id_index: 3 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts2, expr: (ts2 - '00:00:01':Interval) }], output_watermarks: [ts2] } - └─StreamSource { source: t2, columns: [ts2, a2, b2, _row_id] } + └─StreamExchange { dist: HashShard(ts1, a1, _row_id, _row_id) } + └─StreamHashJoin [window, append_only] { type: Inner, predicate: ts1 = ts2 AND a1 = a2, output_watermarks: [ts1, ts2], output: [ts1, a1, b1, ts2, a2, b2, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(ts1, a1) } + │ └─StreamRowIdGen { row_id_index: 3 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts1, expr: (ts1 - '00:00:01':Interval) }], output_watermarks: [ts1] } + │ └─StreamSource { source: t1, columns: [ts1, a1, b1, _row_id] } + └─StreamExchange { dist: HashShard(ts2, a2) } + └─StreamRowIdGen { row_id_index: 3 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts2, expr: (ts2 - '00:00:01':Interval) }], output_watermarks: [ts2] } + └─StreamSource { source: t2, columns: [ts2, a2, b2, _row_id] } - name: Window join expression reorder sql: | create source t1 (ts1 timestamp with time zone, a1 int, b1 int, watermark for ts1 as ts1 - INTERVAL '1' SECOND) with ( @@ -34,12 +35,13 @@ select * from t1, t2 where a1 = a2 and ts1 = ts2; stream_plan: |- StreamMaterialize { columns: [ts1, a1, b1, ts2, a2, b2, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, a1, ts1], pk_columns: [_row_id, _row_id#1, a1, ts1], pk_conflict: NoCheck, watermark_columns: [ts1, ts2] } - └─StreamHashJoin [window, append_only] { type: Inner, predicate: ts1 = ts2 AND a1 = a2, output_watermarks: [ts1, ts2], output: [ts1, a1, b1, ts2, a2, b2, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(ts1, a1) } - │ └─StreamRowIdGen { row_id_index: 3 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts1, expr: (ts1 - '00:00:01':Interval) }], output_watermarks: [ts1] } - │ └─StreamSource { source: t1, columns: [ts1, a1, b1, _row_id] } - └─StreamExchange { dist: HashShard(ts2, a2) } - └─StreamRowIdGen { row_id_index: 3 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts2, expr: (ts2 - '00:00:01':Interval) }], output_watermarks: [ts2] } - └─StreamSource { source: t2, columns: [ts2, a2, b2, _row_id] } + └─StreamExchange { dist: HashShard(ts1, a1, _row_id, _row_id) } + └─StreamHashJoin [window, append_only] { type: Inner, predicate: ts1 = ts2 AND a1 = a2, output_watermarks: [ts1, ts2], output: [ts1, a1, b1, ts2, a2, b2, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(ts1, a1) } + │ └─StreamRowIdGen { row_id_index: 3 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts1, expr: (ts1 - '00:00:01':Interval) }], output_watermarks: [ts1] } + │ └─StreamSource { source: t1, columns: [ts1, a1, b1, _row_id] } + └─StreamExchange { dist: HashShard(ts2, a2) } + └─StreamRowIdGen { row_id_index: 3 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: ts2, expr: (ts2 - '00:00:01':Interval) }], output_watermarks: [ts2] } + └─StreamSource { source: t2, columns: [ts2, a2, b2, _row_id] } diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index d8972436d5c78..9c87f1a34abbd 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -151,7 +151,22 @@ impl StreamMaterialize { TableType::MaterializedView => { assert_matches!(user_distributed_by, RequiredDist::Any); // ensure the same pk will not shuffle to different node - RequiredDist::shard_by_key(input.schema().len(), input.expect_stream_key()) + let required_dist = + RequiredDist::shard_by_key(input.schema().len(), input.expect_stream_key()); + + // If the input is a stream join, enforce the stream key as the materialized + // view distribution key to avoid slow backfilling caused by + // data skew of the dimension table join key. + // See for more information. + let is_stream_join = matches!(input.as_stream_hash_join(), Some(_join)) + || matches!(input.as_stream_temporal_join(), Some(_join)) + || matches!(input.as_stream_delta_join(), Some(_join)); + + if is_stream_join { + return Ok(required_dist.enforce(input, &Order::any())); + } + + required_dist } TableType::Index => { assert_matches!( diff --git a/src/frontend/src/optimizer/property/distribution.rs b/src/frontend/src/optimizer/property/distribution.rs index b6e7715dd155f..2df1d7ae00bc3 100644 --- a/src/frontend/src/optimizer/property/distribution.rs +++ b/src/frontend/src/optimizer/property/distribution.rs @@ -331,7 +331,7 @@ impl RequiredDist { } } - fn enforce(&self, plan: PlanRef, required_order: &Order) -> PlanRef { + pub fn enforce(&self, plan: PlanRef, required_order: &Order) -> PlanRef { let dist = self.to_dist(); match plan.convention() { Convention::Batch => BatchExchange::new(plan, required_order.clone(), dist).into(), diff --git a/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs b/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs index c05e52c927424..776692b2fab90 100644 --- a/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs +++ b/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs @@ -25,7 +25,7 @@ const ROOT_TABLE_CREATE: &str = "create table t1 (v1 int);"; const MV1: &str = "create materialized view m1 as select * from t1 where v1 > 5;"; const MV2: &str = "create materialized view m2 as select * from t1 where v1 > 10;"; const MV3: &str = "create materialized view m3 as select * from m2 where v1 < 15;"; -const MV4: &str = "create materialized view m4 as select m1.v1 as m1v, m3.v1 as m3v from m1 join m3 on m1.v1 = m3.v1;"; +const MV4: &str = "create materialized view m4 as select m1.v1 as m1v, m3.v1 as m3v from m1 join m3 on m1.v1 = m3.v1 limit 100;"; const MV5: &str = "create materialized view m5 as select * from m4;"; #[tokio::test] @@ -40,6 +40,7 @@ async fn test_simple_cascade_materialized_view() -> Result<()> { .locate_one_fragment([ identity_contains("materialize"), no_identity_contains("chain"), + no_identity_contains("topn"), no_identity_contains("hashjoin"), ]) .await?; @@ -129,6 +130,7 @@ async fn test_diamond_cascade_materialized_view() -> Result<()> { .locate_one_fragment([ identity_contains("materialize"), no_identity_contains("chain"), + no_identity_contains("topn"), no_identity_contains("hashjoin"), ]) .await?; diff --git a/src/tests/simulation/tests/integration_tests/scale/plan.rs b/src/tests/simulation/tests/integration_tests/scale/plan.rs index c7244dc826b42..8b62a58998a3f 100644 --- a/src/tests/simulation/tests/integration_tests/scale/plan.rs +++ b/src/tests/simulation/tests/integration_tests/scale/plan.rs @@ -39,10 +39,7 @@ async fn test_resize_normal() -> Result<()> { .await?; let join_fragment = cluster - .locate_one_fragment([ - identity_contains("hashJoin"), - identity_contains("materialize"), - ]) + .locate_one_fragment([identity_contains("hashJoin")]) .await?; let join_fragment_id = join_fragment.inner.fragment_id; @@ -270,7 +267,7 @@ async fn test_resize_no_shuffle() -> Result<()> { session .run( "create materialized view mv7 as select mv1.v as mv1v, mv5.v as mv5v from mv1 -join mv5 on mv1.v = mv5.v;", +join mv5 on mv1.v = mv5.v limit 1;", ) .await?; @@ -316,6 +313,7 @@ join mv5 on mv1.v = mv5.v;", let top_materialize_fragment = cluster .locate_one_fragment([ identity_contains("materialize"), + no_identity_contains("topn"), no_identity_contains("chain"), no_identity_contains("hashJoin"), ]) From 210ae713af70b83fa5b051cfda59ead43e90db73 Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Tue, 24 Oct 2023 23:30:57 +0800 Subject: [PATCH 10/48] feat(sink): use 'create sink ... format ... encode' to create redis sink (#13003) --- integration_tests/redis-sink/create_sink.sql | 10 +- proto/plan_common.proto | 1 + src/connector/src/sink/catalog/mod.rs | 3 + src/connector/src/sink/encoder/template.rs | 22 +++ src/connector/src/sink/formatter/mod.rs | 156 ++++++++++--------- src/connector/src/sink/redis.rs | 149 +++++++++--------- src/frontend/src/handler/create_sink.rs | 6 + src/sqlparser/src/ast/statement.rs | 13 +- 8 files changed, 199 insertions(+), 161 deletions(-) diff --git a/integration_tests/redis-sink/create_sink.sql b/integration_tests/redis-sink/create_sink.sql index 03bfc2d0b0df1..2ba9ba67feb39 100644 --- a/integration_tests/redis-sink/create_sink.sql +++ b/integration_tests/redis-sink/create_sink.sql @@ -3,19 +3,13 @@ FROM bhv_mv WITH ( primary_key = 'user_id', connector = 'redis', - type = 'append-only', - force_append_only='true', redis.url= 'redis://127.0.0.1:6379/', -); +)FORMAT PLAIN ENCODE JSON(force_append_only='true'); CREATE SINK bhv_redis_sink_2 FROM bhv_mv WITH ( primary_key = 'user_id', connector = 'redis', - type = 'append-only', - force_append_only='true', redis.url= 'redis://127.0.0.1:6379/', - redis.keyformat='user_id:{user_id}', - redis.valueformat='username:{username},event_timestamp{event_timestamp}' -); \ No newline at end of file +)FORMAT PLAIN ENCODE TEMPLATE(force_append_only='true', key_format = 'UserID:{user_id}', value_format = 'TargetID:{target_id},EventTimestamp{event_timestamp}'); \ No newline at end of file diff --git a/proto/plan_common.proto b/proto/plan_common.proto index a88242a572693..d4c7a2e04f138 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -106,6 +106,7 @@ enum EncodeType { ENCODE_TYPE_PROTOBUF = 4; ENCODE_TYPE_JSON = 5; ENCODE_TYPE_BYTES = 6; + ENCODE_TYPE_TEMPLATE = 7; } enum RowFormatType { diff --git a/src/connector/src/sink/catalog/mod.rs b/src/connector/src/sink/catalog/mod.rs index c18dd7d10a92c..ca3a09e7f2eda 100644 --- a/src/connector/src/sink/catalog/mod.rs +++ b/src/connector/src/sink/catalog/mod.rs @@ -132,6 +132,7 @@ pub enum SinkEncode { Json, Protobuf, Avro, + Template, } impl SinkFormatDesc { @@ -177,6 +178,7 @@ impl SinkFormatDesc { SinkEncode::Json => E::Json, SinkEncode::Protobuf => E::Protobuf, SinkEncode::Avro => E::Avro, + SinkEncode::Template => E::Template, }; let options = self .options @@ -212,6 +214,7 @@ impl TryFrom for SinkFormatDesc { let encode = match value.encode() { E::Json => SinkEncode::Json, E::Protobuf => SinkEncode::Protobuf, + E::Template => SinkEncode::Template, E::Avro => SinkEncode::Avro, e @ (E::Unspecified | E::Native | E::Csv | E::Bytes) => { return Err(SinkError::Config(anyhow!( diff --git a/src/connector/src/sink/encoder/template.rs b/src/connector/src/sink/encoder/template.rs index 85f085989b6c4..97d8271f9e83a 100644 --- a/src/connector/src/sink/encoder/template.rs +++ b/src/connector/src/sink/encoder/template.rs @@ -12,11 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashSet; + +use regex::Regex; use risingwave_common::catalog::Schema; use risingwave_common::row::Row; use risingwave_common::types::ToText; use super::{Result, RowEncoder}; +use crate::sink::SinkError; /// Encode a row according to a specified string template `user_id:{user_id}` pub struct TemplateEncoder { @@ -34,6 +38,24 @@ impl TemplateEncoder { template, } } + + pub fn check_string_format(format: &str, set: &HashSet) -> Result<()> { + // We will check if the string inside {} corresponds to a column name in rw. + // In other words, the content within {} should exclusively consist of column names from rw, + // which means '{{column_name}}' or '{{column_name1},{column_name2}}' would be incorrect. + let re = Regex::new(r"\{([^}]*)\}").unwrap(); + if !re.is_match(format) { + return Err(SinkError::Redis( + "Can't find {} in key_format or value_format".to_string(), + )); + } + for capture in re.captures_iter(format) { + if let Some(inner_content) = capture.get(1) && !set.contains(inner_content.as_str()){ + return Err(SinkError::Redis(format!("Can't find field({:?}) in key_format or value_format",inner_content.as_str()))) + } + } + Ok(()) + } } impl RowEncoder for TemplateEncoder { diff --git a/src/connector/src/sink/formatter/mod.rs b/src/connector/src/sink/formatter/mod.rs index a7463f7e3b306..17cb708292890 100644 --- a/src/connector/src/sink/formatter/mod.rs +++ b/src/connector/src/sink/formatter/mod.rs @@ -29,6 +29,7 @@ pub use upsert::UpsertFormatter; use super::catalog::{SinkEncode, SinkFormat, SinkFormatDesc}; use super::encoder::template::TemplateEncoder; use super::encoder::KafkaConnectParams; +use super::redis::{KEY_FORMAT, VALUE_FORMAT}; use crate::sink::encoder::{JsonEncoder, ProtoEncoder, TimestampHandlingMode}; /// Transforms a `StreamChunk` into a sequence of key-value pairs according a specific format, @@ -92,7 +93,7 @@ impl SinkFormatterImpl { let key_encoder = (!pk_indices.is_empty()).then(|| { JsonEncoder::new( schema.clone(), - Some(pk_indices), + Some(pk_indices.clone()), TimestampHandlingMode::Milli, ) }); @@ -115,6 +116,28 @@ impl SinkFormatterImpl { Ok(SinkFormatterImpl::AppendOnlyProto(formatter)) } SinkEncode::Avro => err_unsupported(), + SinkEncode::Template => { + let key_format = format_desc.options.get(KEY_FORMAT).ok_or_else(|| { + SinkError::Config(anyhow!( + "Cannot find 'key_format',please set it or use JSON" + )) + })?; + let value_format = + format_desc.options.get(VALUE_FORMAT).ok_or_else(|| { + SinkError::Config(anyhow!( + "Cannot find 'redis_value_format',please set it or use JSON" + )) + })?; + let key_encoder = TemplateEncoder::new( + schema.clone(), + Some(pk_indices), + key_format.clone(), + ); + let val_encoder = TemplateEncoder::new(schema, None, value_format.clone()); + Ok(SinkFormatterImpl::AppendOnlyTemplate( + AppendOnlyFormatter::new(Some(key_encoder), val_encoder), + )) + } } } SinkFormat::Debezium => { @@ -131,85 +154,66 @@ impl SinkFormatterImpl { ))) } SinkFormat::Upsert => { - if format_desc.encode != SinkEncode::Json { - return err_unsupported(); - } + match format_desc.encode { + SinkEncode::Json => { + let mut key_encoder = JsonEncoder::new( + schema.clone(), + Some(pk_indices), + TimestampHandlingMode::Milli, + ); + let mut val_encoder = + JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); - let mut key_encoder = JsonEncoder::new( - schema.clone(), - Some(pk_indices), - TimestampHandlingMode::Milli, - ); - let mut val_encoder = JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); - - if let Some(s) = format_desc.options.get("schemas.enable") { - match s.to_lowercase().parse::() { - Ok(true) => { - let kafka_connect = KafkaConnectParams { - schema_name: format!("{}.{}", db_name, sink_from_name), - }; - key_encoder = key_encoder.with_kafka_connect(kafka_connect.clone()); - val_encoder = val_encoder.with_kafka_connect(kafka_connect); - } - Ok(false) => {} - _ => { - return Err(SinkError::Config(anyhow!( - "schemas.enable is expected to be `true` or `false`, got {}", - s - ))); - } + if let Some(s) = format_desc.options.get("schemas.enable") { + match s.to_lowercase().parse::() { + Ok(true) => { + let kafka_connect = KafkaConnectParams { + schema_name: format!("{}.{}", db_name, sink_from_name), + }; + key_encoder = + key_encoder.with_kafka_connect(kafka_connect.clone()); + val_encoder = val_encoder.with_kafka_connect(kafka_connect); + } + Ok(false) => {} + _ => { + return Err(SinkError::Config(anyhow!( + "schemas.enable is expected to be `true` or `false`, got {}", + s + ))); + } + } + }; + + // Initialize the upsert_stream + let formatter = UpsertFormatter::new(key_encoder, val_encoder); + Ok(SinkFormatterImpl::UpsertJson(formatter)) } - }; - - // Initialize the upsert_stream - let formatter = UpsertFormatter::new(key_encoder, val_encoder); - Ok(SinkFormatterImpl::UpsertJson(formatter)) - } - } - } - - pub fn new_with_redis( - schema: Schema, - pk_indices: Vec, - is_append_only: bool, - key_format: Option, - value_format: Option, - ) -> Result { - match (key_format, value_format) { - (Some(k), Some(v)) => { - let key_encoder = TemplateEncoder::new( - schema.clone(), - Some(pk_indices), - k, - ); - let val_encoder = - TemplateEncoder::new(schema, None, v); - if is_append_only { - Ok(SinkFormatterImpl::AppendOnlyTemplate(AppendOnlyFormatter::new(Some(key_encoder), val_encoder))) - } else { - Ok(SinkFormatterImpl::UpsertTemplate(UpsertFormatter::new(key_encoder, val_encoder))) - } - } - (None, None) => { - let key_encoder = JsonEncoder::new( - schema.clone(), - Some(pk_indices), - TimestampHandlingMode::Milli, - ); - let val_encoder = JsonEncoder::new( - schema, - None, - TimestampHandlingMode::Milli, - ); - if is_append_only { - Ok(SinkFormatterImpl::AppendOnlyJson(AppendOnlyFormatter::new(Some(key_encoder), val_encoder))) - } else { - Ok(SinkFormatterImpl::UpsertJson(UpsertFormatter::new(key_encoder, val_encoder))) + SinkEncode::Template => { + let key_format = format_desc.options.get(KEY_FORMAT).ok_or_else(|| { + SinkError::Config(anyhow!( + "Cannot find 'key_format',please set it or use JSON" + )) + })?; + let value_format = + format_desc.options.get(VALUE_FORMAT).ok_or_else(|| { + SinkError::Config(anyhow!( + "Cannot find 'redis_value_format',please set it or use JSON" + )) + })?; + let key_encoder = TemplateEncoder::new( + schema.clone(), + Some(pk_indices), + key_format.clone(), + ); + let val_encoder = TemplateEncoder::new(schema, None, value_format.clone()); + Ok(SinkFormatterImpl::UpsertTemplate(UpsertFormatter::new( + key_encoder, + val_encoder, + ))) + } + _ => err_unsupported(), } } - _ => { - Err(SinkError::Encode("Please provide template formats for both key and value, or choose the JSON format.".to_string())) - } } } } diff --git a/src/connector/src/sink/redis.rs b/src/connector/src/sink/redis.rs index cc8ff74d0c9c5..6120075a049df 100644 --- a/src/connector/src/sink/redis.rs +++ b/src/connector/src/sink/redis.rs @@ -18,29 +18,27 @@ use anyhow::anyhow; use async_trait::async_trait; use redis::aio::Connection; use redis::{Client as RedisClient, Pipeline}; -use regex::Regex; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; use serde_derive::{Deserialize, Serialize}; use serde_with::serde_as; +use super::catalog::SinkFormatDesc; +use super::encoder::template::TemplateEncoder; use super::formatter::SinkFormatterImpl; use super::writer::FormattedSink; -use super::{SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT}; +use super::{SinkError, SinkParam}; use crate::dispatch_sink_formatter_impl; use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; pub const REDIS_SINK: &str = "redis"; - +pub const KEY_FORMAT: &str = "key_format"; +pub const VALUE_FORMAT: &str = "value_format"; #[derive(Deserialize, Serialize, Debug, Clone)] pub struct RedisCommon { #[serde(rename = "redis.url")] pub url: String, - #[serde(rename = "redis.keyformat")] - pub key_format: Option, - #[serde(rename = "redis.valueformat")] - pub value_format: Option, } impl RedisCommon { @@ -54,23 +52,13 @@ impl RedisCommon { pub struct RedisConfig { #[serde(flatten)] pub common: RedisCommon, - - pub r#type: String, // accept "append-only" or "upsert" } impl RedisConfig { pub fn from_hashmap(properties: HashMap) -> Result { let config = serde_json::from_value::(serde_json::to_value(properties).unwrap()) - .map_err(|e| SinkError::Config(anyhow!(e)))?; - if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { - return Err(SinkError::Config(anyhow!( - "`{}` must be {}, or {}", - SINK_TYPE_OPTION, - SINK_TYPE_APPEND_ONLY, - SINK_TYPE_UPSERT - ))); - } + .map_err(|e| SinkError::Config(anyhow!("{:?}", e)))?; Ok(config) } } @@ -79,28 +67,10 @@ impl RedisConfig { pub struct RedisSink { config: RedisConfig, schema: Schema, - is_append_only: bool, pk_indices: Vec, -} - -fn check_string_format(format: &Option, set: &HashSet) -> Result<()> { - if let Some(format) = format { - // We will check if the string inside {} corresponds to a column name in rw. - // In other words, the content within {} should exclusively consist of column names from rw, - // which means '{{column_name}}' or '{{column_name1},{column_name2}}' would be incorrect. - let re = Regex::new(r"\{([^}]*)\}").unwrap(); - if !re.is_match(format) { - return Err(SinkError::Redis( - "Can't find {} in key_format or value_format".to_string(), - )); - } - for capture in re.captures_iter(format) { - if let Some(inner_content) = capture.get(1) && !set.contains(inner_content.as_str()){ - return Err(SinkError::Redis(format!("Can't find field({:?}) in key_format or value_format",inner_content.as_str()))) - } - } - } - Ok(()) + format_desc: SinkFormatDesc, + db_name: String, + sink_from_name: String, } #[async_trait] @@ -117,8 +87,12 @@ impl TryFrom for RedisSink { Ok(Self { config, schema: param.schema(), - is_append_only: param.sink_type.is_append_only(), pk_indices: param.downstream_pk, + format_desc: param + .format_desc + .ok_or_else(|| SinkError::Config(anyhow!("missing FORMAT ... ENCODE ...")))?, + db_name: param.db_name, + sink_from_name: param.sink_from_name, }) } } @@ -134,7 +108,9 @@ impl Sink for RedisSink { self.config.clone(), self.schema.clone(), self.pk_indices.clone(), - self.is_append_only, + &self.format_desc, + self.db_name.clone(), + self.sink_from_name.clone(), ) .await? .into_log_sinker(writer_param.sink_metrics)) @@ -157,8 +133,23 @@ impl Sink for RedisSink { .filter(|(k, _)| self.pk_indices.contains(k)) .map(|(_, v)| v.name.clone()) .collect(); - check_string_format(&self.config.common.key_format, &pk_set)?; - check_string_format(&self.config.common.value_format, &all_set)?; + if matches!( + self.format_desc.encode, + super::catalog::SinkEncode::Template + ) { + let key_format = self.format_desc.options.get(KEY_FORMAT).ok_or_else(|| { + SinkError::Config(anyhow!( + "Cannot find 'key_format',please set it or use JSON" + )) + })?; + let value_format = self.format_desc.options.get(VALUE_FORMAT).ok_or_else(|| { + SinkError::Config(anyhow!( + "Cannot find 'value_format',please set it or use JSON" + )) + })?; + TemplateEncoder::check_string_format(key_format, &pk_set)?; + TemplateEncoder::check_string_format(value_format, &all_set)?; + } Ok(()) } } @@ -166,7 +157,6 @@ impl Sink for RedisSink { pub struct RedisSinkWriter { epoch: u64, schema: Schema, - is_append_only: bool, pk_indices: Vec, formatter: SinkFormatterImpl, payload_writer: RedisSinkPayloadWriter, @@ -220,21 +210,23 @@ impl RedisSinkWriter { config: RedisConfig, schema: Schema, pk_indices: Vec, - is_append_only: bool, + format_desc: &SinkFormatDesc, + db_name: String, + sink_from_name: String, ) -> Result { let payload_writer = RedisSinkPayloadWriter::new(config.clone()).await?; - let formatter = SinkFormatterImpl::new_with_redis( + let formatter = SinkFormatterImpl::new( + format_desc, schema.clone(), pk_indices.clone(), - is_append_only, - config.common.key_format, - config.common.value_format, - )?; + db_name, + sink_from_name, + ) + .await?; Ok(Self { schema, pk_indices, - is_append_only, epoch: 0, formatter, payload_writer, @@ -242,24 +234,22 @@ impl RedisSinkWriter { } #[cfg(test)] - pub fn mock( + pub async fn mock( schema: Schema, pk_indices: Vec, - is_append_only: bool, - key_format: Option, - value_format: Option, + format_desc: &SinkFormatDesc, ) -> Result { - let formatter = SinkFormatterImpl::new_with_redis( + let formatter = SinkFormatterImpl::new( + format_desc, schema.clone(), pk_indices.clone(), - is_append_only, - key_format, - value_format, - )?; + "d1".to_string(), + "t1".to_string(), + ) + .await?; Ok(Self { schema, pk_indices, - is_append_only, epoch: 0, formatter, payload_writer: RedisSinkPayloadWriter::mock(), @@ -290,6 +280,8 @@ impl SinkWriter for RedisSinkWriter { #[cfg(test)] mod test { + use std::collections::BTreeMap; + use rdkafka::message::FromBytes; use risingwave_common::array::{Array, I32Array, Op, StreamChunk, Utf8Array}; use risingwave_common::catalog::{Field, Schema}; @@ -297,6 +289,7 @@ mod test { use risingwave_common::util::iter_util::ZipEqDebug; use super::*; + use crate::sink::catalog::{SinkEncode, SinkFormat}; #[tokio::test] async fn test_write() { @@ -315,8 +308,15 @@ mod test { }, ]); - let mut redis_sink_writer = - RedisSinkWriter::mock(schema, vec![0], true, None, None).unwrap(); + let format_desc = SinkFormatDesc { + format: SinkFormat::AppendOnly, + encode: SinkEncode::Json, + options: BTreeMap::default(), + }; + + let mut redis_sink_writer = RedisSinkWriter::mock(schema, vec![0], &format_desc) + .await + .unwrap(); let chunk_a = StreamChunk::new( vec![Op::Insert, Op::Insert, Op::Insert], @@ -367,14 +367,21 @@ mod test { }, ]); - let mut redis_sink_writer = RedisSinkWriter::mock( - schema, - vec![0], - true, - Some("key-{id}".to_string()), - Some("values:{id:{id},name:{name}}".to_string()), - ) - .unwrap(); + let mut btree_map = BTreeMap::default(); + btree_map.insert(KEY_FORMAT.to_string(), "key-{id}".to_string()); + btree_map.insert( + VALUE_FORMAT.to_string(), + "values:{id:{id},name:{name}}".to_string(), + ); + let format_desc = SinkFormatDesc { + format: SinkFormat::AppendOnly, + encode: SinkEncode::Template, + options: btree_map, + }; + + let mut redis_sink_writer = RedisSinkWriter::mock(schema, vec![0], &format_desc) + .await + .unwrap(); let chunk_a = StreamChunk::new( vec![Op::Insert, Op::Insert, Op::Insert], diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 32279dd4e70eb..ddb1d697b856d 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -244,6 +244,7 @@ fn bind_sink_format_desc(value: SinkSchema) -> Result { E::Json => SinkEncode::Json, E::Protobuf => SinkEncode::Protobuf, E::Avro => SinkEncode::Avro, + E::Template => SinkEncode::Template, e @ (E::Native | E::Csv | E::Bytes) => { return Err(ErrorCode::BindError(format!("sink encode unsupported: {e}")).into()) } @@ -262,6 +263,7 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock vec![Encode::Json], Format::Debezium => vec![Encode::Json], ), + RedisSink::SINK_NAME => hashmap!( + Format::Plain => vec![Encode::Json,Encode::Template], + Format::Upsert => vec![Encode::Json,Encode::Template], + ), )) }); pub fn validate_compatibility(connector: &str, format_desc: &SinkSchema) -> Result<()> { diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index 76de970a919a9..58fb2d50c6287 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -294,6 +294,7 @@ pub enum Encode { Json, // Keyword::JSON Bytes, // Keyword::BYTES Native, + Template, } // TODO: unify with `from_keyword` @@ -309,6 +310,7 @@ impl fmt::Display for Encode { Encode::Json => "JSON", Encode::Bytes => "BYTES", Encode::Native => "NATIVE", + Encode::Template => "TEMPLATE", } ) } @@ -322,13 +324,12 @@ impl Encode { "CSV" => Encode::Csv, "PROTOBUF" => Encode::Protobuf, "JSON" => Encode::Json, + "TEMPLATE" => Encode::Template, "NATIVE" => Encode::Native, // used internally for schema change - _ => { - return Err(ParserError::ParserError( - "expected AVRO | BYTES | CSV | PROTOBUF | JSON | NATIVE after Encode" - .to_string(), - )) - } + _ => return Err(ParserError::ParserError( + "expected AVRO | BYTES | CSV | PROTOBUF | JSON | NATIVE | TEMPLATE after Encode" + .to_string(), + )), }) } } From 543bf2d25d10e57be3671bea548bd0f450a59471 Mon Sep 17 00:00:00 2001 From: TennyZhuang Date: Wed, 25 Oct 2023 04:21:18 +0800 Subject: [PATCH 11/48] build(toolchain): bump to nightly-2023-10-21 (#12992) Signed-off-by: TennyZhuang Co-authored-by: TennyZhuang --- Cargo.lock | 69 +++++++++++-------- Cargo.toml | 11 ++- ci/build-ci-image.sh | 2 +- ci/docker-compose.yml | 10 +-- ci/rust-toolchain | 2 +- src/batch/src/lib.rs | 6 +- src/batch/src/rpc/service/task_service.rs | 6 +- src/cmd/src/bin/compactor.rs | 2 +- src/cmd/src/bin/compute_node.rs | 2 +- src/cmd/src/bin/ctl.rs | 2 +- src/cmd/src/bin/frontend_node.rs | 2 +- src/cmd/src/bin/meta_node.rs | 2 +- src/cmd/src/lib.rs | 2 +- src/cmd_all/src/bin/risingwave.rs | 4 +- src/common/proc_macro/src/config.rs | 2 +- src/common/proc_macro/src/lib.rs | 4 +- src/common/src/lib.rs | 4 +- src/common/src/types/ordered.rs | 2 +- src/compute/src/lib.rs | 4 +- .../src/rpc/service/exchange_service.rs | 2 +- .../src/rpc/service/monitor_service.rs | 10 +-- src/compute/src/rpc/service/stream_service.rs | 16 ++--- src/compute/tests/cdc_tests.rs | 2 +- src/compute/tests/integration_tests.rs | 2 +- src/connector/Cargo.toml | 2 +- src/connector/src/lib.rs | 6 +- .../src/sink/formatter/append_only.rs | 2 +- .../src/sink/formatter/debezium_json.rs | 2 +- src/connector/src/sink/formatter/upsert.rs | 2 +- src/expr/core/src/lib.rs | 2 +- src/expr/impl/src/lib.rs | 2 +- src/frontend/src/binder/mod.rs | 14 ++-- src/frontend/src/lib.rs | 3 +- src/meta/node/src/lib.rs | 2 +- src/meta/service/src/ddl_service.rs | 2 +- src/meta/service/src/heartbeat_service.rs | 2 +- src/meta/service/src/lib.rs | 2 +- src/meta/service/src/meta_member_service.rs | 2 +- src/meta/service/src/notification_service.rs | 2 +- src/meta/service/src/scale_service.rs | 6 +- src/meta/service/src/stream_service.rs | 14 ++-- src/meta/service/src/user_service.rs | 10 +-- src/meta/src/hummock/manager/mod.rs | 2 +- src/meta/src/lib.rs | 3 +- src/prost/helpers/src/lib.rs | 6 +- src/rpc_client/src/lib.rs | 2 +- src/source/src/lib.rs | 2 +- src/storage/Cargo.toml | 2 +- src/storage/backup/src/lib.rs | 2 +- src/storage/hummock_test/Cargo.toml | 4 +- .../hummock_test/src/bin/replay/main.rs | 2 +- src/storage/hummock_test/src/lib.rs | 1 - src/storage/hummock_trace/Cargo.toml | 2 +- src/storage/hummock_trace/src/lib.rs | 2 +- .../src/hummock/compactor/compactor_runner.rs | 3 - src/storage/src/hummock/compactor/mod.rs | 4 +- .../src/hummock/event_handler/uploader.rs | 3 - src/storage/src/lib.rs | 6 +- .../src/common/table/state_table_cache.rs | 4 +- src/stream/src/executor/backfill/utils.rs | 3 - src/stream/src/executor/hash_join.rs | 2 +- src/stream/src/lib.rs | 6 +- .../tests/integration_tests/hash_agg.rs | 2 +- .../compaction_test/src/bin/compaction.rs | 4 +- .../compaction_test/src/bin/delete_range.rs | 4 +- src/utils/pgwire/src/lib.rs | 2 - src/workspace-hack/Cargo.toml | 4 +- 67 files changed, 158 insertions(+), 166 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ce71e2bf98950..32cb936bdaae3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1825,13 +1825,13 @@ dependencies = [ [[package]] name = "comfy-table" -version = "7.1.0" +version = "7.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c64043d6c7b7a4c58e39e7efccfdea7b93d885a795d0c054a69dbbf4dd52686" +checksum = "9ab77dbd8adecaf3f0db40581631b995f312a8a5ae3aa9993188bb8f23d83a5b" dependencies = [ - "crossterm 0.27.0", - "strum 0.25.0", - "strum_macros 0.25.2", + "crossterm 0.26.1", + "strum 0.24.1", + "strum_macros 0.24.3", "unicode-width", ] @@ -2140,14 +2140,17 @@ dependencies = [ [[package]] name = "crossterm" -version = "0.27.0" +version = "0.26.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f476fe445d41c9e991fd07515a6f463074b782242ccf4a5b7b1d1012e70824df" +checksum = "a84cda67535339806297f1b331d6dd6320470d2a0fe65381e79ee9e156dd3d13" dependencies = [ - "bitflags 2.4.0", + "bitflags 1.3.2", "crossterm_winapi", "libc", + "mio", "parking_lot 0.12.1", + "signal-hook", + "signal-hook-mio", "winapi", ] @@ -2445,10 +2448,11 @@ dependencies = [ [[package]] name = "deranged" -version = "0.3.8" +version = "0.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2696e8a945f658fd14dc3b87242e6b80cd0f36ff04ea560fa39082368847946" +checksum = "0f32d04922c60427da6f9fef14d042d9edddef64cb9d4ce0d64d0685fbeb1fd3" dependencies = [ + "powerfmt", "serde", ] @@ -2551,8 +2555,7 @@ checksum = "86e3bdc80eee6e16b2b6b0f87fbc98c04bee3455e35174c0de1a125d0688c632" [[package]] name = "dlv-list" version = "0.5.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8aead04dc46b5f263c25721cf25c9e595951d15055f8063f92392fa0d7f64cf4" +source = "git+https://github.com/sgodwincs/dlv-list-rs.git?rev=5bbc5d0#5bbc5d0cc84f257e173d851f8dc1674fb6e46f95" dependencies = [ "const-random", ] @@ -3015,7 +3018,7 @@ dependencies = [ [[package]] name = "foyer" version = "0.1.0" -source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783" +source = "git+https://github.com/MrCroxx/foyer?rev=2261151#2261151107ad362851f5fff9ce4fa56e61911b10" dependencies = [ "foyer-common", "foyer-intrusive", @@ -3026,7 +3029,7 @@ dependencies = [ [[package]] name = "foyer-common" version = "0.1.0" -source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783" +source = "git+https://github.com/MrCroxx/foyer?rev=2261151#2261151107ad362851f5fff9ce4fa56e61911b10" dependencies = [ "bytes", "foyer-workspace-hack", @@ -3041,7 +3044,7 @@ dependencies = [ [[package]] name = "foyer-intrusive" version = "0.1.0" -source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783" +source = "git+https://github.com/MrCroxx/foyer?rev=2261151#2261151107ad362851f5fff9ce4fa56e61911b10" dependencies = [ "bytes", "cmsketch", @@ -3058,7 +3061,7 @@ dependencies = [ [[package]] name = "foyer-storage" version = "0.1.0" -source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783" +source = "git+https://github.com/MrCroxx/foyer?rev=2261151#2261151107ad362851f5fff9ce4fa56e61911b10" dependencies = [ "anyhow", "async-channel", @@ -3087,7 +3090,7 @@ dependencies = [ [[package]] name = "foyer-workspace-hack" version = "0.1.0" -source = "git+https://github.com/mrcroxx/foyer?rev=5d0134b#5d0134b28c0edb03277b01ce08b035ef52c1b783" +source = "git+https://github.com/MrCroxx/foyer?rev=2261151#2261151107ad362851f5fff9ce4fa56e61911b10" dependencies = [ "crossbeam-utils", "either", @@ -3225,9 +3228,9 @@ dependencies = [ [[package]] name = "futures-async-stream" -version = "0.2.7" +version = "0.2.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f529ccdeacfa2446a9577041686cf1abb839b1b3e15fee4c1b1232ab3b7d799f" +checksum = "379790776b0d953337df4ab7ecc51936c66ea112484cad7912907b1d34253ebf" dependencies = [ "futures-async-stream-macro", "futures-core", @@ -3236,13 +3239,13 @@ dependencies = [ [[package]] name = "futures-async-stream-macro" -version = "0.2.7" +version = "0.2.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca2b48ee06dc8d2808ba5ebad075d06c3406085bb19deaac33be64c39113bf80" +checksum = "5df2c13d48c8cb8a3ec093ede6f0f4482f327d7bb781120c5fb483ef0f17e758" dependencies = [ "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.37", ] [[package]] @@ -5492,8 +5495,7 @@ dependencies = [ [[package]] name = "ordered-multimap" version = "0.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ed8acf08e98e744e5384c8bc63ceb0364e68a6854187221c18df61c4797690e" +source = "git+https://github.com/risingwavelabs/ordered-multimap-rs.git?rev=19c743f#19c743f3e3d106c99ba37628f06a2ca6faa2284f" dependencies = [ "dlv-list", "hashbrown 0.13.2", @@ -6023,6 +6025,12 @@ dependencies = [ "serde_json", ] +[[package]] +name = "powerfmt" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "439ee305def115ba05938db6eb1644ff94165c5ab5e9420d1c1bcedbba909391" + [[package]] name = "pprof" version = "0.13.0" @@ -9803,14 +9811,15 @@ dependencies = [ [[package]] name = "time" -version = "0.3.28" +version = "0.3.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17f6bb557fd245c28e6411aa56b6403c689ad95061f50e4be16c274e70a17e48" +checksum = "c4a34ab300f2dee6e562c10a046fc05e358b29f9bf92277f30c3c8d82275f6f5" dependencies = [ "deranged", "itoa", "libc", "num_threads", + "powerfmt", "serde", "time-core", "time-macros", @@ -9818,15 +9827,15 @@ dependencies = [ [[package]] name = "time-core" -version = "0.1.1" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7300fbefb4dadc1af235a9cef3737cea692a9d97e1b9cbcd4ebdae6f8868e6fb" +checksum = "ef927ca75afb808a4d64dd374f00a2adf8d0fcff8e7b184af886c3c87ec4a3f3" [[package]] name = "time-macros" -version = "0.2.14" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a942f44339478ef67935ab2bbaec2fb0322496cf3cbe84b261e06ac3814c572" +checksum = "4ad70d68dba9e1f8aceda7aa6711965dfec1cac869f311a51bd08b3a2ccbce20" dependencies = [ "time-core", ] diff --git a/Cargo.toml b/Cargo.toml index ef09221b818a2..f8a9b7d0e2fa5 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -97,7 +97,7 @@ aws-smithy-types = "0.55" aws-endpoint = "0.55" aws-types = "0.55" etcd-client = { package = "madsim-etcd-client", version = "0.4" } -futures-async-stream = "0.2" +futures-async-stream = "0.2.9" hytra = "0.1" rdkafka = { package = "madsim-rdkafka", version = "0.3.0", features = [ "cmake-build", @@ -165,6 +165,8 @@ unused_must_use = "forbid" future_incompatible = "warn" nonstandard_style = "warn" rust_2018_idioms = "warn" +# Backward compatibility is not important for an application. +async_fn_in_trait = "allow" [workspace.lints.clippy] uninlined_format_args = "allow" @@ -229,8 +231,8 @@ opt-level = 2 incremental = false debug = 1 -# Patch third-party crates for deterministic simulation. [patch.crates-io] +# Patch third-party crates for deterministic simulation. quanta = { git = "https://github.com/madsim-rs/quanta.git", rev = "948bdc3" } getrandom = { git = "https://github.com/madsim-rs/getrandom.git", rev = "8daf97e" } tokio-stream = { git = "https://github.com/madsim-rs/tokio.git", rev = "fe39bb8e" } @@ -238,3 +240,8 @@ tokio-retry = { git = "https://github.com/madsim-rs/rust-tokio-retry.git", rev = tokio-postgres = { git = "https://github.com/madsim-rs/rust-postgres.git", rev = "ac00d88" } # patch: unlimit 4MB message size for grpc client etcd-client = { git = "https://github.com/risingwavelabs/etcd-client.git", rev = "4e84d40" } + +# Patch for coverage_attribute. +# https://github.com/sgodwincs/dlv-list-rs/pull/19#issuecomment-1774786289 +dlv-list = { git = "https://github.com/sgodwincs/dlv-list-rs.git", rev = "5bbc5d0" } +ordered-multimap = { git = "https://github.com/risingwavelabs/ordered-multimap-rs.git", rev = "19c743f" } diff --git a/ci/build-ci-image.sh b/ci/build-ci-image.sh index 43ff81ade2b85..59c88e5e9a9ae 100755 --- a/ci/build-ci-image.sh +++ b/ci/build-ci-image.sh @@ -13,7 +13,7 @@ cat ../rust-toolchain # !!! CHANGE THIS WHEN YOU WANT TO BUMP CI IMAGE !!! # # AND ALSO docker-compose.yml # ###################################################### -export BUILD_ENV_VERSION=v20230919 +export BUILD_ENV_VERSION=v20231022 export BUILD_TAG="public.ecr.aws/x5u3w5h6/rw-build-env:${BUILD_ENV_VERSION}" diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index 6fe7cfbfdeca2..48d88e7096889 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -71,7 +71,7 @@ services: retries: 5 source-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230919 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231022 depends_on: - mysql - db @@ -81,7 +81,7 @@ services: - ..:/risingwave sink-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230919 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231022 depends_on: - mysql - db @@ -92,12 +92,12 @@ services: - ..:/risingwave rw-build-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230919 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231022 volumes: - ..:/risingwave ci-flamegraph-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230919 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231022 # NOTE(kwannoel): This is used in order to permit # syscalls for `nperf` (perf_event_open), # so it can do CPU profiling. @@ -108,7 +108,7 @@ services: - ..:/risingwave regress-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20230919 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231022 depends_on: db: condition: service_healthy diff --git a/ci/rust-toolchain b/ci/rust-toolchain index ebc0b6c285a4e..fe2a026f6e40f 100644 --- a/ci/rust-toolchain +++ b/ci/rust-toolchain @@ -1,2 +1,2 @@ [toolchain] -channel = "nightly-2023-09-09" +channel = "nightly-2023-10-21" diff --git a/src/batch/src/lib.rs b/src/batch/src/lib.rs index 9104c96c951f5..809c096eb49df 100644 --- a/src/batch/src/lib.rs +++ b/src/batch/src/lib.rs @@ -17,8 +17,8 @@ #![feature(trait_alias)] #![feature(exact_size_is_empty)] #![feature(type_alias_impl_trait)] -#![cfg_attr(coverage, feature(no_coverage))] -#![feature(generators)] +#![cfg_attr(coverage, feature(coverage_attribute))] +#![feature(coroutines)] #![feature(proc_macro_hygiene, stmt_expr_attributes)] #![feature(iterator_try_collect)] #![feature(lint_reasons)] @@ -27,13 +27,11 @@ #![feature(let_chains)] #![feature(bound_map)] #![feature(int_roundings)] -#![feature(async_fn_in_trait)] #![feature(allocator_api)] #![feature(impl_trait_in_assoc_type)] #![feature(result_option_inspect)] #![feature(assert_matches)] #![feature(lazy_cell)] -#![feature(return_position_impl_trait_in_trait)] mod error; pub mod exchange_source; diff --git a/src/batch/src/rpc/service/task_service.rs b/src/batch/src/rpc/service/task_service.rs index b49a023acb22b..fb60e352ec293 100644 --- a/src/batch/src/rpc/service/task_service.rs +++ b/src/batch/src/rpc/service/task_service.rs @@ -53,7 +53,7 @@ impl TaskService for BatchServiceImpl { type CreateTaskStream = ReceiverStream; type ExecuteStream = ReceiverStream; - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn create_task( &self, request: Request, @@ -97,7 +97,7 @@ impl TaskService for BatchServiceImpl { } } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn cancel_task( &self, req: Request, @@ -109,7 +109,7 @@ impl TaskService for BatchServiceImpl { Ok(Response::new(CancelTaskResponse { status: None })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn execute( &self, req: Request, diff --git a/src/cmd/src/bin/compactor.rs b/src/cmd/src/bin/compactor.rs index 21b7db2405e2d..554168d8a6683 100644 --- a/src/cmd/src/bin/compactor.rs +++ b/src/cmd/src/bin/compactor.rs @@ -12,6 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] risingwave_cmd::main!(compactor); diff --git a/src/cmd/src/bin/compute_node.rs b/src/cmd/src/bin/compute_node.rs index 0bb1e5211ac57..a24d132b70b94 100644 --- a/src/cmd/src/bin/compute_node.rs +++ b/src/cmd/src/bin/compute_node.rs @@ -12,6 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] risingwave_cmd::main!(compute); diff --git a/src/cmd/src/bin/ctl.rs b/src/cmd/src/bin/ctl.rs index 38345c7a3fc2e..7b4c3132e747d 100644 --- a/src/cmd/src/bin/ctl.rs +++ b/src/cmd/src/bin/ctl.rs @@ -12,6 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] risingwave_cmd::main!(ctl); diff --git a/src/cmd/src/bin/frontend_node.rs b/src/cmd/src/bin/frontend_node.rs index 32d563be109fc..546bacbf1a901 100644 --- a/src/cmd/src/bin/frontend_node.rs +++ b/src/cmd/src/bin/frontend_node.rs @@ -12,6 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] risingwave_cmd::main!(frontend); diff --git a/src/cmd/src/bin/meta_node.rs b/src/cmd/src/bin/meta_node.rs index 032cc6bc28285..4bebfc5f915a2 100644 --- a/src/cmd/src/bin/meta_node.rs +++ b/src/cmd/src/bin/meta_node.rs @@ -12,6 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] risingwave_cmd::main!(meta); diff --git a/src/cmd/src/lib.rs b/src/cmd/src/lib.rs index 12de26657bd33..93df94a63816a 100644 --- a/src/cmd/src/lib.rs +++ b/src/cmd/src/lib.rs @@ -30,7 +30,7 @@ macro_rules! main { #[cfg(not(enable_task_local_alloc))] risingwave_common::enable_jemalloc!(); - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] fn main() { let opts = clap::Parser::parse(); $crate::$component(opts); diff --git a/src/cmd_all/src/bin/risingwave.rs b/src/cmd_all/src/bin/risingwave.rs index 3e9088e16b9e2..b7693c6fa06a2 100644 --- a/src/cmd_all/src/bin/risingwave.rs +++ b/src/cmd_all/src/bin/risingwave.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] use std::str::FromStr; @@ -158,7 +158,7 @@ impl Component { } } -#[cfg_attr(coverage, no_coverage)] +#[cfg_attr(coverage, coverage(off))] fn main() -> Result<()> { let risingwave = || { command!(BINARY_NAME) diff --git a/src/common/proc_macro/src/config.rs b/src/common/proc_macro/src/config.rs index 285834eb123cf..6e369fbad33eb 100644 --- a/src/common/proc_macro/src/config.rs +++ b/src/common/proc_macro/src/config.rs @@ -41,7 +41,7 @@ fn type_is_option(ty: &syn::Type) -> bool { false } -#[cfg_attr(coverage, no_coverage)] +#[cfg_attr(coverage, coverage(off))] pub fn produce_override_config(input: DeriveInput) -> TokenStream { let syn::Data::Struct(syn::DataStruct { fields, .. }) = input.data else { abort!(input, "Only struct is supported"); diff --git a/src/common/proc_macro/src/lib.rs b/src/common/proc_macro/src/lib.rs index 060ee1950624e..a11e407c6c053 100644 --- a/src/common/proc_macro/src/lib.rs +++ b/src/common/proc_macro/src/lib.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] use estimate_size::{ add_trait_bounds, extract_ignored_generics_list, has_nested_flag_attribute_list, @@ -52,7 +52,7 @@ mod estimate_size; /// } /// } /// ``` -#[cfg_attr(coverage, no_coverage)] +#[cfg_attr(coverage, coverage(off))] #[proc_macro_derive(OverrideConfig, attributes(override_opts))] #[proc_macro_error] pub fn override_config(input: TokenStream) -> TokenStream { diff --git a/src/common/src/lib.rs b/src/common/src/lib.rs index 2a3575d8dae78..fbcd3854fa572 100644 --- a/src/common/src/lib.rs +++ b/src/common/src/lib.rs @@ -24,12 +24,11 @@ #![feature(trusted_len)] #![feature(allocator_api)] #![feature(lint_reasons)] -#![feature(generators)] +#![feature(coroutines)] #![feature(map_try_insert)] #![feature(lazy_cell)] #![feature(error_generic_member_access)] #![feature(let_chains)] -#![feature(return_position_impl_trait_in_trait)] #![feature(portable_simd)] #![feature(array_chunks)] #![feature(inline_const_pat)] @@ -43,7 +42,6 @@ #![feature(result_option_inspect)] #![feature(map_entry_replace)] #![feature(negative_impls)] -#![feature(async_fn_in_trait)] #![feature(bound_map)] #![feature(array_methods)] diff --git a/src/common/src/types/ordered.rs b/src/common/src/types/ordered.rs index 75b07e529d7b9..68cd6329287e2 100644 --- a/src/common/src/types/ordered.rs +++ b/src/common/src/types/ordered.rs @@ -138,7 +138,7 @@ impl From for DefaultOrdered { } } -#[allow(clippy::incorrect_partial_ord_impl_on_ord_type)] +#[allow(clippy::non_canonical_partial_ord_impl)] impl PartialOrd for DefaultOrdered { fn partial_cmp(&self, other: &Self) -> Option { self.0.default_partial_cmp(other.as_inner()) diff --git a/src/compute/src/lib.rs b/src/compute/src/lib.rs index 65bf59eedf19e..fc5ae9ff19854 100644 --- a/src/compute/src/lib.rs +++ b/src/compute/src/lib.rs @@ -13,14 +13,14 @@ // limitations under the License. #![feature(trait_alias)] -#![feature(generators)] +#![feature(coroutines)] #![feature(type_alias_impl_trait)] #![feature(let_chains)] #![feature(result_option_inspect)] #![feature(lint_reasons)] #![feature(impl_trait_in_assoc_type)] #![feature(lazy_cell)] -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] #[macro_use] extern crate tracing; diff --git a/src/compute/src/rpc/service/exchange_service.rs b/src/compute/src/rpc/service/exchange_service.rs index b59cc39587c2f..6225cef2a7e30 100644 --- a/src/compute/src/rpc/service/exchange_service.rs +++ b/src/compute/src/rpc/service/exchange_service.rs @@ -49,7 +49,7 @@ impl ExchangeService for ExchangeServiceImpl { type GetDataStream = BatchDataStream; type GetStreamStream = StreamDataStream; - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn get_data( &self, request: Request, diff --git a/src/compute/src/rpc/service/monitor_service.rs b/src/compute/src/rpc/service/monitor_service.rs index 97a0b80773791..8fc24664ec016 100644 --- a/src/compute/src/rpc/service/monitor_service.rs +++ b/src/compute/src/rpc/service/monitor_service.rs @@ -53,7 +53,7 @@ impl MonitorServiceImpl { #[async_trait::async_trait] impl MonitorService for MonitorServiceImpl { - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn stack_trace( &self, request: Request, @@ -85,7 +85,7 @@ impl MonitorService for MonitorServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn profiling( &self, request: Request, @@ -115,7 +115,7 @@ impl MonitorService for MonitorServiceImpl { } } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn heap_profiling( &self, request: Request, @@ -166,7 +166,7 @@ impl MonitorService for MonitorServiceImpl { } } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn list_heap_profiling( &self, _request: Request, @@ -206,7 +206,7 @@ impl MonitorService for MonitorServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn analyze_heap( &self, request: Request, diff --git a/src/compute/src/rpc/service/stream_service.rs b/src/compute/src/rpc/service/stream_service.rs index 525364b60dc1c..1c1448b3d1e45 100644 --- a/src/compute/src/rpc/service/stream_service.rs +++ b/src/compute/src/rpc/service/stream_service.rs @@ -45,7 +45,7 @@ impl StreamServiceImpl { #[async_trait::async_trait] impl StreamService for StreamServiceImpl { - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn update_actors( &self, request: Request, @@ -61,7 +61,7 @@ impl StreamService for StreamServiceImpl { } } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn build_actors( &self, request: Request, @@ -85,7 +85,7 @@ impl StreamService for StreamServiceImpl { } } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn broadcast_actor_info_table( &self, request: Request, @@ -104,7 +104,7 @@ impl StreamService for StreamServiceImpl { } } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn drop_actors( &self, request: Request, @@ -118,7 +118,7 @@ impl StreamService for StreamServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn force_stop_actors( &self, request: Request, @@ -132,7 +132,7 @@ impl StreamService for StreamServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn inject_barrier( &self, request: Request, @@ -173,7 +173,7 @@ impl StreamService for StreamServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn barrier_complete( &self, request: Request, @@ -243,7 +243,7 @@ impl StreamService for StreamServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn wait_epoch_commit( &self, request: Request, diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index b3e39ece95002..6a50b8410bbd4 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -13,7 +13,7 @@ // limitations under the License. #![feature(let_chains)] -#![feature(generators)] +#![feature(coroutines)] use std::sync::atomic::AtomicU64; use std::sync::Arc; diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index a43ae2e5762da..6d7e93365c275 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![feature(generators)] +#![feature(coroutines)] #![feature(proc_macro_hygiene, stmt_expr_attributes)] use std::sync::atomic::AtomicU64; diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 4886b1b52fcc5..87d2a0bdef689 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -113,7 +113,7 @@ strum = "0.25" strum_macros = "0.25" tempfile = "3" thiserror = "1" -time = "0.3.28" +time = "0.3.30" tokio = { version = "0.2", package = "madsim-tokio", features = [ "rt", "rt-multi-thread", diff --git a/src/connector/src/lib.rs b/src/connector/src/lib.rs index 8ccf62486ce65..aa613b4043c23 100644 --- a/src/connector/src/lib.rs +++ b/src/connector/src/lib.rs @@ -14,7 +14,7 @@ #![expect(dead_code)] #![allow(clippy::derive_partial_eq_without_eq)] -#![feature(generators)] +#![feature(coroutines)] #![feature(proc_macro_hygiene)] #![feature(stmt_expr_attributes)] #![feature(box_patterns)] @@ -25,11 +25,9 @@ #![feature(let_chains)] #![feature(box_into_inner)] #![feature(type_alias_impl_trait)] -#![feature(return_position_impl_trait_in_trait)] -#![feature(async_fn_in_trait)] #![feature(associated_type_defaults)] #![feature(impl_trait_in_assoc_type)] -#![feature(iter_from_generator)] +#![feature(iter_from_coroutine)] #![feature(if_let_guard)] #![feature(iterator_try_collect)] diff --git a/src/connector/src/sink/formatter/append_only.rs b/src/connector/src/sink/formatter/append_only.rs index 523a52dab91bb..f0efcc21d9009 100644 --- a/src/connector/src/sink/formatter/append_only.rs +++ b/src/connector/src/sink/formatter/append_only.rs @@ -40,7 +40,7 @@ impl SinkFormatter for AppendOnlyFormatter impl Iterator, Option)>> { - std::iter::from_generator(|| { + std::iter::from_coroutine(|| { for (op, row) in chunk.rows() { if op != Op::Insert { continue; diff --git a/src/connector/src/sink/formatter/debezium_json.rs b/src/connector/src/sink/formatter/debezium_json.rs index 637aa23f06410..ce98daab88756 100644 --- a/src/connector/src/sink/formatter/debezium_json.rs +++ b/src/connector/src/sink/formatter/debezium_json.rs @@ -85,7 +85,7 @@ impl SinkFormatter for DebeziumJsonFormatter { &self, chunk: &StreamChunk, ) -> impl Iterator, Option)>> { - std::iter::from_generator(|| { + std::iter::from_coroutine(|| { let DebeziumJsonFormatter { schema, pk_indices, diff --git a/src/connector/src/sink/formatter/upsert.rs b/src/connector/src/sink/formatter/upsert.rs index 6ef2b5f2ca333..af8e70ff92850 100644 --- a/src/connector/src/sink/formatter/upsert.rs +++ b/src/connector/src/sink/formatter/upsert.rs @@ -40,7 +40,7 @@ impl SinkFormatter for UpsertFormatter { &self, chunk: &StreamChunk, ) -> impl Iterator, Option)>> { - std::iter::from_generator(|| { + std::iter::from_coroutine(|| { for (op, row) in chunk.rows() { let event_key_object = Some(tri!(self.key_encoder.encode(row))); diff --git a/src/expr/core/src/lib.rs b/src/expr/core/src/lib.rs index c2f46d5632274..b49c4ae161dfc 100644 --- a/src/expr/core/src/lib.rs +++ b/src/expr/core/src/lib.rs @@ -17,7 +17,7 @@ #![feature(lint_reasons)] #![feature(iterator_try_collect)] #![feature(lazy_cell)] -#![feature(generators)] +#![feature(coroutines)] #![feature(arc_unwrap_or_clone)] #![feature(never_type)] diff --git a/src/expr/impl/src/lib.rs b/src/expr/impl/src/lib.rs index a5906e4320282..6ea82d30ac5f1 100644 --- a/src/expr/impl/src/lib.rs +++ b/src/expr/impl/src/lib.rs @@ -28,7 +28,7 @@ #![feature(exclusive_range_pattern)] #![feature(lazy_cell)] #![feature(round_ties_even)] -#![feature(generators)] +#![feature(coroutines)] #![feature(test)] #![feature(arc_unwrap_or_clone)] diff --git a/src/frontend/src/binder/mod.rs b/src/frontend/src/binder/mod.rs index 974730cd16237..f1038f9bf5943 100644 --- a/src/frontend/src/binder/mod.rs +++ b/src/frontend/src/binder/mod.rs @@ -363,6 +363,13 @@ impl Binder { } } +/// The column name stored in [`BindContext`] for a column without an alias. +pub const UNNAMED_COLUMN: &str = "?column?"; +/// The table name stored in [`BindContext`] for a subquery without an alias. +const UNNAMED_SUBQUERY: &str = "?subquery?"; +/// The table name stored in [`BindContext`] for a column group. +const COLUMN_GROUP_PREFIX: &str = "?column_group_id?"; + #[cfg(test)] pub mod test_utils { use risingwave_common::types::DataType; @@ -380,10 +387,3 @@ pub mod test_utils { Binder::new_with_param_types(&SessionImpl::mock(), param_types) } } - -/// The column name stored in [`BindContext`] for a column without an alias. -pub const UNNAMED_COLUMN: &str = "?column?"; -/// The table name stored in [`BindContext`] for a subquery without an alias. -const UNNAMED_SUBQUERY: &str = "?subquery?"; -/// The table name stored in [`BindContext`] for a column group. -const COLUMN_GROUP_PREFIX: &str = "?column_group_id?"; diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 0a036b8e96233..450f49b6394cf 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -15,7 +15,7 @@ #![allow(clippy::derive_partial_eq_without_eq)] #![feature(map_try_insert)] #![feature(negative_impls)] -#![feature(generators)] +#![feature(coroutines)] #![feature(proc_macro_hygiene, stmt_expr_attributes)] #![feature(trait_alias)] #![feature(extract_if)] @@ -32,7 +32,6 @@ #![feature(extend_one)] #![feature(type_alias_impl_trait)] #![feature(impl_trait_in_assoc_type)] -#![feature(async_fn_in_trait)] #![feature(result_flattening)] #![recursion_limit = "256"] diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index 55c7b27b0c80a..bf1bddad2070f 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -14,7 +14,7 @@ #![feature(lint_reasons)] #![feature(let_chains)] -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] mod server; use std::time::Duration; diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 935d398aeacb0..2fa5f50e15666 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -717,7 +717,7 @@ impl DdlService for DdlServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn get_tables( &self, request: Request, diff --git a/src/meta/service/src/heartbeat_service.rs b/src/meta/service/src/heartbeat_service.rs index 7c51b39346894..e31058ff2bdc5 100644 --- a/src/meta/service/src/heartbeat_service.rs +++ b/src/meta/service/src/heartbeat_service.rs @@ -32,7 +32,7 @@ impl HeartbeatServiceImpl { #[async_trait::async_trait] impl HeartbeatService for HeartbeatServiceImpl { - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn heartbeat( &self, request: Request, diff --git a/src/meta/service/src/lib.rs b/src/meta/service/src/lib.rs index 0d473a6ed031f..6c8cc11f8971c 100644 --- a/src/meta/service/src/lib.rs +++ b/src/meta/service/src/lib.rs @@ -16,7 +16,7 @@ #![feature(let_chains)] #![feature(lazy_cell)] #![feature(impl_trait_in_assoc_type)] -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] use risingwave_meta::*; diff --git a/src/meta/service/src/meta_member_service.rs b/src/meta/service/src/meta_member_service.rs index 25c4c7ad4cc84..5753061176e8c 100644 --- a/src/meta/service/src/meta_member_service.rs +++ b/src/meta/service/src/meta_member_service.rs @@ -36,7 +36,7 @@ impl MetaMemberServiceImpl { #[async_trait::async_trait] impl MetaMemberService for MetaMemberServiceImpl { - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn members( &self, _request: Request, diff --git a/src/meta/service/src/notification_service.rs b/src/meta/service/src/notification_service.rs index bd247c1e18980..0fcc470a70e39 100644 --- a/src/meta/service/src/notification_service.rs +++ b/src/meta/service/src/notification_service.rs @@ -207,7 +207,7 @@ impl NotificationServiceImpl { impl NotificationService for NotificationServiceImpl { type SubscribeStream = UnboundedReceiverStream; - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn subscribe( &self, request: Request, diff --git a/src/meta/service/src/scale_service.rs b/src/meta/service/src/scale_service.rs index f231ea5f4955d..676180adc7581 100644 --- a/src/meta/service/src/scale_service.rs +++ b/src/meta/service/src/scale_service.rs @@ -59,7 +59,7 @@ impl ScaleServiceImpl { #[async_trait::async_trait] impl ScaleService for ScaleServiceImpl { - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn get_cluster_info( &self, _: Request, @@ -110,7 +110,7 @@ impl ScaleService for ScaleServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn reschedule( &self, request: Request, @@ -174,7 +174,7 @@ impl ScaleService for ScaleServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn get_reschedule_plan( &self, request: Request, diff --git a/src/meta/service/src/stream_service.rs b/src/meta/service/src/stream_service.rs index ef232d9b04ffd..92af1d4beb707 100644 --- a/src/meta/service/src/stream_service.rs +++ b/src/meta/service/src/stream_service.rs @@ -59,7 +59,7 @@ impl StreamServiceImpl { #[async_trait::async_trait] impl StreamManagerService for StreamServiceImpl { - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn flush(&self, request: Request) -> TonicResponse { self.env.idle_manager().record_activity(); let req = request.into_inner(); @@ -71,7 +71,7 @@ impl StreamManagerService for StreamServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn pause(&self, _: Request) -> Result, Status> { let i = self .barrier_scheduler @@ -83,7 +83,7 @@ impl StreamManagerService for StreamServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn resume(&self, _: Request) -> Result, Status> { let i = self .barrier_scheduler @@ -122,7 +122,7 @@ impl StreamManagerService for StreamServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn list_table_fragments( &self, request: Request, @@ -165,7 +165,7 @@ impl StreamManagerService for StreamServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn list_table_fragment_states( &self, _request: Request, @@ -186,7 +186,7 @@ impl StreamManagerService for StreamServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn list_fragment_distribution( &self, _request: Request, @@ -215,7 +215,7 @@ impl StreamManagerService for StreamServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn list_actor_states( &self, _request: Request, diff --git a/src/meta/service/src/user_service.rs b/src/meta/service/src/user_service.rs index 8c982521b112a..cb290766e6fd1 100644 --- a/src/meta/service/src/user_service.rs +++ b/src/meta/service/src/user_service.rs @@ -107,7 +107,7 @@ impl UserServiceImpl { #[async_trait::async_trait] impl UserService for UserServiceImpl { - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn create_user( &self, request: Request, @@ -128,7 +128,7 @@ impl UserService for UserServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn drop_user( &self, request: Request, @@ -142,7 +142,7 @@ impl UserService for UserServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn update_user( &self, request: Request, @@ -165,7 +165,7 @@ impl UserService for UserServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn grant_privilege( &self, request: Request, @@ -185,7 +185,7 @@ impl UserService for UserServiceImpl { })) } - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] async fn revoke_privilege( &self, request: Request, diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 2b0c3e3db87dc..1b3a284e9ccc9 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -1761,7 +1761,7 @@ impl HummockManager { } /// Get version deltas from meta store - #[cfg_attr(coverage, no_coverage)] + #[cfg_attr(coverage, coverage(off))] #[named] pub async fn list_version_deltas( &self, diff --git a/src/meta/src/lib.rs b/src/meta/src/lib.rs index afe66d27ad8e8..f549578f079c6 100644 --- a/src/meta/src/lib.rs +++ b/src/meta/src/lib.rs @@ -26,13 +26,12 @@ #![feature(error_generic_member_access)] #![feature(assert_matches)] #![feature(try_blocks)] -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] #![feature(custom_test_frameworks)] #![test_runner(risingwave_test_runner::test_runner::run_failpont_tests)] #![feature(is_sorted)] #![feature(impl_trait_in_assoc_type)] #![feature(type_name_of_val)] -#![feature(async_fn_in_trait)] pub mod backup_restore; pub mod barrier; diff --git a/src/prost/helpers/src/lib.rs b/src/prost/helpers/src/lib.rs index f4d1d1a45baa1..5796e14273fe9 100644 --- a/src/prost/helpers/src/lib.rs +++ b/src/prost/helpers/src/lib.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] #![feature(iterator_try_collect)] use proc_macro::TokenStream; @@ -24,7 +24,7 @@ mod generate; /// This attribute will be placed before any pb types, including messages and enums. /// See `prost/helpers/README.md` for more details. -#[cfg_attr(coverage, no_coverage)] +#[cfg_attr(coverage, coverage(off))] #[proc_macro_derive(AnyPB)] pub fn any_pb(input: TokenStream) -> TokenStream { // Parse the string representation @@ -37,7 +37,7 @@ pub fn any_pb(input: TokenStream) -> TokenStream { } // Procedure macros can not be tested from the same crate. -#[cfg_attr(coverage, no_coverage)] +#[cfg_attr(coverage, coverage(off))] fn produce(ast: &DeriveInput) -> Result { let name = &ast.ident; diff --git a/src/rpc_client/src/lib.rs b/src/rpc_client/src/lib.rs index 3e744bb61608d..be871521d3bc4 100644 --- a/src/rpc_client/src/lib.rs +++ b/src/rpc_client/src/lib.rs @@ -19,7 +19,7 @@ #![feature(result_option_inspect)] #![feature(type_alias_impl_trait)] #![feature(associated_type_defaults)] -#![feature(generators)] +#![feature(coroutines)] #![feature(iterator_try_collect)] #![feature(hash_extract_if)] #![feature(try_blocks)] diff --git a/src/source/src/lib.rs b/src/source/src/lib.rs index 1a32888cdf651..aaa045c607c95 100644 --- a/src/source/src/lib.rs +++ b/src/source/src/lib.rs @@ -16,7 +16,7 @@ #![feature(trait_alias)] #![feature(lint_reasons)] #![feature(result_option_inspect)] -#![feature(generators)] +#![feature(coroutines)] #![feature(hash_extract_if)] #![feature(type_alias_impl_trait)] #![feature(box_patterns)] diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index c6fc5531acd33..fc01eba294564 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -25,7 +25,7 @@ dyn-clone = "1.0.14" either = "1" enum-as-inner = "0.6" fail = "0.5" -foyer = { git = "https://github.com/mrcroxx/foyer", rev = "5d0134b" } +foyer = { git = "https://github.com/MrCroxx/foyer", rev = "2261151" } futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } hex = "0.4" diff --git a/src/storage/backup/src/lib.rs b/src/storage/backup/src/lib.rs index 3e0549db188a2..1daacbf691c0d 100644 --- a/src/storage/backup/src/lib.rs +++ b/src/storage/backup/src/lib.rs @@ -25,7 +25,7 @@ #![feature(lazy_cell)] #![feature(let_chains)] #![feature(error_generic_member_access)] -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] pub mod error; pub mod meta_snapshot; diff --git a/src/storage/hummock_test/Cargo.toml b/src/storage/hummock_test/Cargo.toml index 600a5249ddf1b..8abf2f45e6855 100644 --- a/src/storage/hummock_test/Cargo.toml +++ b/src/storage/hummock_test/Cargo.toml @@ -20,7 +20,7 @@ bytes = { version = "1" } clap = { version = "4", features = ["derive"] } fail = "0.5" futures = { version = "0.3", default-features = false, features = ["alloc"] } -futures-async-stream = "0.2" +futures-async-stream = "0.2.9" itertools = "0.11" parking_lot = "0.12" rand = "0.8" @@ -47,7 +47,7 @@ futures = { version = "0.3", default-features = false, features = [ "executor", ] } -futures-async-stream = "0.2" +futures-async-stream = "0.2.9" risingwave_test_runner = { workspace = true } serial_test = "2.0" sync-point = { path = "../../utils/sync-point" } diff --git a/src/storage/hummock_test/src/bin/replay/main.rs b/src/storage/hummock_test/src/bin/replay/main.rs index 1e9c9591bc864..ae6038d8b5d16 100644 --- a/src/storage/hummock_test/src/bin/replay/main.rs +++ b/src/storage/hummock_test/src/bin/replay/main.rs @@ -13,7 +13,7 @@ // limitations under the License. #![feature(bound_map)] -#![feature(generators)] +#![feature(coroutines)] #![feature(stmt_expr_attributes)] #![feature(proc_macro_hygiene)] diff --git a/src/storage/hummock_test/src/lib.rs b/src/storage/hummock_test/src/lib.rs index 73e1d8cd0eaad..593771435f1e0 100644 --- a/src/storage/hummock_test/src/lib.rs +++ b/src/storage/hummock_test/src/lib.rs @@ -17,7 +17,6 @@ #![feature(bound_map)] #![feature(type_alias_impl_trait)] #![feature(associated_type_bounds)] -#![feature(return_position_impl_trait_in_trait)] #[cfg(test)] mod compactor_tests; diff --git a/src/storage/hummock_trace/Cargo.toml b/src/storage/hummock_trace/Cargo.toml index 46eabf17835e4..150b35b79cda0 100644 --- a/src/storage/hummock_trace/Cargo.toml +++ b/src/storage/hummock_trace/Cargo.toml @@ -14,7 +14,7 @@ bincode = { version = "=2.0.0-rc.3", features = ["serde"] } byteorder = "1" bytes = { version = "1", features = ["serde"] } futures = { version = "0.3", default-features = false, features = ["alloc"] } -futures-async-stream = "0.2" +futures-async-stream = "0.2.9" parking_lot = "0.12" prost = { workspace = true } risingwave_common = { workspace = true } diff --git a/src/storage/hummock_trace/src/lib.rs b/src/storage/hummock_trace/src/lib.rs index df757c58cc4fa..8c6c8913205ab 100644 --- a/src/storage/hummock_trace/src/lib.rs +++ b/src/storage/hummock_trace/src/lib.rs @@ -16,7 +16,7 @@ #![feature(cursor_remaining)] #![feature(bound_map)] #![feature(trait_alias)] -#![feature(generators)] +#![feature(coroutines)] mod collector; mod error; diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index 583bab3d10b3c..a21016014d247 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -140,9 +140,6 @@ impl CompactorRunner { Ok((self.split_index, ssts, compaction_stat)) } - // This is a clippy bug, see https://github.com/rust-lang/rust-clippy/issues/11380. - // TODO: remove `allow` here after the issued is closed. - #[expect(clippy::needless_pass_by_ref_mut)] pub async fn build_delete_range_iter( sstable_infos: &Vec, sstable_store: &SstableStoreRef, diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index d2f36167675e7..137682d6f7825 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -325,7 +325,7 @@ impl Compactor { /// The background compaction thread that receives compaction tasks from hummock compaction /// manager and runs compaction tasks. -#[cfg_attr(coverage, no_coverage)] +#[cfg_attr(coverage, coverage(off))] pub fn start_compactor( compactor_context: CompactorContext, hummock_meta_client: Arc, @@ -618,7 +618,7 @@ pub fn start_compactor( /// The background compaction thread that receives compaction tasks from hummock compaction /// manager and runs compaction tasks. -#[cfg_attr(coverage, no_coverage)] +#[cfg_attr(coverage, coverage(off))] pub fn start_shared_compactor( grpc_proxy_client: GrpcCompactorProxyClient, mut receiver: mpsc::UnboundedReceiver>, diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index 995a9d181e2f5..a07da55fb7046 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -1641,9 +1641,6 @@ mod tests { (buffer_tracker, uploader, new_task_notifier) } - // This is a clippy bug, see https://github.com/rust-lang/rust-clippy/issues/11380. - // TODO: remove `allow` here after the issued is closed. - #[expect(clippy::needless_pass_by_ref_mut)] async fn assert_uploader_pending(uploader: &mut HummockUploader) { for _ in 0..10 { yield_now().await; diff --git a/src/storage/src/lib.rs b/src/storage/src/lib.rs index c5ffe656ab893..0f2f155f6a903 100644 --- a/src/storage/src/lib.rs +++ b/src/storage/src/lib.rs @@ -18,7 +18,7 @@ #![feature(bound_map)] #![feature(custom_test_frameworks)] #![feature(extract_if)] -#![feature(generators)] +#![feature(coroutines)] #![feature(hash_extract_if)] #![feature(lint_reasons)] #![feature(proc_macro_hygiene)] @@ -35,15 +35,13 @@ #![feature(btree_extract_if)] #![feature(exact_size_is_empty)] #![feature(lazy_cell)] -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] #![recursion_limit = "256"] #![feature(error_generic_member_access)] #![feature(let_chains)] #![feature(associated_type_bounds)] #![feature(exclusive_range_pattern)] #![feature(impl_trait_in_assoc_type)] -#![feature(async_fn_in_trait)] -#![feature(return_position_impl_trait_in_trait)] pub mod hummock; pub mod memory; diff --git a/src/stream/src/common/table/state_table_cache.rs b/src/stream/src/common/table/state_table_cache.rs index 156637a41a1a4..b458ef52537e4 100644 --- a/src/stream/src/common/table/state_table_cache.rs +++ b/src/stream/src/common/table/state_table_cache.rs @@ -67,9 +67,9 @@ type WatermarkCacheKey = DefaultOrdered; /// Issue delete ranges. /// /// B. Refreshing the cache: -/// On barrier, do table scan from most_recently_cleaned_watermark (inclusive) to +inf. +/// On barrier, do table scan from `most_recently_cleaned_watermark` (inclusive) to +inf. /// Take the Top N rows and insert into cache. -/// This has to be implemented in state_table. +/// This has to be implemented in `state_table`. /// We do not need to store any values, just the keys. /// /// TODO(kwannoel): diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 259b67d5f202b..663f9be94cf5e 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -309,9 +309,6 @@ pub(crate) async fn get_progress_per_vnode( table: &mut StateTableInner, epoch: EpochPair, diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index 4178012cb9d9e..75414fe24a379 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -202,11 +202,11 @@ impl std::fmt::Debug for JoinSide { impl JoinSide { // WARNING: Please do not call this until we implement it. - #[expect(dead_code)] fn is_dirty(&self) -> bool { unimplemented!() } + #[expect(dead_code)] fn clear_cache(&mut self) { assert!( !self.is_dirty(), diff --git a/src/stream/src/lib.rs b/src/stream/src/lib.rs index 5a68b1b712b26..389dfae7b8c0c 100644 --- a/src/stream/src/lib.rs +++ b/src/stream/src/lib.rs @@ -21,8 +21,8 @@ #![feature(let_chains)] #![feature(hash_extract_if)] #![feature(extract_if)] -#![feature(generators)] -#![feature(iter_from_generator)] +#![feature(coroutines)] +#![feature(iter_from_coroutine)] #![feature(proc_macro_hygiene)] #![feature(stmt_expr_attributes)] #![feature(allocator_api)] @@ -36,13 +36,11 @@ #![feature(bound_map)] #![feature(iter_order_by)] #![feature(exact_size_is_empty)] -#![feature(return_position_impl_trait_in_trait)] #![feature(impl_trait_in_assoc_type)] #![feature(test)] #![feature(is_sorted)] #![feature(btree_cursors)] #![feature(assert_matches)] -#![feature(async_fn_in_trait)] #[macro_use] extern crate tracing; diff --git a/src/stream/tests/integration_tests/hash_agg.rs b/src/stream/tests/integration_tests/hash_agg.rs index 1b61bc5cd1d7f..9f4908f252532 100644 --- a/src/stream/tests/integration_tests/hash_agg.rs +++ b/src/stream/tests/integration_tests/hash_agg.rs @@ -284,7 +284,7 @@ async fn test_hash_agg_emit_on_window_close() { }; check_with_script( - || create_executor(), + create_executor, &format!( r###" - !barrier 1 diff --git a/src/tests/compaction_test/src/bin/compaction.rs b/src/tests/compaction_test/src/bin/compaction.rs index 443b79ad625b8..d9ba16f7437b8 100644 --- a/src/tests/compaction_test/src/bin/compaction.rs +++ b/src/tests/compaction_test/src/bin/compaction.rs @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] -#[cfg_attr(coverage, no_coverage)] +#[cfg_attr(coverage, coverage(off))] fn main() { use clap::Parser; diff --git a/src/tests/compaction_test/src/bin/delete_range.rs b/src/tests/compaction_test/src/bin/delete_range.rs index 348a71dc3cce5..592f61a3db4fa 100644 --- a/src/tests/compaction_test/src/bin/delete_range.rs +++ b/src/tests/compaction_test/src/bin/delete_range.rs @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![cfg_attr(coverage, feature(no_coverage))] +#![cfg_attr(coverage, feature(coverage_attribute))] -#[cfg_attr(coverage, no_coverage)] +#[cfg_attr(coverage, coverage(off))] fn main() { use clap::Parser; diff --git a/src/utils/pgwire/src/lib.rs b/src/utils/pgwire/src/lib.rs index 1cda373ee9568..84a17d9907879 100644 --- a/src/utils/pgwire/src/lib.rs +++ b/src/utils/pgwire/src/lib.rs @@ -17,8 +17,6 @@ #![feature(result_option_inspect)] #![feature(iterator_try_collect)] #![feature(trusted_len)] -#![feature(async_fn_in_trait)] -#![feature(return_position_impl_trait_in_trait)] #![feature(lazy_cell)] #![expect(clippy::doc_markdown, reason = "FIXME: later")] diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index 6c08e08490f7d..67b218c787652 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -37,7 +37,7 @@ combine = { version = "4", features = ["tokio"] } crossbeam-epoch = { version = "0.9" } crossbeam-queue = { version = "0.3" } crossbeam-utils = { version = "0.8" } -deranged = { version = "0.3", default-features = false, features = ["serde", "std"] } +deranged = { version = "0.3", default-features = false, features = ["powerfmt", "serde", "std"] } digest = { version = "0.10", features = ["mac", "oid", "std"] } either = { version = "1", features = ["serde"] } fail = { version = "0.5", default-features = false, features = ["failpoints"] } @@ -147,7 +147,7 @@ auto_enums = { version = "0.8", features = ["futures03"] } bitflags = { version = "2", default-features = false, features = ["serde", "std"] } bytes = { version = "1", features = ["serde"] } cc = { version = "1", default-features = false, features = ["parallel"] } -deranged = { version = "0.3", default-features = false, features = ["serde", "std"] } +deranged = { version = "0.3", default-features = false, features = ["powerfmt", "serde", "std"] } either = { version = "1", features = ["serde"] } fixedbitset = { version = "0.4" } frunk_core = { version = "0.4", default-features = false, features = ["std"] } From a4b37a3f7cd990f21706596351115b678dab0aa7 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Wed, 25 Oct 2023 10:49:52 +0800 Subject: [PATCH 12/48] ci(sink): as source, start kafka with docker-compose rather than risedev (#13011) --- ci/docker-compose.yml | 1 + ci/scripts/e2e-kafka-sink-test.sh | 40 ++++++++++++++--------------- ci/scripts/e2e-sink-test.sh | 2 +- e2e_test/sink/kafka/create_sink.slt | 22 ++++++++-------- e2e_test/sink/kafka/protobuf.slt | 10 ++++---- risedev.yml | 34 ------------------------ 6 files changed, 38 insertions(+), 71 deletions(-) diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index 48d88e7096889..66dd2d175e675 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -85,6 +85,7 @@ services: depends_on: - mysql - db + - message_queue - elasticsearch - clickhouse-server - pulsar diff --git a/ci/scripts/e2e-kafka-sink-test.sh b/ci/scripts/e2e-kafka-sink-test.sh index 06ef185f46e8b..71a91f2d8fba9 100755 --- a/ci/scripts/e2e-kafka-sink-test.sh +++ b/ci/scripts/e2e-kafka-sink-test.sh @@ -3,10 +3,10 @@ # Exits as soon as any line fails. set -euo pipefail -./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-append-only --create > /dev/null 2>&1 -./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-upsert --create > /dev/null 2>&1 -./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-upsert-schema --create > /dev/null 2>&1 -./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-debezium --create > /dev/null 2>&1 +./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-append-only --create > /dev/null 2>&1 +./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-upsert --create > /dev/null 2>&1 +./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-upsert-schema --create > /dev/null 2>&1 +./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-debezium --create > /dev/null 2>&1 sqllogictest -p 4566 -d dev 'e2e_test/sink/kafka/create_sink.slt' sleep 2 @@ -14,7 +14,7 @@ sleep 2 # test append-only kafka sink echo "testing append-only kafka sink" diff ./e2e_test/sink/kafka/append_only1.result \ -<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-append-only --from-beginning --max-messages 10 | sort) 2> /dev/null) +<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-append-only --from-beginning --max-messages 10 | sort) 2> /dev/null) if [ $? -ne 0 ]; then echo "The output for append-only sink is not as expected." exit 1 @@ -23,7 +23,7 @@ fi # test upsert kafka sink echo "testing upsert kafka sink" diff ./e2e_test/sink/kafka/upsert1.result \ -<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-upsert --from-beginning --property print.key=true --max-messages 10 | sort) 2> /dev/null) +<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-upsert --from-beginning --property print.key=true --max-messages 10 | sort) 2> /dev/null) if [ $? -ne 0 ]; then echo "The output for upsert sink is not as expected." exit 1 @@ -32,7 +32,7 @@ fi # test upsert kafka sink with schema echo "testing upsert kafka sink with schema" diff ./e2e_test/sink/kafka/upsert_schema1.result \ -<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-upsert-schema --from-beginning --property print.key=true --max-messages 10 | sort) 2> /dev/null) +<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-upsert-schema --from-beginning --property print.key=true --max-messages 10 | sort) 2> /dev/null) if [ $? -ne 0 ]; then echo "The output for upsert sink with schema is not as expected." exit 1 @@ -40,7 +40,7 @@ fi # test debezium kafka sink echo "testing debezium kafka sink" -(./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-debezium --property print.key=true --from-beginning --max-messages 10 | sort) > ./e2e_test/sink/kafka/debezium1.tmp.result 2> /dev/null +(./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-debezium --property print.key=true --from-beginning --max-messages 10 | sort) > ./e2e_test/sink/kafka/debezium1.tmp.result 2> /dev/null python3 e2e_test/sink/kafka/debezium.py e2e_test/sink/kafka/debezium1.result e2e_test/sink/kafka/debezium1.tmp.result if [ $? -ne 0 ]; then echo "The output for debezium sink is not as expected." @@ -57,7 +57,7 @@ psql -h localhost -p 4566 -d dev -U root -c "update t_kafka set v_varchar = '', # test append-only kafka sink after update echo "testing append-only kafka sink after updating data" diff ./e2e_test/sink/kafka/append_only2.result \ -<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-append-only --from-beginning --max-messages 11 | sort) 2> /dev/null) +<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-append-only --from-beginning --max-messages 11 | sort) 2> /dev/null) if [ $? -ne 0 ]; then echo "The output for append-only sink after update is not as expected." exit 1 @@ -66,7 +66,7 @@ fi # test upsert kafka sink after update echo "testing upsert kafka sink after updating data" diff ./e2e_test/sink/kafka/upsert2.result \ -<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-upsert --from-beginning --property print.key=true --max-messages 11 | sort) 2> /dev/null) +<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-upsert --from-beginning --property print.key=true --max-messages 11 | sort) 2> /dev/null) if [ $? -ne 0 ]; then echo "The output for upsert sink after update is not as expected." exit 1 @@ -75,7 +75,7 @@ fi # test upsert kafka sink with schema after update echo "testing upsert kafka sink with schema after updating data" diff ./e2e_test/sink/kafka/upsert_schema2.result \ -<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-upsert-schema --from-beginning --property print.key=true --max-messages 11 | sort) 2> /dev/null) +<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-upsert-schema --from-beginning --property print.key=true --max-messages 11 | sort) 2> /dev/null) if [ $? -ne 0 ]; then echo "The output for upsert sink with schema is not as expected." exit 1 @@ -83,7 +83,7 @@ fi # test debezium kafka sink after update echo "testing debezium kafka sink after updating data" -(./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-debezium --property print.key=true --from-beginning --max-messages 11 | sort) > ./e2e_test/sink/kafka/debezium2.tmp.result 2> /dev/null +(./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-debezium --property print.key=true --from-beginning --max-messages 11 | sort) > ./e2e_test/sink/kafka/debezium2.tmp.result 2> /dev/null python3 e2e_test/sink/kafka/debezium.py e2e_test/sink/kafka/debezium2.result e2e_test/sink/kafka/debezium2.tmp.result if [ $? -ne 0 ]; then echo "The output for debezium sink after update is not as expected." @@ -100,7 +100,7 @@ psql -h localhost -p 4566 -d dev -U root -c "delete from t_kafka where id = 1;" # test upsert kafka sink after delete echo "testing upsert kafka sink after deleting data" diff ./e2e_test/sink/kafka/upsert3.result \ -<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-upsert --from-beginning --property print.key=true --max-messages 12 | sort) 2> /dev/null) +<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-upsert --from-beginning --property print.key=true --max-messages 12 | sort) 2> /dev/null) if [ $? -ne 0 ]; then echo "The output for upsert sink after update is not as expected." exit 1 @@ -109,7 +109,7 @@ fi # test upsert kafka sink with schema after delete echo "testing upsert kafka sink with schema after deleting data" diff ./e2e_test/sink/kafka/upsert_schema3.result \ -<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-upsert-schema --from-beginning --property print.key=true --max-messages 12 | sort) 2> /dev/null) +<((./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-upsert-schema --from-beginning --property print.key=true --max-messages 12 | sort) 2> /dev/null) if [ $? -ne 0 ]; then echo "The output for upsert sink with schema is not as expected." exit 1 @@ -117,7 +117,7 @@ fi # test debezium kafka sink after delete echo "testing debezium kafka sink after deleting data" -(./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-debezium --property print.key=true --from-beginning --max-messages 13 | sort) > ./e2e_test/sink/kafka/debezium3.tmp.result 2> /dev/null +(./.risingwave/bin/kafka/bin/kafka-console-consumer.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-debezium --property print.key=true --from-beginning --max-messages 13 | sort) > ./e2e_test/sink/kafka/debezium3.tmp.result 2> /dev/null python3 e2e_test/sink/kafka/debezium.py e2e_test/sink/kafka/debezium3.result e2e_test/sink/kafka/debezium3.tmp.result if [ $? -ne 0 ]; then echo "The output for debezium sink after delete is not as expected." @@ -128,13 +128,13 @@ else fi sqllogictest -p 4566 -d dev 'e2e_test/sink/kafka/drop_sink.slt' -./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-append-only --delete > /dev/null 2>&1 -./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-upsert --delete > /dev/null 2>&1 -./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-debezium --delete > /dev/null 2>&1 +./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-append-only --delete > /dev/null 2>&1 +./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-upsert --delete > /dev/null 2>&1 +./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-debezium --delete > /dev/null 2>&1 # test different encoding echo "testing protobuf" cp src/connector/src/test_data/proto_recursive/recursive.pb ./proto-recursive -./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-append-only-protobuf --create > /dev/null 2>&1 +./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-append-only-protobuf --create > /dev/null 2>&1 sqllogictest -p 4566 -d dev 'e2e_test/sink/kafka/protobuf.slt' -./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-append-only-protobuf --delete > /dev/null 2>&1 +./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-append-only-protobuf --delete > /dev/null 2>&1 diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index 2dc02f0eada7a..ce2cc46381eba 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -57,7 +57,7 @@ node_port=50051 node_timeout=10 echo "--- starting risingwave cluster with connector node" -cargo make ci-start ci-kafka +cargo make ci-start ci-1cn-1fe ./connector-node/start-service.sh -p $node_port > .risingwave/log/connector-node.log 2>&1 & echo "waiting for connector node to start" diff --git a/e2e_test/sink/kafka/create_sink.slt b/e2e_test/sink/kafka/create_sink.slt index 25e3a59fdff3a..a1f296774f526 100644 --- a/e2e_test/sink/kafka/create_sink.slt +++ b/e2e_test/sink/kafka/create_sink.slt @@ -31,7 +31,7 @@ create connection mock with ( statement error create sink si_kafka_append_only_conn from t_kafka with ( connector = 'kafka', - properties.bootstrap.server = '127.0.0.1:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'test-rw-sink-append-only', type = 'append-only', force_append_only = 'true', @@ -42,7 +42,7 @@ create sink si_kafka_append_only_conn from t_kafka with ( statement ok create sink si_kafka_append_only_conn from t_kafka with ( connector = 'kafka', - properties.bootstrap.server = '127.0.0.1:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'test-rw-sink-append-only', type = 'append-only', force_append_only = 'true', @@ -66,7 +66,7 @@ drop connection mock; statement error sink cannot be append-only create sink si_kafka_append_only from t_kafka with ( connector = 'kafka', - properties.bootstrap.server = '127.0.0.1:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'test-rw-sink-append-only', type = 'append-only', ); @@ -74,7 +74,7 @@ create sink si_kafka_append_only from t_kafka with ( statement ok create sink si_kafka_append_only from t_kafka with ( connector = 'kafka', - properties.bootstrap.server = '127.0.0.1:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'test-rw-sink-append-only', type = 'append-only', force_append_only = 'true' @@ -83,7 +83,7 @@ create sink si_kafka_append_only from t_kafka with ( statement error primary key not defined create sink si_kafka_upsert from t_kafka with ( connector = 'kafka', - properties.bootstrap.server = '127.0.0.1:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'test-rw-sink-upsert', type = 'upsert', ); @@ -91,7 +91,7 @@ create sink si_kafka_upsert from t_kafka with ( statement ok create sink si_kafka_upsert from t_kafka with ( connector = 'kafka', - properties.bootstrap.server = '127.0.0.1:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'test-rw-sink-upsert', type = 'upsert', primary_key = 'id', @@ -100,7 +100,7 @@ create sink si_kafka_upsert from t_kafka with ( statement ok create sink si_kafka_upsert_schema from t_kafka with ( connector = 'kafka', - properties.bootstrap.server = '127.0.0.1:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'test-rw-sink-upsert-schema', primary_key = 'id', ) format upsert encode json ( @@ -110,7 +110,7 @@ create sink si_kafka_upsert_schema from t_kafka with ( statement ok create sink si_kafka_debezium from t_kafka with ( connector = 'kafka', - properties.bootstrap.server = '127.0.0.1:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'test-rw-sink-debezium', type = 'debezium', primary_key = 'id', @@ -119,7 +119,7 @@ create sink si_kafka_debezium from t_kafka with ( statement error primary key not defined create sink debezium_without_pk from t_kafka with ( connector = 'kafka', - properties.bootstrap.server = '127.0.0.1:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'test-rw-sink-debezium', type = 'debezium', ); @@ -127,7 +127,7 @@ create sink debezium_without_pk from t_kafka with ( statement ok create sink multiple_pk from t_kafka with ( connector = 'kafka', - properties.bootstrap.server = '127.0.0.1:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'test-rw-sink-debezium', type = 'debezium', primary_key = 'id,v_varchar' @@ -139,7 +139,7 @@ drop sink multiple_pk; statement error Sink primary key column not found: invalid. create sink invalid_pk_column from t_kafka with ( connector = 'kafka', - properties.bootstrap.server = '127.0.0.1:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'test-rw-sink-debezium', type = 'debezium', primary_key = 'id,invalid' diff --git a/e2e_test/sink/kafka/protobuf.slt b/e2e_test/sink/kafka/protobuf.slt index f69c4a9d07110..87ab884eddbde 100644 --- a/e2e_test/sink/kafka/protobuf.slt +++ b/e2e_test/sink/kafka/protobuf.slt @@ -2,7 +2,7 @@ statement ok create table from_kafka with ( connector = 'kafka', topic = 'test-rw-sink-append-only-protobuf', - properties.bootstrap.server = '127.0.0.1:29092') + properties.bootstrap.server = 'message_queue:29092') format plain encode protobuf ( schema.location = 'file:///risingwave/proto-recursive', message = 'recursive.AllTypes'); @@ -37,7 +37,7 @@ statement ok create sink sink0 from into_kafka with ( connector = 'kafka', topic = 'test-rw-sink-append-only-protobuf', - properties.bootstrap.server = '127.0.0.1:29092') + properties.bootstrap.server = 'message_queue:29092') format plain encode protobuf ( force_append_only = true, schema.location = 'file:///risingwave/proto-recursive', @@ -70,7 +70,7 @@ statement error failed to read file create sink sink_err from into_kafka with ( connector = 'kafka', topic = 'test-rw-sink-append-only-protobuf', - properties.bootstrap.server = '127.0.0.1:29092') + properties.bootstrap.server = 'message_queue:29092') format plain encode protobuf ( force_append_only = true, schema.location = 'file:///risingwave/proto-recursiv', @@ -80,7 +80,7 @@ statement error encode extra_column error: field not in proto create sink sink_err as select 1 as extra_column with ( connector = 'kafka', topic = 'test-rw-sink-append-only-protobuf', - properties.bootstrap.server = '127.0.0.1:29092') + properties.bootstrap.server = 'message_queue:29092') format plain encode protobuf ( force_append_only = true, schema.location = 'file:///risingwave/proto-recursive', @@ -90,7 +90,7 @@ statement error s3 URL not supported yet create sink sink_err from into_kafka with ( connector = 'kafka', topic = 'test-rw-sink-append-only-protobuf', - properties.bootstrap.server = '127.0.0.1:29092') + properties.bootstrap.server = 'message_queue:29092') format plain encode protobuf ( force_append_only = true, schema.location = 's3:///risingwave/proto-recursive', diff --git a/risedev.yml b/risedev.yml index a5ba8a7b43f97..135a33f602a6a 100644 --- a/risedev.yml +++ b/risedev.yml @@ -685,40 +685,6 @@ profile: - use: pubsub persist-data: true - ci-kafka: - config-path: src/config/ci.toml - steps: - - use: minio - - use: etcd - unsafe-no-fsync: true - - use: meta-node - - use: compute-node - enable-tiered-cache: true - - use: frontend - - use: compactor - - use: zookeeper - persist-data: true - - use: kafka - persist-data: true - - ci-kafka-plus-pubsub: - config-path: src/config/ci.toml - steps: - - use: minio - - use: etcd - unsafe-no-fsync: true - - use: meta-node - - use: compute-node - enable-tiered-cache: true - - use: frontend - - use: compactor - - use: zookeeper - persist-data: true - - use: kafka - persist-data: true - - use: pubsub - persist-data: true - ci-redis: config-path: src/config/ci.toml steps: From 272eff41b64bb4f2743084ee367c33d88edc5ed5 Mon Sep 17 00:00:00 2001 From: Tao Wu Date: Wed, 25 Oct 2023 11:07:52 +0800 Subject: [PATCH 13/48] ci: create a doc issue when a user-facing issue is done (#12991) --- .../auto-create-doc-issue-by-issue.yml | 31 +++++++++++++++++++ ...pr.yml => auto-create-doc-issue-by-pr.yml} | 0 2 files changed, 31 insertions(+) create mode 100644 .github/workflows/auto-create-doc-issue-by-issue.yml rename .github/workflows/{auto-create-docs-pr.yml => auto-create-doc-issue-by-pr.yml} (100%) diff --git a/.github/workflows/auto-create-doc-issue-by-issue.yml b/.github/workflows/auto-create-doc-issue-by-issue.yml new file mode 100644 index 0000000000000..0c8d78062977a --- /dev/null +++ b/.github/workflows/auto-create-doc-issue-by-issue.yml @@ -0,0 +1,31 @@ +name: Issue Documentation Checker + +on: + issues: + types: + - closed + - labeled + +jobs: + create-issue: + runs-on: ubuntu-latest + + steps: + - uses: actions/checkout@v3 + - name: Log the event payload + run: echo "${{ toJSON(github.event) }}" + - name: Check if issue is done and labeled 'user-facing-changes' + uses: dacbd/create-issue-action@main + if: ${{ github.event.action == 'closed' && contains(github.event.issue.labels.*.name, 'user-facing-changes') }} + with: + token: ${{ secrets.ACCESS_TOKEN }} + owner: risingwavelabs + repo: risingwave-docs + title: | + Document: ${{ github.event.issue.title }} + body: | + ## Context + Source Issue URL: ${{ github.event.issue.html_url }} + Created At: ${{ github.event.issue.created_at }} + Created By: ${{ github.event.issue.user.login }} + Closed At: ${{ github.event.issue.closed_at }} diff --git a/.github/workflows/auto-create-docs-pr.yml b/.github/workflows/auto-create-doc-issue-by-pr.yml similarity index 100% rename from .github/workflows/auto-create-docs-pr.yml rename to .github/workflows/auto-create-doc-issue-by-pr.yml From 8fdcfb8ded255b5e59d8ab59c86dea8535416a30 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 25 Oct 2023 11:21:44 +0800 Subject: [PATCH 14/48] feat(sink): monitor error in remote sink response stream (#13028) --- Cargo.lock | 1 + src/common/src/util/future_utils.rs | 37 +++- src/common/src/util/mod.rs | 4 +- src/connector/src/sink/remote.rs | 276 +++++++++++++++++++++++----- src/jni_core/Cargo.toml | 1 + src/jni_core/src/lib.rs | 7 +- src/rpc_client/src/lib.rs | 31 +--- 7 files changed, 285 insertions(+), 72 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 32cb936bdaae3..99a5a675c0342 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7621,6 +7621,7 @@ dependencies = [ name = "risingwave_jni_core" version = "0.1.0" dependencies = [ + "anyhow", "bytes", "cfg-or-panic", "futures", diff --git a/src/common/src/util/future_utils.rs b/src/common/src/util/future_utils.rs index 75c38488457ac..20844d8cd15d4 100644 --- a/src/common/src/util/future_utils.rs +++ b/src/common/src/util/future_utils.rs @@ -13,9 +13,11 @@ // limitations under the License. use std::future::pending; +use std::pin::{pin, Pin}; -use futures::future::Either; -use futures::{Future, FutureExt, Stream}; +use futures::future::{select, Either}; +use futures::stream::Peekable; +use futures::{Future, FutureExt, Stream, StreamExt}; /// Convert a list of streams into a [`Stream`] of results from the streams. pub fn select_all( @@ -43,3 +45,34 @@ pub fn drop_either_future( Either::Right((right, _)) => Either::Right(right), } } + +/// Await on a future while monitoring on a peekable stream that may return error. +/// The peekable stream is polled at a higher priority than the future. +/// +/// When the peekable stream returns with a error and end of stream, the future will +/// return the error immediately. Otherwise, it will keep polling the given future. +/// +/// Return: +/// - Ok(output) as the output of the given future. +/// - Err(None) to indicate that the stream has reached the end. +/// - Err(e) to indicate that the stream returns an error. +pub async fn await_future_with_monitor_error_stream( + peek_stream: &mut Peekable> + Unpin>, + future: F, +) -> Result> { + // Poll the response stream to early see the error + match select(pin!(Pin::new(&mut *peek_stream).peek()), pin!(future)).await { + Either::Left((response_result, send_future)) => match response_result { + None => Err(None), + Some(Err(_)) => { + let err = match peek_stream.next().now_or_never() { + Some(Some(Err(err))) => err, + _ => unreachable!("peek has output, peek output not None, have check err"), + }; + Err(Some(err)) + } + Some(Ok(_)) => Ok(send_future.await), + }, + Either::Right((output, _)) => Ok(output), + } +} diff --git a/src/common/src/util/mod.rs b/src/common/src/util/mod.rs index f4140b558faa7..e1f85263e1415 100644 --- a/src/common/src/util/mod.rs +++ b/src/common/src/util/mod.rs @@ -45,7 +45,9 @@ pub mod tracing; pub mod value_encoding; pub mod worker_util; -pub use future_utils::{drop_either_future, pending_on_none, select_all}; +pub use future_utils::{ + await_future_with_monitor_error_stream, drop_either_future, pending_on_none, select_all, +}; #[macro_use] pub mod match_util; diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index ad182e734a33a..310213262b2ad 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -13,17 +13,23 @@ // limitations under the License. use std::collections::HashMap; +use std::fmt::Formatter; +use std::future::Future; use std::marker::PhantomData; use std::ops::Deref; +use std::time::Instant; use anyhow::anyhow; use async_trait::async_trait; +use futures::stream::Peekable; +use futures::{StreamExt, TryFutureExt, TryStreamExt}; use itertools::Itertools; use jni::objects::{JByteArray, JValue, JValueOwned}; use prost::Message; use risingwave_common::array::StreamChunk; use risingwave_common::error::anyhow_error; use risingwave_common::types::DataType; +use risingwave_common::util::await_future_with_monitor_error_stream; use risingwave_jni_core::jvm_runtime::JVM; use risingwave_pb::connector_service::sink_coordinator_stream_request::{ CommitMetadata, StartCoordinator, @@ -43,15 +49,17 @@ use risingwave_pb::connector_service::{ }; use tokio::sync::mpsc; use tokio::sync::mpsc::{Receiver, Sender}; +use tokio_stream::wrappers::ReceiverStream; use tracing::warn; use super::encoder::{JsonEncoder, RowEncoder}; use crate::sink::coordinate::CoordinatedSinkWriter; use crate::sink::encoder::TimestampHandlingMode; +use crate::sink::log_store::{LogReader, LogStoreReadItem, TruncateOffset}; use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkCommitCoordinator, SinkError, SinkMetrics, - SinkParam, SinkWriterParam, + DummySinkCommitCoordinator, LogSinker, Result, Sink, SinkCommitCoordinator, SinkError, + SinkMetrics, SinkParam, SinkWriterParam, }; use crate::ConnectorParams; @@ -101,18 +109,12 @@ impl TryFrom for RemoteSink { impl Sink for RemoteSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf>; + type LogSinker = RemoteLogSinker; const SINK_NAME: &'static str = R::SINK_NAME; async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { - Ok(RemoteSinkWriter::new( - self.param.clone(), - writer_param.connector_params, - writer_param.sink_metrics.clone(), - ) - .await? - .into_log_sinker(writer_param.sink_metrics)) + RemoteLogSinker::new(self.param.clone(), writer_param).await } async fn validate(&self) -> Result<()> { @@ -192,6 +194,139 @@ impl Sink for RemoteSink { } } +pub struct RemoteLogSinker { + writer: RemoteSinkWriter, + sink_metrics: SinkMetrics, +} + +impl RemoteLogSinker { + async fn new(sink_param: SinkParam, writer_param: SinkWriterParam) -> Result { + let writer = RemoteSinkWriter::new( + sink_param, + writer_param.connector_params, + writer_param.sink_metrics.clone(), + ) + .await?; + let sink_metrics = writer_param.sink_metrics; + Ok(RemoteLogSinker { + writer, + sink_metrics, + }) + } +} + +/// Await the given future while monitoring on error of the receiver stream. +async fn await_future_with_monitor_receiver_err>>( + receiver: &mut SinkWriterStreamJniReceiver, + future: F, +) -> Result { + match await_future_with_monitor_error_stream(&mut receiver.response_stream, future).await { + Ok(result) => result, + Err(None) => Err(SinkError::Remote(anyhow!("end of remote receiver stream"))), + Err(Some(err)) => Err(SinkError::Internal(err)), + } +} + +impl LogSinker for RemoteLogSinker { + async fn consume_log_and_sink(self, mut log_reader: impl LogReader) -> Result<()> { + // Note: this is a total copy of the implementation of LogSinkerOf, + // except that we monitor the future of `log_reader.next_item` with await_future_with_monitor_receiver_err + // to monitor the error in the response stream. + + let mut sink_writer = self.writer; + let sink_metrics = self.sink_metrics; + #[derive(Debug)] + enum LogConsumerState { + /// Mark that the log consumer is not initialized yet + Uninitialized, + + /// Mark that a new epoch has begun. + EpochBegun { curr_epoch: u64 }, + + /// Mark that the consumer has just received a barrier + BarrierReceived { prev_epoch: u64 }, + } + + let mut state = LogConsumerState::Uninitialized; + + log_reader.init().await?; + + loop { + let (epoch, item): (u64, LogStoreReadItem) = await_future_with_monitor_receiver_err( + &mut sink_writer.stream_handle.response_rx, + log_reader.next_item().map_err(SinkError::Internal), + ) + .await?; + if let LogStoreReadItem::UpdateVnodeBitmap(_) = &item { + match &state { + LogConsumerState::BarrierReceived { .. } => {} + _ => unreachable!( + "update vnode bitmap can be accepted only right after \ + barrier, but current state is {:?}", + state + ), + } + } + // begin_epoch when not previously began + state = match state { + LogConsumerState::Uninitialized => { + sink_writer.begin_epoch(epoch).await?; + LogConsumerState::EpochBegun { curr_epoch: epoch } + } + LogConsumerState::EpochBegun { curr_epoch } => { + assert!( + epoch >= curr_epoch, + "new epoch {} should not be below the current epoch {}", + epoch, + curr_epoch + ); + LogConsumerState::EpochBegun { curr_epoch: epoch } + } + LogConsumerState::BarrierReceived { prev_epoch } => { + assert!( + epoch > prev_epoch, + "new epoch {} should be greater than prev epoch {}", + epoch, + prev_epoch + ); + sink_writer.begin_epoch(epoch).await?; + LogConsumerState::EpochBegun { curr_epoch: epoch } + } + }; + match item { + LogStoreReadItem::StreamChunk { chunk, .. } => { + if let Err(e) = sink_writer.write_batch(chunk).await { + sink_writer.abort().await?; + return Err(e); + } + } + LogStoreReadItem::Barrier { is_checkpoint } => { + let prev_epoch = match state { + LogConsumerState::EpochBegun { curr_epoch } => curr_epoch, + _ => unreachable!("epoch must have begun before handling barrier"), + }; + if is_checkpoint { + let start_time = Instant::now(); + sink_writer.barrier(true).await?; + sink_metrics + .sink_commit_duration_metrics + .observe(start_time.elapsed().as_millis() as f64); + log_reader + .truncate(TruncateOffset::Barrier { epoch }) + .await?; + } else { + sink_writer.barrier(false).await?; + } + state = LogConsumerState::BarrierReceived { prev_epoch } + } + LogStoreReadItem::UpdateVnodeBitmap(vnode_bitmap) => { + sink_writer.update_vnode_bitmap(vnode_bitmap).await?; + } + } + } + } +} + #[derive(Debug)] pub struct CoordinatedRemoteSink(pub RemoteSink); @@ -286,14 +421,11 @@ impl SinkCoordinatorStreamJniHandle { } } -const DEFAULT_CHANNEL_SIZE: usize = 16; -#[derive(Debug)] -pub struct SinkWriterStreamJniHandle { +struct SinkWriterStreamJniSender { request_tx: Sender, - response_rx: Receiver, } -impl SinkWriterStreamJniHandle { +impl SinkWriterStreamJniSender { pub async fn start_epoch(&mut self, epoch: u64) -> Result<()> { self.request_tx .send(SinkWriterStreamRequest { @@ -316,33 +448,29 @@ impl SinkWriterStreamJniHandle { .map_err(|err| SinkError::Internal(err.into())) } - pub async fn barrier(&mut self, epoch: u64) -> Result<()> { + pub async fn barrier(&mut self, epoch: u64, is_checkpoint: bool) -> Result<()> { self.request_tx .send(SinkWriterStreamRequest { request: Some(SinkRequest::Barrier(Barrier { epoch, - is_checkpoint: false, + is_checkpoint, })), }) .await .map_err(|err| SinkError::Internal(err.into())) } +} - pub async fn commit(&mut self, epoch: u64) -> Result { - self.request_tx - .send(SinkWriterStreamRequest { - request: Some(SinkRequest::Barrier(Barrier { - epoch, - is_checkpoint: true, - })), - }) - .await - .map_err(|err| SinkError::Internal(err.into()))?; +struct SinkWriterStreamJniReceiver { + response_stream: Peekable>>, +} - match self.response_rx.recv().await { - Some(SinkWriterStreamResponse { +impl SinkWriterStreamJniReceiver { + async fn next_commit_response(&mut self) -> Result { + match self.response_stream.try_next().await { + Ok(Some(SinkWriterStreamResponse { response: Some(sink_writer_stream_response::Response::Commit(rsp)), - }) => Ok(rsp), + })) => Ok(rsp), msg => Err(SinkError::Internal(anyhow!( "should get Sync response but get {:?}", msg @@ -351,6 +479,53 @@ impl SinkWriterStreamJniHandle { } } +const DEFAULT_CHANNEL_SIZE: usize = 16; +struct SinkWriterStreamJniHandle { + request_tx: SinkWriterStreamJniSender, + response_rx: SinkWriterStreamJniReceiver, +} + +impl std::fmt::Debug for SinkWriterStreamJniHandle { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("SinkWriterStreamJniHandle").finish() + } +} + +impl SinkWriterStreamJniHandle { + async fn start_epoch(&mut self, epoch: u64) -> Result<()> { + await_future_with_monitor_receiver_err( + &mut self.response_rx, + self.request_tx.start_epoch(epoch), + ) + .await + } + + async fn write_batch(&mut self, epoch: u64, batch_id: u64, payload: Payload) -> Result<()> { + await_future_with_monitor_receiver_err( + &mut self.response_rx, + self.request_tx.write_batch(epoch, batch_id, payload), + ) + .await + } + + async fn barrier(&mut self, epoch: u64) -> Result<()> { + await_future_with_monitor_receiver_err( + &mut self.response_rx, + self.request_tx.barrier(epoch, false), + ) + .await + } + + async fn commit(&mut self, epoch: u64) -> Result { + await_future_with_monitor_receiver_err( + &mut self.response_rx, + self.request_tx.barrier(epoch, true), + ) + .await?; + self.response_rx.next_commit_response().await + } +} + pub type RemoteSinkWriter = RemoteSinkWriterInner<(), R>; pub type CoordinatedRemoteSinkWriter = RemoteSinkWriterInner, R>; @@ -374,10 +549,7 @@ impl RemoteSinkWriterInner { let (request_tx, request_rx) = mpsc::channel(DEFAULT_CHANNEL_SIZE); let (response_tx, response_rx) = mpsc::channel(DEFAULT_CHANNEL_SIZE); - let mut stream_handle = SinkWriterStreamJniHandle { - request_tx, - response_rx, - }; + let mut response_stream = ReceiverStream::new(response_rx).peekable(); std::thread::spawn(move || { let mut env = JVM.get_or_init().unwrap().attach_current_thread().unwrap(); @@ -388,7 +560,10 @@ impl RemoteSinkWriterInner { "(JJ)V", &[ JValue::from(&request_rx as *const Receiver as i64), - JValue::from(&response_tx as *const Sender as i64), + JValue::from( + &response_tx as *const Sender> + as i64, + ), ], ); @@ -410,8 +585,7 @@ impl RemoteSinkWriterInner { }; // First request - stream_handle - .request_tx + request_tx .send(sink_writer_stream_request) .await .map_err(|err| { @@ -423,17 +597,18 @@ impl RemoteSinkWriterInner { })?; // First response - match stream_handle.response_rx.recv().await { - Some(SinkWriterStreamResponse { + match response_stream.try_next().await { + Ok(Some(SinkWriterStreamResponse { response: Some(sink_writer_stream_response::Response::Start(_)), - }) => {} - msg => { + })) => {} + Ok(msg) => { return Err(SinkError::Internal(anyhow!( "should get start response for connector `{}` but get {:?}", R::SINK_NAME, msg ))); } + Err(e) => return Err(SinkError::Internal(e)), }; tracing::trace!( @@ -444,6 +619,11 @@ impl RemoteSinkWriterInner { let schema = param.schema(); + let stream_handle = SinkWriterStreamJniHandle { + request_tx: SinkWriterStreamJniSender { request_tx }, + response_rx: SinkWriterStreamJniReceiver { response_stream }, + }; + Ok(Self { properties: param.properties, epoch: None, @@ -458,7 +638,7 @@ impl RemoteSinkWriterInner { #[cfg(test)] fn for_test( - response_receiver: Receiver, + response_receiver: Receiver>, request_sender: Sender, ) -> RemoteSinkWriter { use risingwave_common::catalog::{Field, Schema}; @@ -480,8 +660,12 @@ impl RemoteSinkWriterInner { ]); let stream_handle = SinkWriterStreamJniHandle { - request_tx: request_sender, - response_rx: response_receiver, + request_tx: SinkWriterStreamJniSender { + request_tx: request_sender, + }, + response_rx: SinkWriterStreamJniReceiver { + response_stream: ReceiverStream::new(response_receiver).peekable(), + }, }; RemoteSinkWriter { @@ -828,12 +1012,12 @@ mod test { // test commit response_sender - .send(SinkWriterStreamResponse { + .send(Ok(SinkWriterStreamResponse { response: Some(Response::Commit(CommitResponse { epoch: 2022, metadata: None, })), - }) + })) .await .expect("test failed: failed to sync epoch"); sink.barrier(true).await.unwrap(); diff --git a/src/jni_core/Cargo.toml b/src/jni_core/Cargo.toml index 69c11a7f21e24..77cafd155000d 100644 --- a/src/jni_core/Cargo.toml +++ b/src/jni_core/Cargo.toml @@ -10,6 +10,7 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] +anyhow = "1" bytes = "1" cfg-or-panic = "0.2" futures = { version = "0.3", default-features = false, features = ["alloc"] } diff --git a/src/jni_core/src/lib.rs b/src/jni_core/src/lib.rs index 29bbf76929b45..4815cd7368370 100644 --- a/src/jni_core/src/lib.rs +++ b/src/jni_core/src/lib.rs @@ -902,14 +902,17 @@ pub extern "system" fn Java_com_risingwave_java_binding_Binding_sendSinkWriterRe 'a, >( env: EnvParam<'a>, - channel: Pointer<'a, Sender>, + channel: Pointer<'a, Sender>>, msg: JByteArray<'a>, ) -> jboolean { execute_and_catch(env, move |env| { let sink_writer_stream_response: SinkWriterStreamResponse = Message::decode(to_guarded_slice(&msg, env)?.deref())?; - match channel.as_ref().blocking_send(sink_writer_stream_response) { + match channel + .as_ref() + .blocking_send(Ok(sink_writer_stream_response)) + { Ok(_) => Ok(JNI_TRUE), Err(e) => { tracing::info!("send error. {:?}", e); diff --git a/src/rpc_client/src/lib.rs b/src/rpc_client/src/lib.rs index be871521d3bc4..6afa67ef88efe 100644 --- a/src/rpc_client/src/lib.rs +++ b/src/rpc_client/src/lib.rs @@ -30,12 +30,11 @@ use std::any::type_name; use std::fmt::{Debug, Formatter}; use std::future::Future; use std::iter::repeat; -use std::pin::pin; use std::sync::Arc; use anyhow::anyhow; use async_trait::async_trait; -use futures::future::{select, try_join_all, Either}; +use futures::future::try_join_all; use futures::stream::{BoxStream, Peekable}; use futures::{Stream, StreamExt}; use moka::future::Cache; @@ -58,13 +57,12 @@ mod sink_coordinate_client; mod stream_client; mod tracing; -use std::pin::Pin; - pub use compactor_client::{CompactorClient, GrpcCompactorProxyClient}; pub use compute_client::{ComputeClient, ComputeClientPool, ComputeClientPoolRef}; pub use connector_client::{ConnectorClient, SinkCoordinatorStreamHandle, SinkWriterStreamHandle}; pub use hummock_meta_client::{CompactionEventItem, HummockMetaClient}; pub use meta_client::{MetaClient, SinkCoordinationRpcClient}; +use risingwave_common::util::await_future_with_monitor_error_stream; pub use sink_coordinate_client::CoordinatorStreamHandle; pub use stream_client::{StreamClient, StreamClientPool, StreamClientPoolRef}; @@ -240,25 +238,16 @@ impl BidiStreamHandle { } pub async fn send_request(&mut self, request: REQ) -> Result<()> { - // Poll the response stream to early see the error - let send_request_result = match select( - pin!(self.request_sender.send(request)), - pin!(Pin::new(&mut self.response_stream).peek()), + match await_future_with_monitor_error_stream( + &mut self.response_stream, + self.request_sender.send(request), ) .await { - Either::Left((result, _)) => result, - Either::Right((response_result, send_future)) => match response_result { - None => { - return Err(anyhow!("end of response stream").into()); - } - Some(Err(e)) => { - return Err(e.clone().into()); - } - Some(Ok(_)) => send_future.await, - }, - }; - send_request_result - .map_err(|_| anyhow!("unable to send request {}", type_name::()).into()) + Ok(send_result) => send_result + .map_err(|_| anyhow!("unable to send request {}", type_name::()).into()), + Err(None) => Err(anyhow!("end of response stream").into()), + Err(Some(e)) => Err(e.into()), + } } } From 8221d3a16cec7122504e49423ae74a400d902ff1 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 25 Oct 2023 11:55:34 +0800 Subject: [PATCH 15/48] feat: add `WAIT` command (#13027) --- proto/ddl_service.proto | 5 +++++ src/frontend/src/handler/mod.rs | 2 ++ src/frontend/src/handler/wait.rs | 31 +++++++++++++++++++++++++++++ src/frontend/src/meta_client.rs | 6 ++++++ src/frontend/src/test_utils.rs | 4 ++++ src/meta/service/src/ddl_service.rs | 5 +++++ src/meta/src/rpc/ddl_controller.rs | 16 +++++++++++++++ src/rpc_client/src/meta_client.rs | 7 +++++++ src/sqlparser/src/ast/mod.rs | 6 ++++++ src/sqlparser/src/keywords.rs | 1 + src/sqlparser/src/parser.rs | 1 + src/utils/pgwire/src/pg_response.rs | 2 ++ 12 files changed, 86 insertions(+) create mode 100644 src/frontend/src/handler/wait.rs diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 27c9f2ee82f83..1efc933a7d033 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -314,6 +314,10 @@ message GetTablesResponse { map tables = 1; } +message WaitRequest {} + +message WaitResponse {} + service DdlService { rpc CreateDatabase(CreateDatabaseRequest) returns (CreateDatabaseResponse); rpc DropDatabase(DropDatabaseRequest) returns (DropDatabaseResponse); @@ -343,4 +347,5 @@ service DdlService { rpc ListConnections(ListConnectionsRequest) returns (ListConnectionsResponse); rpc DropConnection(DropConnectionRequest) returns (DropConnectionResponse); rpc GetTables(GetTablesRequest) returns (GetTablesResponse); + rpc Wait(WaitRequest) returns (WaitResponse); } diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 149f39bead330..174ed23e03ec5 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -73,6 +73,7 @@ mod show; mod transaction; pub mod util; pub mod variable; +mod wait; /// The [`PgResponseBuilder`] used by RisingWave. pub type RwPgResponseBuilder = PgResponseBuilder; @@ -419,6 +420,7 @@ pub async fn handle( } } Statement::Flush => flush::handle_flush(handler_args).await, + Statement::Wait => wait::handle_wait(handler_args).await, Statement::SetVariable { local: _, variable, diff --git a/src/frontend/src/handler/wait.rs b/src/frontend/src/handler/wait.rs new file mode 100644 index 0000000000000..83f2784ec8c17 --- /dev/null +++ b/src/frontend/src/handler/wait.rs @@ -0,0 +1,31 @@ +// 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 pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::error::Result; + +use super::RwPgResponse; +use crate::handler::HandlerArgs; +use crate::session::SessionImpl; + +pub(super) async fn handle_wait(handler_args: HandlerArgs) -> Result { + do_wait(&handler_args.session).await?; + Ok(PgResponse::empty_result(StatementType::WAIT)) +} + +pub(crate) async fn do_wait(session: &SessionImpl) -> Result<()> { + let client = session.env().meta_client(); + client.wait().await?; + Ok(()) +} diff --git a/src/frontend/src/meta_client.rs b/src/frontend/src/meta_client.rs index ae90c2e345f9f..d37c5dec127f1 100644 --- a/src/frontend/src/meta_client.rs +++ b/src/frontend/src/meta_client.rs @@ -43,6 +43,8 @@ pub trait FrontendMetaClient: Send + Sync { async fn flush(&self, checkpoint: bool) -> Result; + async fn wait(&self) -> Result<()>; + async fn cancel_creating_jobs(&self, jobs: PbJobs) -> Result>; async fn list_table_fragments( @@ -111,6 +113,10 @@ impl FrontendMetaClient for FrontendMetaClientImpl { self.0.flush(checkpoint).await } + async fn wait(&self) -> Result<()> { + self.0.wait().await + } + async fn cancel_creating_jobs(&self, infos: PbJobs) -> Result> { self.0.cancel_creating_jobs(infos).await } diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 20eb252fc5053..cf915ae35713d 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -773,6 +773,10 @@ impl FrontendMetaClient for MockFrontendMetaClient { }) } + async fn wait(&self) -> RpcResult<()> { + Ok(()) + } + async fn cancel_creating_jobs(&self, _infos: PbJobs) -> RpcResult> { Ok(vec![]) } diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 2fa5f50e15666..061ff93589163 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -732,6 +732,11 @@ impl DdlService for DdlServiceImpl { } Ok(Response::new(GetTablesResponse { tables })) } + + async fn wait(&self, _request: Request) -> Result, Status> { + self.ddl_controller.wait().await; + Ok(Response::new(WaitResponse {})) + } } impl DdlServiceImpl { diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 04b9729c5a5b8..36615bd93b757 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -15,6 +15,7 @@ use std::cmp::Ordering; use std::num::NonZeroUsize; use std::sync::Arc; +use std::time::Duration; use itertools::Itertools; use risingwave_common::config::DefaultParallelism; @@ -29,6 +30,7 @@ use risingwave_pb::ddl_service::alter_relation_name_request::Relation; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::stream_plan::StreamFragmentGraph as StreamFragmentGraphProto; use tokio::sync::Semaphore; +use tokio::time::sleep; use tracing::log::warn; use tracing::Instrument; @@ -1094,4 +1096,18 @@ impl DdlController { } } } + + pub async fn wait(&self) { + for _ in 0..30 * 60 { + if self + .catalog_manager + .list_creating_background_mvs() + .await + .is_empty() + { + break; + } + sleep(Duration::from_secs(1)).await; + } + } } diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 827860d1af7b3..95b746ea33e6c 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -698,6 +698,12 @@ impl MetaClient { Ok(resp.snapshot.unwrap()) } + pub async fn wait(&self) -> Result<()> { + let request = WaitRequest {}; + self.inner.wait(request).await?; + Ok(()) + } + pub async fn cancel_creating_jobs(&self, jobs: PbJobs) -> Result> { let request = CancelCreatingJobsRequest { jobs: Some(jobs) }; let resp = self.inner.cancel_creating_jobs(request).await?; @@ -1719,6 +1725,7 @@ macro_rules! for_all_meta_rpc { ,{ ddl_client, list_connections, ListConnectionsRequest, ListConnectionsResponse } ,{ ddl_client, drop_connection, DropConnectionRequest, DropConnectionResponse } ,{ ddl_client, get_tables, GetTablesRequest, GetTablesResponse } + ,{ ddl_client, wait, WaitRequest, WaitResponse } ,{ hummock_client, unpin_version_before, UnpinVersionBeforeRequest, UnpinVersionBeforeResponse } ,{ hummock_client, get_current_version, GetCurrentVersionRequest, GetCurrentVersionResponse } ,{ hummock_client, replay_version_delta, ReplayVersionDeltaRequest, ReplayVersionDeltaResponse } diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index ecae5a9663a88..5d802bae99cdc 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -1294,6 +1294,9 @@ pub enum Statement { /// /// Note: RisingWave specific statement. Flush, + /// WAIT for ALL running stream jobs to finish. + /// It will block the current session the condition is met. + Wait, } impl fmt::Display for Statement { @@ -1787,6 +1790,9 @@ impl fmt::Display for Statement { Statement::Flush => { write!(f, "FLUSH") } + Statement::Wait => { + write!(f, "WAIT") + } Statement::Begin { modes } => { write!(f, "BEGIN")?; if !modes.is_empty() { diff --git a/src/sqlparser/src/keywords.rs b/src/sqlparser/src/keywords.rs index 5c2fedb0ea547..4188f06f76ae3 100644 --- a/src/sqlparser/src/keywords.rs +++ b/src/sqlparser/src/keywords.rs @@ -540,6 +540,7 @@ define_keywords!( VIEWS, VIRTUAL, VOLATILE, + WAIT, WATERMARK, WHEN, WHENEVER, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index ee054f7d17031..5cc094a204268 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -259,6 +259,7 @@ impl Parser { Keyword::PREPARE => Ok(self.parse_prepare()?), Keyword::COMMENT => Ok(self.parse_comment()?), Keyword::FLUSH => Ok(Statement::Flush), + Keyword::WAIT => Ok(Statement::Wait), _ => self.expected( "an SQL statement", Token::Word(w).with_location(token.location), diff --git a/src/utils/pgwire/src/pg_response.rs b/src/utils/pgwire/src/pg_response.rs index 29ea77f83b71b..eeec929732f50 100644 --- a/src/utils/pgwire/src/pg_response.rs +++ b/src/utils/pgwire/src/pg_response.rs @@ -92,6 +92,7 @@ pub enum StatementType { ROLLBACK, SET_TRANSACTION, CANCEL_COMMAND, + WAIT, } impl std::fmt::Display for StatementType { @@ -278,6 +279,7 @@ impl StatementType { }, Statement::Explain { .. } => Ok(StatementType::EXPLAIN), Statement::Flush => Ok(StatementType::FLUSH), + Statement::Wait => Ok(StatementType::WAIT), _ => Err("unsupported statement type".to_string()), } } From 6495d90ac0e7077a7b57490cc618cce3399b3f38 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 25 Oct 2023 12:50:17 +0800 Subject: [PATCH 16/48] feat(sink): async truncate for kinesis, pulsar, nats, redis and clickhouse sink (#12930) --- src/connector/src/sink/blackhole.rs | 3 + src/connector/src/sink/clickhouse.rs | 34 +++---- src/connector/src/sink/kafka.rs | 116 ++++++++++-------------- src/connector/src/sink/kinesis.rs | 40 ++++---- src/connector/src/sink/mod.rs | 9 +- src/connector/src/sink/nats.rs | 35 +++---- src/connector/src/sink/pulsar.rs | 131 ++++++++++++++------------- src/connector/src/sink/redis.rs | 43 +++++---- src/connector/src/sink/remote.rs | 1 + src/connector/src/sink/writer.rs | 101 +++++++++++++++++---- 10 files changed, 284 insertions(+), 229 deletions(-) diff --git a/src/connector/src/sink/blackhole.rs b/src/connector/src/sink/blackhole.rs index 1f1ace3b0d104..60b0506604c97 100644 --- a/src/connector/src/sink/blackhole.rs +++ b/src/connector/src/sink/blackhole.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use async_trait::async_trait; + use crate::sink::log_store::{LogReader, LogStoreReadItem, TruncateOffset}; use crate::sink::{ DummySinkCommitCoordinator, LogSinker, Result, Sink, SinkError, SinkParam, SinkWriterParam, @@ -45,6 +47,7 @@ impl Sink for BlackHoleSink { } } +#[async_trait] impl LogSinker for BlackHoleSink { async fn consume_log_and_sink(self, mut log_reader: impl LogReader) -> Result<()> { log_reader.init().await?; diff --git a/src/connector/src/sink/clickhouse.rs b/src/connector/src/sink/clickhouse.rs index 2bddf8026216f..f4fdf9b761f38 100644 --- a/src/connector/src/sink/clickhouse.rs +++ b/src/connector/src/sink/clickhouse.rs @@ -29,7 +29,11 @@ use serde_derive::Deserialize; use serde_with::serde_as; use super::{DummySinkCommitCoordinator, SinkWriterParam}; -use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; +use crate::sink::catalog::desc::SinkDesc; +use crate::sink::log_store::DeliveryFutureManagerAddFuture; +use crate::sink::writer::{ + AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, +}; use crate::sink::{ Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; @@ -243,10 +247,14 @@ impl ClickHouseSink { } impl Sink for ClickHouseSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; + type LogSinker = AsyncTruncateLogSinkerOf; const SINK_NAME: &'static str = CLICKHOUSE_SINK; + fn default_sink_decouple(desc: &SinkDesc) -> bool { + desc.sink_type.is_append_only() + } + async fn validate(&self) -> Result<()> { // For upsert clickhouse sink, the primary key must be defined. if !self.is_append_only && self.pk_indices.is_empty() { @@ -277,7 +285,7 @@ impl Sink for ClickHouseSink { Ok(()) } - async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + async fn new_log_sinker(&self, _writer_param: SinkWriterParam) -> Result { Ok(ClickHouseSinkWriter::new( self.config.clone(), self.schema.clone(), @@ -285,7 +293,7 @@ impl Sink for ClickHouseSink { self.is_append_only, ) .await? - .into_log_sinker(writer_param.sink_metrics)) + .into_log_sinker(usize::MAX)) } } pub struct ClickHouseSinkWriter { @@ -496,24 +504,18 @@ impl ClickHouseSinkWriter { } } -#[async_trait::async_trait] -impl SinkWriter for ClickHouseSinkWriter { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { +impl AsyncTruncateSinkWriter for ClickHouseSinkWriter { + async fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> Result<()> { if self.is_append_only { self.append_only(chunk).await } else { self.upsert(chunk).await } } - - async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { - // clickhouse no transactional guarantees, so we do nothing here. - Ok(()) - } - - async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { - Ok(()) - } } #[derive(ClickHouseRow, Deserialize, Clone)] diff --git a/src/connector/src/sink/kafka.rs b/src/connector/src/sink/kafka.rs index a204a8d121706..f77b2b0a88c36 100644 --- a/src/connector/src/sink/kafka.rs +++ b/src/connector/src/sink/kafka.rs @@ -14,20 +14,18 @@ use std::collections::HashMap; use std::fmt::Debug; -use std::pin::pin; use std::sync::Arc; use std::time::Duration; use anyhow::anyhow; -use futures::future::{select, Either}; use futures::{Future, FutureExt, TryFuture}; use rdkafka::error::KafkaError; use rdkafka::message::ToBytes; use rdkafka::producer::{DeliveryFuture, FutureProducer, FutureRecord}; use rdkafka::types::RDKafkaErrorCode; use rdkafka::ClientConfig; +use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; -use risingwave_common::util::drop_either_future; use serde_derive::{Deserialize, Serialize}; use serde_with::{serde_as, DisplayFromStr}; use strum_macros::{Display, EnumString}; @@ -37,11 +35,11 @@ use super::{Sink, SinkError, SinkParam}; use crate::common::KafkaCommon; use crate::sink::catalog::desc::SinkDesc; use crate::sink::formatter::SinkFormatterImpl; -use crate::sink::log_store::{ - DeliveryFutureManager, DeliveryFutureManagerAddFuture, LogReader, LogStoreReadItem, +use crate::sink::log_store::DeliveryFutureManagerAddFuture; +use crate::sink::writer::{ + AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, FormattedSink, }; -use crate::sink::writer::FormattedSink; -use crate::sink::{DummySinkCommitCoordinator, LogSinker, Result, SinkWriterParam}; +use crate::sink::{DummySinkCommitCoordinator, Result, SinkWriterParam}; use crate::source::kafka::{KafkaProperties, KafkaSplitEnumerator, PrivateLinkProducerContext}; use crate::source::{SourceEnumeratorContext, SplitEnumerator}; use crate::{ @@ -299,7 +297,7 @@ impl TryFrom for KafkaSink { impl Sink for KafkaSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = KafkaLogSinker; + type LogSinker = AsyncTruncateLogSinkerOf; const SINK_NAME: &'static str = KAFKA_SINK; @@ -316,7 +314,18 @@ impl Sink for KafkaSink { self.sink_from_name.clone(), ) .await?; - KafkaLogSinker::new(self.config.clone(), formatter).await + let max_delivery_buffer_size = (self + .config + .rdkafka_properties + .queue_buffering_max_messages + .as_ref() + .cloned() + .unwrap_or(KAFKA_WRITER_MAX_QUEUE_SIZE) as f32 + * KAFKA_WRITER_MAX_QUEUE_SIZE_RATIO) as usize; + + Ok(KafkaSinkWriter::new(self.config.clone(), formatter) + .await? + .into_log_sinker(max_delivery_buffer_size)) } async fn validate(&self) -> Result<()> { @@ -370,16 +379,15 @@ struct KafkaPayloadWriter<'a> { config: &'a KafkaConfig, } -type KafkaSinkDeliveryFuture = impl TryFuture + Unpin + 'static; +pub type KafkaSinkDeliveryFuture = impl TryFuture + Unpin + 'static; -pub struct KafkaLogSinker { +pub struct KafkaSinkWriter { formatter: SinkFormatterImpl, inner: FutureProducer, - future_manager: DeliveryFutureManager, config: KafkaConfig, } -impl KafkaLogSinker { +impl KafkaSinkWriter { async fn new(config: KafkaConfig, formatter: SinkFormatterImpl) -> Result { let inner: FutureProducer = { let mut c = ClientConfig::new(); @@ -403,19 +411,29 @@ impl KafkaLogSinker { c.create_with_context(producer_ctx).await? }; - let max_delivery_buffer_size = (config - .rdkafka_properties - .queue_buffering_max_messages - .as_ref() - .cloned() - .unwrap_or(KAFKA_WRITER_MAX_QUEUE_SIZE) as f32 - * KAFKA_WRITER_MAX_QUEUE_SIZE_RATIO) as usize; - - Ok(KafkaLogSinker { + Ok(KafkaSinkWriter { formatter, inner, config: config.clone(), - future_manager: DeliveryFutureManager::new(max_delivery_buffer_size), + }) + } +} + +impl AsyncTruncateSinkWriter for KafkaSinkWriter { + type DeliveryFuture = KafkaSinkDeliveryFuture; + + async fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> Result<()> { + let mut payload_writer = KafkaPayloadWriter { + inner: &mut self.inner, + add_future, + config: &self.config, + }; + dispatch_sink_formatter_impl!(&self.formatter, formatter, { + payload_writer.write_chunk(chunk, formatter).await }) } } @@ -537,50 +555,6 @@ impl<'a> FormattedSink for KafkaPayloadWriter<'a> { } } -impl LogSinker for KafkaLogSinker { - async fn consume_log_and_sink(mut self, mut log_reader: impl LogReader) -> Result<()> { - log_reader.init().await?; - loop { - let select_result = drop_either_future( - select( - pin!(log_reader.next_item()), - pin!(self.future_manager.next_truncate_offset()), - ) - .await, - ); - match select_result { - Either::Left(item_result) => { - let (epoch, item) = item_result?; - match item { - LogStoreReadItem::StreamChunk { chunk_id, chunk } => { - dispatch_sink_formatter_impl!(&self.formatter, formatter, { - let mut writer = KafkaPayloadWriter { - inner: &self.inner, - add_future: self - .future_manager - .start_write_chunk(epoch, chunk_id), - config: &self.config, - }; - writer.write_chunk(chunk, formatter).await?; - }) - } - LogStoreReadItem::Barrier { - is_checkpoint: _is_checkpoint, - } => { - self.future_manager.add_barrier(epoch); - } - LogStoreReadItem::UpdateVnodeBitmap(_) => {} - } - } - Either::Right(offset_result) => { - let offset = offset_result?; - log_reader.truncate(offset).await?; - } - } - } - } -} - #[cfg(test)] mod test { use maplit::hashmap; @@ -748,7 +722,7 @@ mod test { let kafka_config = KafkaConfig::from_hashmap(properties)?; // Create the actual sink writer to Kafka - let mut sink = KafkaLogSinker::new( + let sink = KafkaSinkWriter::new( kafka_config.clone(), SinkFormatterImpl::AppendOnlyJson(AppendOnlyFormatter::new( // We do not specify primary key for this schema @@ -759,12 +733,16 @@ mod test { .await .unwrap(); + use crate::sink::log_store::DeliveryFutureManager; + + let mut future_manager = DeliveryFutureManager::new(usize::MAX); + for i in 0..10 { println!("epoch: {}", i); for j in 0..100 { let mut writer = KafkaPayloadWriter { inner: &sink.inner, - add_future: sink.future_manager.start_write_chunk(i, j), + add_future: future_manager.start_write_chunk(i, j), config: &sink.config, }; match writer diff --git a/src/connector/src/sink/kinesis.rs b/src/connector/src/sink/kinesis.rs index dd8518af39948..605edde3b1eb0 100644 --- a/src/connector/src/sink/kinesis.rs +++ b/src/connector/src/sink/kinesis.rs @@ -30,8 +30,12 @@ use super::catalog::SinkFormatDesc; use super::SinkParam; use crate::common::KinesisCommon; use crate::dispatch_sink_formatter_impl; +use crate::sink::catalog::desc::SinkDesc; use crate::sink::formatter::SinkFormatterImpl; -use crate::sink::writer::{FormattedSink, LogSinkerOf, SinkWriter, SinkWriterExt}; +use crate::sink::log_store::DeliveryFutureManagerAddFuture; +use crate::sink::writer::{ + AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, FormattedSink, +}; use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkError, SinkWriterParam}; pub const KINESIS_SINK: &str = "kinesis"; @@ -67,10 +71,14 @@ impl TryFrom for KinesisSink { impl Sink for KinesisSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; + type LogSinker = AsyncTruncateLogSinkerOf; const SINK_NAME: &'static str = KINESIS_SINK; + fn default_sink_decouple(desc: &SinkDesc) -> bool { + desc.sink_type.is_append_only() + } + async fn validate(&self) -> Result<()> { // Kinesis requires partition key. There is no builtin support for round-robin as in kafka/pulsar. // https://docs.aws.amazon.com/kinesis/latest/APIReference/API_PutRecord.html#Streams-PutRecord-request-PartitionKey @@ -103,7 +111,7 @@ impl Sink for KinesisSink { Ok(()) } - async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + async fn new_log_sinker(&self, _writer_param: SinkWriterParam) -> Result { Ok(KinesisSinkWriter::new( self.config.clone(), self.schema.clone(), @@ -113,7 +121,7 @@ impl Sink for KinesisSink { self.sink_from_name.clone(), ) .await? - .into_log_sinker(writer_param.sink_metrics)) + .into_log_sinker(usize::MAX)) } } @@ -214,20 +222,16 @@ impl FormattedSink for KinesisSinkPayloadWriter { } } -#[async_trait::async_trait] -impl SinkWriter for KinesisSinkWriter { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - dispatch_sink_formatter_impl!(&self.formatter, formatter, { +impl AsyncTruncateSinkWriter for KinesisSinkWriter { + async fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> Result<()> { + dispatch_sink_formatter_impl!( + &self.formatter, + formatter, self.payload_writer.write_chunk(chunk, formatter).await - }) - } - - async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { - // Kinesis offers no transactional guarantees, so we do nothing here. - Ok(()) - } - - async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { - Ok(()) + ) } } diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 7769a87f4e715..6afd08778cd96 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -34,7 +34,6 @@ pub mod utils; pub mod writer; use std::collections::HashMap; -use std::future::Future; use ::clickhouse::error::Error as ClickHouseError; use ::redis::RedisError; @@ -278,11 +277,9 @@ pub trait Sink: TryFrom { } } -pub trait LogSinker: Send + 'static { - fn consume_log_and_sink( - self, - log_reader: impl LogReader, - ) -> impl Future> + Send + 'static; +#[async_trait] +pub trait LogSinker: 'static { + async fn consume_log_and_sink(self, log_reader: impl LogReader) -> Result<()>; } #[async_trait] diff --git a/src/connector/src/sink/nats.rs b/src/connector/src/sink/nats.rs index 8e3f3e2c18022..2f810eed786a9 100644 --- a/src/connector/src/sink/nats.rs +++ b/src/connector/src/sink/nats.rs @@ -25,10 +25,14 @@ use tokio_retry::strategy::{jitter, ExponentialBackoff}; use tokio_retry::Retry; use super::utils::chunk_to_json; -use super::{DummySinkCommitCoordinator, SinkWriter, SinkWriterParam}; +use super::{DummySinkCommitCoordinator, SinkWriterParam}; use crate::common::NatsCommon; +use crate::sink::catalog::desc::SinkDesc; use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode}; -use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; +use crate::sink::log_store::DeliveryFutureManagerAddFuture; +use crate::sink::writer::{ + AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, +}; use crate::sink::{Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY}; pub const NATS_SINK: &str = "nats"; @@ -88,10 +92,14 @@ impl TryFrom for NatsSink { impl Sink for NatsSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; + type LogSinker = AsyncTruncateLogSinkerOf; const SINK_NAME: &'static str = NATS_SINK; + fn default_sink_decouple(desc: &SinkDesc) -> bool { + desc.sink_type.is_append_only() + } + async fn validate(&self) -> Result<()> { if !self.is_append_only { return Err(SinkError::Nats(anyhow!( @@ -110,11 +118,11 @@ impl Sink for NatsSink { Ok(()) } - async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + async fn new_log_sinker(&self, _writer_param: SinkWriterParam) -> Result { Ok( NatsSinkWriter::new(self.config.clone(), self.schema.clone()) .await? - .into_log_sinker(writer_param.sink_metrics), + .into_log_sinker(usize::MAX), ) } } @@ -153,17 +161,12 @@ impl NatsSinkWriter { } } -#[async_trait::async_trait] -impl SinkWriter for NatsSinkWriter { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { +impl AsyncTruncateSinkWriter for NatsSinkWriter { + async fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> Result<()> { self.append_only(chunk).await } - - async fn begin_epoch(&mut self, _epoch_id: u64) -> Result<()> { - Ok(()) - } - - async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { - Ok(()) - } } diff --git a/src/connector/src/sink/pulsar.rs b/src/connector/src/sink/pulsar.rs index f980b2ad9f9b1..9eb57c1ae0771 100644 --- a/src/connector/src/sink/pulsar.rs +++ b/src/connector/src/sink/pulsar.rs @@ -12,14 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{HashMap, VecDeque}; +use std::collections::HashMap; use std::fmt::Debug; use std::time::Duration; use anyhow::anyhow; -use async_trait::async_trait; -use futures::future::try_join_all; -use futures::TryFutureExt; +use futures::{FutureExt, TryFuture, TryFutureExt}; use pulsar::producer::{Message, SendFuture}; use pulsar::{Producer, ProducerOptions, Pulsar, TokioExecutor}; use risingwave_common::array::StreamChunk; @@ -28,10 +26,15 @@ use serde::Deserialize; use serde_with::{serde_as, DisplayFromStr}; use super::catalog::{SinkFormat, SinkFormatDesc}; -use super::{Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam}; +use super::{Sink, SinkError, SinkParam, SinkWriterParam}; use crate::common::PulsarCommon; -use crate::sink::formatter::SinkFormatterImpl; -use crate::sink::writer::{FormattedSink, LogSinkerOf, SinkWriterExt}; +use crate::sink::catalog::desc::SinkDesc; +use crate::sink::encoder::SerTo; +use crate::sink::formatter::{SinkFormatter, SinkFormatterImpl}; +use crate::sink::log_store::DeliveryFutureManagerAddFuture; +use crate::sink::writer::{ + AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, FormattedSink, +}; use crate::sink::{DummySinkCommitCoordinator, Result}; use crate::{deserialize_duration_from_string, dispatch_sink_formatter_impl}; @@ -155,11 +158,15 @@ impl TryFrom for PulsarSink { impl Sink for PulsarSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; + type LogSinker = AsyncTruncateLogSinkerOf; const SINK_NAME: &'static str = PULSAR_SINK; - async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + fn default_sink_decouple(desc: &SinkDesc) -> bool { + desc.sink_type.is_append_only() + } + + async fn new_log_sinker(&self, _writer_param: SinkWriterParam) -> Result { Ok(PulsarSinkWriter::new( self.config.clone(), self.schema.clone(), @@ -169,7 +176,7 @@ impl Sink for PulsarSink { self.sink_from_name.clone(), ) .await? - .into_log_sinker(writer_param.sink_metrics)) + .into_log_sinker(PULSAR_SEND_FUTURE_BUFFER_MAX_SIZE)) } async fn validate(&self) -> Result<()> { @@ -199,15 +206,26 @@ impl Sink for PulsarSink { } pub struct PulsarSinkWriter { - payload_writer: PulsarPayloadWriter, formatter: SinkFormatterImpl, -} - -struct PulsarPayloadWriter { pulsar: Pulsar, producer: Producer, config: PulsarConfig, - send_future_buffer: VecDeque, +} + +struct PulsarPayloadWriter<'w> { + producer: &'w mut Producer, + config: &'w PulsarConfig, + add_future: DeliveryFutureManagerAddFuture<'w, PulsarDeliveryFuture>, +} + +pub type PulsarDeliveryFuture = impl TryFuture + Unpin + 'static; + +fn may_delivery_future(future: SendFuture) -> PulsarDeliveryFuture { + future.map(|result| { + result + .map(|_| ()) + .map_err(|e: pulsar::Error| SinkError::Pulsar(anyhow!(e))) + }) } impl PulsarSinkWriter { @@ -226,17 +244,14 @@ impl PulsarSinkWriter { let producer = build_pulsar_producer(&pulsar, &config).await?; Ok(Self { formatter, - payload_writer: PulsarPayloadWriter { - pulsar, - producer, - config, - send_future_buffer: VecDeque::new(), - }, + pulsar, + producer, + config, }) } } -impl PulsarPayloadWriter { +impl<'w> PulsarPayloadWriter<'w> { async fn send_message(&mut self, message: Message) -> Result<()> { let mut success_flag = false; let mut connection_err = None; @@ -247,17 +262,10 @@ impl PulsarPayloadWriter { // a SendFuture holding the message receipt // or error after sending is returned Ok(send_future) => { - // Check if send_future_buffer is greater than the preset limit - while self.send_future_buffer.len() >= PULSAR_SEND_FUTURE_BUFFER_MAX_SIZE { - self.send_future_buffer - .pop_front() - .expect("Expect the SendFuture not to be None") - .map_err(|e| SinkError::Pulsar(anyhow!(e))) - .await?; - } - + self.add_future + .add_future_may_await(may_delivery_future(send_future)) + .await?; success_flag = true; - self.send_future_buffer.push_back(send_future); break; } // error upon sending @@ -295,24 +303,9 @@ impl PulsarPayloadWriter { self.send_message(message).await?; Ok(()) } - - async fn commit_inner(&mut self) -> Result<()> { - self.producer - .send_batch() - .map_err(pulsar_to_sink_err) - .await?; - try_join_all( - self.send_future_buffer - .drain(..) - .map(|send_future| send_future.map_err(|e| SinkError::Pulsar(anyhow!(e)))), - ) - .await?; - - Ok(()) - } } -impl FormattedSink for PulsarPayloadWriter { +impl<'w> FormattedSink for PulsarPayloadWriter<'w> { type K = String; type V = Vec; @@ -321,23 +314,33 @@ impl FormattedSink for PulsarPayloadWriter { } } -#[async_trait] -impl SinkWriter for PulsarSinkWriter { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { +impl AsyncTruncateSinkWriter for PulsarSinkWriter { + type DeliveryFuture = PulsarDeliveryFuture; + + async fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> Result<()> { dispatch_sink_formatter_impl!(&self.formatter, formatter, { - self.payload_writer.write_chunk(chunk, formatter).await + let mut payload_writer = PulsarPayloadWriter { + producer: &mut self.producer, + add_future, + config: &self.config, + }; + // TODO: we can call `payload_writer.write_chunk(chunk, formatter)`, + // but for an unknown reason, this will greatly increase the compile time, + // by nearly 4x. May investigate it later. + for r in formatter.format_chunk(&chunk) { + let (key, value) = r?; + payload_writer + .write_inner( + key.map(SerTo::ser_to).transpose()?, + value.map(SerTo::ser_to).transpose()?, + ) + .await?; + } + Ok(()) }) } - - async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { - Ok(()) - } - - async fn barrier(&mut self, is_checkpoint: bool) -> Result { - if is_checkpoint { - self.payload_writer.commit_inner().await?; - } - - Ok(()) - } } diff --git a/src/connector/src/sink/redis.rs b/src/connector/src/sink/redis.rs index 6120075a049df..af3ec3b981620 100644 --- a/src/connector/src/sink/redis.rs +++ b/src/connector/src/sink/redis.rs @@ -29,8 +29,11 @@ use super::formatter::SinkFormatterImpl; use super::writer::FormattedSink; use super::{SinkError, SinkParam}; use crate::dispatch_sink_formatter_impl; -use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; -use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; +use crate::sink::log_store::DeliveryFutureManagerAddFuture; +use crate::sink::writer::{ + AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, +}; +use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriterParam}; pub const REDIS_SINK: &str = "redis"; pub const KEY_FORMAT: &str = "key_format"; @@ -99,11 +102,11 @@ impl TryFrom for RedisSink { impl Sink for RedisSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; + type LogSinker = AsyncTruncateLogSinkerOf; const SINK_NAME: &'static str = "redis"; - async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + async fn new_log_sinker(&self, _writer_param: SinkWriterParam) -> Result { Ok(RedisSinkWriter::new( self.config.clone(), self.schema.clone(), @@ -113,7 +116,7 @@ impl Sink for RedisSink { self.sink_from_name.clone(), ) .await? - .into_log_sinker(writer_param.sink_metrics)) + .into_log_sinker(usize::MAX)) } async fn validate(&self) -> Result<()> { @@ -257,25 +260,16 @@ impl RedisSinkWriter { } } -#[async_trait] -impl SinkWriter for RedisSinkWriter { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { +impl AsyncTruncateSinkWriter for RedisSinkWriter { + async fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> Result<()> { dispatch_sink_formatter_impl!(&self.formatter, formatter, { self.payload_writer.write_chunk(chunk, formatter).await }) } - - async fn begin_epoch(&mut self, epoch: u64) -> Result<()> { - self.epoch = epoch; - Ok(()) - } - - async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { - if is_checkpoint { - self.payload_writer.commit().await?; - } - Ok(()) - } } #[cfg(test)] @@ -290,6 +284,7 @@ mod test { use super::*; use crate::sink::catalog::{SinkEncode, SinkFormat}; + use crate::sink::log_store::DeliveryFutureManager; #[tokio::test] async fn test_write() { @@ -326,8 +321,10 @@ mod test { ], ); + let mut manager = DeliveryFutureManager::new(0); + redis_sink_writer - .write_batch(chunk_a) + .write_chunk(chunk_a, manager.start_write_chunk(0, 0)) .await .expect("failed to write batch"); let expected_a = @@ -383,6 +380,8 @@ mod test { .await .unwrap(); + let mut future_manager = DeliveryFutureManager::new(0); + let chunk_a = StreamChunk::new( vec![Op::Insert, Op::Insert, Op::Insert], vec![ @@ -392,7 +391,7 @@ mod test { ); redis_sink_writer - .write_batch(chunk_a) + .write_chunk(chunk_a, future_manager.start_write_chunk(0, 0)) .await .expect("failed to write batch"); let expected_a = vec![ diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index 310213262b2ad..3c52cb720dbd4 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -227,6 +227,7 @@ async fn await_future_with_monitor_receiver_err> } } +#[async_trait] impl LogSinker for RemoteLogSinker { async fn consume_log_and_sink(self, mut log_reader: impl LogReader) -> Result<()> { // Note: this is a total copy of the implementation of LogSinkerOf, diff --git a/src/connector/src/sink/writer.rs b/src/connector/src/sink/writer.rs index 37ad452831b2e..64261bb42ab48 100644 --- a/src/connector/src/sink/writer.rs +++ b/src/connector/src/sink/writer.rs @@ -12,17 +12,25 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::future::{Future, Ready}; +use std::pin::pin; use std::sync::Arc; use std::time::Instant; use async_trait::async_trait; +use futures::future::{select, Either}; +use futures::TryFuture; use risingwave_common::array::StreamChunk; use risingwave_common::buffer::Bitmap; +use risingwave_common::util::drop_either_future; use crate::sink::encoder::SerTo; use crate::sink::formatter::SinkFormatter; -use crate::sink::log_store::{LogReader, LogStoreReadItem, TruncateOffset}; -use crate::sink::{LogSinker, Result, SinkMetrics}; +use crate::sink::log_store::{ + DeliveryFutureManager, DeliveryFutureManagerAddFuture, LogReader, LogStoreReadItem, + TruncateOffset, +}; +use crate::sink::{LogSinker, Result, SinkError, SinkMetrics}; #[async_trait] pub trait SinkWriter: Send + 'static { @@ -48,22 +56,17 @@ pub trait SinkWriter: Send + 'static { } } -// TODO: remove this trait after KafkaSinkWriter implements SinkWriter -#[async_trait] -// An old version of SinkWriter for backward compatibility -pub trait SinkWriterV1: Send + 'static { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()>; - - // the following interface is for transactions, if not supported, return Ok(()) - // start a transaction with epoch number. Note that epoch number should be increasing. - async fn begin_epoch(&mut self, epoch: u64) -> Result<()>; +pub type DummyDeliveryFuture = Ready>; - // commits the current transaction and marks all messages in the transaction success. - async fn commit(&mut self) -> Result<()>; +pub trait AsyncTruncateSinkWriter: Send + 'static { + type DeliveryFuture: TryFuture + Unpin + Send + 'static = + DummyDeliveryFuture; - // aborts the current transaction because some error happens. we should rollback to the last - // commit point. - async fn abort(&mut self) -> Result<()>; + fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> impl Future> + Send + 'a; } /// A free-form sink that may output in multiple formats and encodings. Examples include kafka, @@ -104,12 +107,12 @@ pub trait FormattedSink { } } -pub struct LogSinkerOf> { +pub struct LogSinkerOf { writer: W, sink_metrics: SinkMetrics, } -impl> LogSinkerOf { +impl LogSinkerOf { pub fn new(writer: W, sink_metrics: SinkMetrics) -> Self { LogSinkerOf { writer, @@ -118,6 +121,7 @@ impl> LogSinkerOf { } } +#[async_trait] impl> LogSinker for LogSinkerOf { async fn consume_log_and_sink(self, mut log_reader: impl LogReader) -> Result<()> { let mut sink_writer = self.writer; @@ -222,3 +226,64 @@ where } } } + +pub struct AsyncTruncateLogSinkerOf { + writer: W, + future_manager: DeliveryFutureManager, +} + +impl AsyncTruncateLogSinkerOf { + pub fn new(writer: W, max_future_count: usize) -> Self { + AsyncTruncateLogSinkerOf { + writer, + future_manager: DeliveryFutureManager::new(max_future_count), + } + } +} + +#[async_trait] +impl LogSinker for AsyncTruncateLogSinkerOf { + async fn consume_log_and_sink(mut self, mut log_reader: impl LogReader) -> Result<()> { + log_reader.init().await?; + loop { + let select_result = drop_either_future( + select( + pin!(log_reader.next_item()), + pin!(self.future_manager.next_truncate_offset()), + ) + .await, + ); + match select_result { + Either::Left(item_result) => { + let (epoch, item) = item_result?; + match item { + LogStoreReadItem::StreamChunk { chunk_id, chunk } => { + let add_future = self.future_manager.start_write_chunk(epoch, chunk_id); + self.writer.write_chunk(chunk, add_future).await?; + } + LogStoreReadItem::Barrier { + is_checkpoint: _is_checkpoint, + } => { + self.future_manager.add_barrier(epoch); + } + LogStoreReadItem::UpdateVnodeBitmap(_) => {} + } + } + Either::Right(offset_result) => { + let offset = offset_result?; + log_reader.truncate(offset).await?; + } + } + } + } +} + +#[easy_ext::ext(AsyncTruncateSinkWriterExt)] +impl T +where + T: AsyncTruncateSinkWriter + Sized, +{ + pub fn into_log_sinker(self, max_future_count: usize) -> AsyncTruncateLogSinkerOf { + AsyncTruncateLogSinkerOf::new(self, max_future_count) + } +} From 3e89c83a5cd2aec90002591531a8e44a7cd5c826 Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Wed, 25 Oct 2023 13:03:39 +0800 Subject: [PATCH 17/48] feat: add iceberg cdc test (#12956) Co-authored-by: ZENOTME --- .gitignore | 3 + ci/scripts/e2e-iceberg-cdc.sh | 91 +++++++++++ ci/workflows/integration-tests.yml | 5 + ci/workflows/pull-request.yml | 15 ++ docker/docker-compose.yml | 1 + e2e_test/iceberg/main.py | 8 +- e2e_test/iceberg/test_case/cdc/load.slt | 46 ++++++ e2e_test/iceberg/test_case/cdc/mysql_cdc.sql | 21 +++ .../test_case/cdc/mysql_cdc_insert.sql | 7 + .../test_case/cdc/no_partition_cdc.toml | 25 +++ .../test_case/cdc/no_partition_cdc_init.toml | 31 ++++ integration_tests/iceberg-cdc/README.md | 5 + .../iceberg-cdc/docker-compose.yaml | 142 ++++++++++++++++++ .../iceberg-cdc/mysql_prepare.sql | 15 ++ integration_tests/iceberg-cdc/python/check.py | 25 +++ .../iceberg-cdc/python/config.ini | 8 + integration_tests/iceberg-cdc/python/init.py | 103 +++++++++++++ .../iceberg-cdc/python/pyproject.toml | 16 ++ integration_tests/iceberg-cdc/run_test.sh | 19 +++ .../iceberg-cdc/spark/.gitignore | 3 + .../iceberg-cdc/spark/spark-connect-server.sh | 21 +++ integration_tests/scripts/run_demos.py | 9 ++ 22 files changed, 616 insertions(+), 3 deletions(-) create mode 100755 ci/scripts/e2e-iceberg-cdc.sh create mode 100644 e2e_test/iceberg/test_case/cdc/load.slt create mode 100644 e2e_test/iceberg/test_case/cdc/mysql_cdc.sql create mode 100644 e2e_test/iceberg/test_case/cdc/mysql_cdc_insert.sql create mode 100644 e2e_test/iceberg/test_case/cdc/no_partition_cdc.toml create mode 100644 e2e_test/iceberg/test_case/cdc/no_partition_cdc_init.toml create mode 100644 integration_tests/iceberg-cdc/README.md create mode 100644 integration_tests/iceberg-cdc/docker-compose.yaml create mode 100644 integration_tests/iceberg-cdc/mysql_prepare.sql create mode 100644 integration_tests/iceberg-cdc/python/check.py create mode 100644 integration_tests/iceberg-cdc/python/config.ini create mode 100644 integration_tests/iceberg-cdc/python/init.py create mode 100644 integration_tests/iceberg-cdc/python/pyproject.toml create mode 100755 integration_tests/iceberg-cdc/run_test.sh create mode 100644 integration_tests/iceberg-cdc/spark/.gitignore create mode 100755 integration_tests/iceberg-cdc/spark/spark-connect-server.sh diff --git a/.gitignore b/.gitignore index 19fb6643dd8a6..375738f67093e 100644 --- a/.gitignore +++ b/.gitignore @@ -74,4 +74,7 @@ simulation-it-test.tar.zst # hummock-trace .trace +# spark binary +e2e_test/iceberg/spark-*-bin* + **/poetry.lock \ No newline at end of file diff --git a/ci/scripts/e2e-iceberg-cdc.sh b/ci/scripts/e2e-iceberg-cdc.sh new file mode 100755 index 0000000000000..081f5bbd2afcb --- /dev/null +++ b/ci/scripts/e2e-iceberg-cdc.sh @@ -0,0 +1,91 @@ +#!/usr/bin/env bash + +# Exits as soon as any line fails. +set -euo pipefail + +source ci/scripts/common.sh + +# prepare environment +export CONNECTOR_RPC_ENDPOINT="localhost:50051" +export CONNECTOR_LIBS_PATH="./connector-node/libs" + +while getopts 'p:' opt; do + case ${opt} in + p ) + profile=$OPTARG + ;; + \? ) + echo "Invalid Option: -$OPTARG" 1>&2 + exit 1 + ;; + : ) + echo "Invalid option: $OPTARG requires an argument" 1>&2 + ;; + esac +done +shift $((OPTIND -1)) + +download_and_prepare_rw "$profile" source + +echo "--- Download connector node package" +buildkite-agent artifact download risingwave-connector.tar.gz ./ +mkdir ./connector-node +tar xf ./risingwave-connector.tar.gz -C ./connector-node + +echo "--- e2e, ci-1cn-1fe, iceberg cdc" + +node_port=50051 +node_timeout=10 + +wait_for_connector_node_start() { + start_time=$(date +%s) + while : + do + if nc -z localhost $node_port; then + echo "Port $node_port is listened! Connector Node is up!" + break + fi + + current_time=$(date +%s) + elapsed_time=$((current_time - start_time)) + if [ $elapsed_time -ge $node_timeout ]; then + echo "Timeout waiting for port $node_port to be listened!" + exit 1 + fi + sleep 0.1 + done + sleep 2 +} + +echo "--- starting risingwave cluster with connector node" + +RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ +cargo make ci-start ci-1cn-1fe-with-recovery +./connector-node/start-service.sh -p $node_port > .risingwave/log/connector-node.log 2>&1 & +echo "waiting for connector node to start" +wait_for_connector_node_start + +# prepare minio iceberg sink +echo "--- preparing iceberg" +.risingwave/bin/mcli -C .risingwave/config/mcli mb hummock-minio/icebergdata + +cd e2e_test/iceberg +bash ./start_spark_connect_server.sh + +# Don't remove the `--quiet` option since poetry has a bug when printing output, see +# https://github.com/python-poetry/poetry/issues/3412 +"$HOME"/.local/bin/poetry update --quiet + +# 1. import data to mysql +mysql --host=mysql --port=3306 -u root -p123456 < ./test_case/cdc/mysql_cdc.sql + +# 2. create table and sink +"$HOME"/.local/bin/poetry run python main.py -t ./test_case/cdc/no_partition_cdc_init.toml + +# 3. insert new data to mysql +mysql --host=mysql --port=3306 -u root -p123456 < ./test_case/cdc/mysql_cdc_insert.sql + +sleep 20 + +# 4. check change +"$HOME"/.local/bin/poetry run python main.py -t ./test_case/cdc/no_partition_cdc.toml \ No newline at end of file diff --git a/ci/workflows/integration-tests.yml b/ci/workflows/integration-tests.yml index 4bd0ec1a000b1..455f29b210ec1 100644 --- a/ci/workflows/integration-tests.yml +++ b/ci/workflows/integration-tests.yml @@ -29,6 +29,7 @@ steps: - "postgres-cdc" - "mysql-sink" - "postgres-sink" + - "iceberg-cdc" # - "iceberg-sink" - "debezium-mysql" format: @@ -79,6 +80,10 @@ steps: # testcase: "iceberg-sink" # format: "protobuf" # skip: true + - with: + testcase: "iceberg-cdc" + format: "protobuf" + skip: true - with: testcase: "debezium-mysql" format: "protobuf" diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 985bd0be4b822..3aaa09f0d7716 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -209,6 +209,21 @@ steps: timeout_in_minutes: 10 retry: *auto-retry + - label: "end-to-end iceberg cdc test" + if: build.pull_request.labels includes "ci/run-e2e-iceberg-sink-tests" + command: "ci/scripts/e2e-iceberg-cdc.sh -p ci-dev" + depends_on: + - "build" + - "build-other" + plugins: + - docker-compose#v4.9.0: + run: sink-test-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 10 + retry: *auto-retry + - label: "end-to-end pulsar sink test" if: build.pull_request.labels includes "ci/run-e2e-pulsar-sink-tests" command: "ci/scripts/e2e-pulsar-sink-test.sh -p ci-dev" diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index d25c94daf2670..4dbd5fe5bb28d 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -260,6 +260,7 @@ services: MINIO_PROMETHEUS_URL: "http://prometheus-0:9500" MINIO_ROOT_PASSWORD: hummockadmin MINIO_ROOT_USER: hummockadmin + MINIO_DOMAIN: "minio-0" container_name: minio-0 healthcheck: test: diff --git a/e2e_test/iceberg/main.py b/e2e_test/iceberg/main.py index fa07aa367a9b3..3f3120227e6e7 100644 --- a/e2e_test/iceberg/main.py +++ b/e2e_test/iceberg/main.py @@ -42,14 +42,16 @@ def init_iceberg_table(args,init_sqls): spark.sql(sql) -def init_risingwave_mv(args,slt): +def execute_slt(args,slt): + if slt is None or slt == "": + return rw_config = args['risingwave'] cmd = f"sqllogictest -p {rw_config['port']} -d {rw_config['db']} {slt}" print(f"Command line is [{cmd}]") subprocess.run(cmd, shell=True, check=True) - time.sleep(10) + time.sleep(30) def verify_result(args,verify_sql,verify_schema,verify_data): @@ -110,6 +112,6 @@ def drop_table(args,drop_sqls): print({section: dict(config[section]) for section in config.sections()}) init_iceberg_table(config,init_sqls) - init_risingwave_mv(config,slt) + execute_slt(config,slt) verify_result(config,verify_sql,verify_schema,verify_data) drop_table(config,drop_sqls) diff --git a/e2e_test/iceberg/test_case/cdc/load.slt b/e2e_test/iceberg/test_case/cdc/load.slt new file mode 100644 index 0000000000000..caefd1326bbda --- /dev/null +++ b/e2e_test/iceberg/test_case/cdc/load.slt @@ -0,0 +1,46 @@ +# CDC source basic test + +# enable cdc backfill in ci +statement ok +set cdc_backfill='true'; + +statement ok +create table products ( id INT, + name STRING, + description STRING, + PRIMARY KEY (id) +) with ( + connector = 'mysql-cdc', + hostname = 'mysql', + port = '3306', + username = 'root', + password = '123456', + database.name = 'my@db', + table.name = 'products', + server.id = '5085' +); + + +statement ok +CREATE SINK s1 AS select * from products WITH ( + connector = 'iceberg', + type = 'upsert', + force_append_only = 'false', + database.name = 'demo', + table.name = 'demo_db.demo_table', + catalog.type = 'storage', + warehouse.path = 's3://icebergdata/demo', + s3.endpoint = 'http://127.0.0.1:9301', + s3.region = 'us-east-1', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + primary_key = 'id' +); + +query I +select count(*) from products; +---- +8 + +statement ok +flush; diff --git a/e2e_test/iceberg/test_case/cdc/mysql_cdc.sql b/e2e_test/iceberg/test_case/cdc/mysql_cdc.sql new file mode 100644 index 0000000000000..b7b6f13af83cf --- /dev/null +++ b/e2e_test/iceberg/test_case/cdc/mysql_cdc.sql @@ -0,0 +1,21 @@ +DROP DATABASE IF EXISTS `my@db`; +CREATE DATABASE `my@db`; + +USE `my@db`; + +CREATE TABLE products ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + name VARCHAR(255) NOT NULL, + description VARCHAR(512) +); + +ALTER TABLE products AUTO_INCREMENT = 101; + +INSERT INTO products VALUES (default,"101","101"), +(default,"102","102"), +(default,"103","103"), +(default,"104","104"), +(default,"105","105"), +(default,"106","106"), +(default,"107","107"), +(default,"108","108") diff --git a/e2e_test/iceberg/test_case/cdc/mysql_cdc_insert.sql b/e2e_test/iceberg/test_case/cdc/mysql_cdc_insert.sql new file mode 100644 index 0000000000000..641d6220ea8dc --- /dev/null +++ b/e2e_test/iceberg/test_case/cdc/mysql_cdc_insert.sql @@ -0,0 +1,7 @@ +USE `my@db`; + +INSERT INTO products VALUES (default,"109","109"), +(default,"110","110"), +(default,"111","111"), +(default,"112","112"), +(default,"113","113"); diff --git a/e2e_test/iceberg/test_case/cdc/no_partition_cdc.toml b/e2e_test/iceberg/test_case/cdc/no_partition_cdc.toml new file mode 100644 index 0000000000000..5ab9647b12eb0 --- /dev/null +++ b/e2e_test/iceberg/test_case/cdc/no_partition_cdc.toml @@ -0,0 +1,25 @@ +init_sqls = [] + +slt = '' + +verify_schema = ['int','string','string'] + +verify_sql = 'SELECT * FROM demo_db.demo_table ORDER BY id ASC' + +verify_data = """ +101,101,101 +102,102,102 +103,103,103 +104,104,104 +105,105,105 +106,106,106 +107,107,107 +108,108,108 +109,109,109 +110,110,110 +111,111,111 +112,112,112 +113,113,113 +""" + +drop_sqls = [] diff --git a/e2e_test/iceberg/test_case/cdc/no_partition_cdc_init.toml b/e2e_test/iceberg/test_case/cdc/no_partition_cdc_init.toml new file mode 100644 index 0000000000000..17e5f7497aae5 --- /dev/null +++ b/e2e_test/iceberg/test_case/cdc/no_partition_cdc_init.toml @@ -0,0 +1,31 @@ +init_sqls = [ + 'CREATE SCHEMA IF NOT EXISTS demo_db', + 'DROP TABLE IF EXISTS demo_db.demo_table', + ''' + CREATE TABLE demo_db.demo_table ( + id int, + name string, + description string + ) USING iceberg + TBLPROPERTIES ('format-version'='2'); + ''' +] + +slt = 'test_case/cdc/load.slt' + +verify_schema = ['int','string','string'] + +verify_sql = 'SELECT * FROM demo_db.demo_table ORDER BY id ASC' + +verify_data = """ +101,101,101 +102,102,102 +103,103,103 +104,104,104 +105,105,105 +106,106,106 +107,107,107 +108,108,108 +""" + +drop_sqls = [] diff --git a/integration_tests/iceberg-cdc/README.md b/integration_tests/iceberg-cdc/README.md new file mode 100644 index 0000000000000..56f40172c3dfa --- /dev/null +++ b/integration_tests/iceberg-cdc/README.md @@ -0,0 +1,5 @@ +# Iceberg CDC Integration Tests +`mysql -> rw -> iceberg` + +# How to run +./run_test.sh \ No newline at end of file diff --git a/integration_tests/iceberg-cdc/docker-compose.yaml b/integration_tests/iceberg-cdc/docker-compose.yaml new file mode 100644 index 0000000000000..8e9ad1062ef38 --- /dev/null +++ b/integration_tests/iceberg-cdc/docker-compose.yaml @@ -0,0 +1,142 @@ +version: '3.8' + +services: + compactor-0: + extends: + file: ../../docker/docker-compose.yml + service: compactor-0 + compute-node-0: + extends: + file: ../../docker/docker-compose.yml + service: compute-node-0 + etcd-0: + extends: + file: ../../docker/docker-compose.yml + service: etcd-0 + frontend-node-0: + extends: + file: ../../docker/docker-compose.yml + service: frontend-node-0 + meta-node-0: + extends: + file: ../../docker/docker-compose.yml + service: meta-node-0 + grafana-0: + extends: + file: ../../docker/docker-compose.yml + service: grafana-0 + prometheus-0: + extends: + file: ../../docker/docker-compose.yml + service: prometheus-0 + minio-0: + extends: + file: ../../docker/docker-compose.yml + service: minio-0 + mc: + depends_on: + - minio-0 + image: minio/mc + environment: + - AWS_ACCESS_KEY_ID=hummockadmin + - AWS_SECRET_ACCESS_KEY=hummockadmin + - AWS_REGION=us-east-1 + entrypoint: > + /bin/sh -c " + until (/usr/bin/mc config host add minio http://minio-0:9301 hummockadmin hummockadmin) do echo '...waiting...' && sleep 1; done; + /usr/bin/mc rm -r --force minio/icebergdata; + /usr/bin/mc mb minio/icebergdata; + /usr/bin/mc anonymous set public minio/icebergdata; + tail -f /dev/null + " + + mysql: + image: mysql:8.0 + expose: + - 3306 + ports: + - "3306:3306" + environment: + - MYSQL_ROOT_PASSWORD=123456 + - MYSQL_USER=mysqluser + - MYSQL_PASSWORD=mysqlpw + - MYSQL_DATABASE=mydb + healthcheck: + test: [ "CMD-SHELL", "mysqladmin ping -h 127.0.0.1 -u root -p123456" ] + interval: 5s + timeout: 5s + retries: 5 + container_name: mysql + prepare_mysql: + image: mysql:8.0 + depends_on: + - mysql + command: + - /bin/sh + - -c + - "mysql -p123456 -h mysql mydb < mysql_prepare.sql" + volumes: + - "./mysql_prepare.sql:/mysql_prepare.sql" + container_name: prepare_mysql + restart: on-failure + + rest: + image: tabulario/iceberg-rest:0.6.0 + environment: + - AWS_ACCESS_KEY_ID=hummockadmin + - AWS_SECRET_ACCESS_KEY=hummockadmin + - AWS_REGION=us-east-1 + - CATALOG_CATOLOG__IMPL=org.apache.iceberg.jdbc.JdbcCatalog + - CATALOG_URI=jdbc:sqlite:file:/tmp/iceberg_rest_mode=memory + - CATALOG_WAREHOUSE=s3://icebergdata/demo + - CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO + - CATALOG_S3_ENDPOINT=http://minio-0:9301 + depends_on: + - minio-0 + # let the rest access minio through: hummock001.minio-0 + links: + - minio-0:icebergdata.minio-0 + expose: + - 8181 + ports: + - "8181:8181" + + spark: + depends_on: + - minio-0 + - rest + image: ghcr.io/icelake-io/icelake-spark:latest + environment: + - AWS_ACCESS_KEY_ID=hummockadmin + - AWS_SECRET_ACCESS_KEY=hummockadmin + - AWS_REGION=us-east-1 + - SPARK_HOME=/opt/spark + - PATH=/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/opt/spark/bin:/opt/spark/sbin + user: root + links: + - minio-0:icebergdata.minio-0 + expose: + - 15002 + ports: + - "15002:15002" + healthcheck: + test: netstat -ltn | grep -c 15002 + interval: 1s + retries: 1200 + volumes: + - ./spark:/spark + command: [ "bash", "/spark/spark-connect-server.sh" ] + +volumes: + compute-node-0: + external: false + etcd-0: + external: false + grafana-0: + external: false + minio-0: + external: false + prometheus-0: + external: false + spark: + external: false diff --git a/integration_tests/iceberg-cdc/mysql_prepare.sql b/integration_tests/iceberg-cdc/mysql_prepare.sql new file mode 100644 index 0000000000000..3e5a236a41205 --- /dev/null +++ b/integration_tests/iceberg-cdc/mysql_prepare.sql @@ -0,0 +1,15 @@ +-- mysql -p123456 -uroot -h 127.0.0.1 mydb < mysql_prepare.sql +-- +-- Mysql +USE mydb; + +CREATE TABLE user_behaviors ( + user_id VARCHAR(60), + target_id VARCHAR(60), + target_type VARCHAR(60), + event_timestamp VARCHAR(100), + behavior_type VARCHAR(60), + parent_target_type VARCHAR(60), + parent_target_id VARCHAR(60), + PRIMARY KEY(user_id, target_id, event_timestamp) +); diff --git a/integration_tests/iceberg-cdc/python/check.py b/integration_tests/iceberg-cdc/python/check.py new file mode 100644 index 0000000000000..699fa4df29c30 --- /dev/null +++ b/integration_tests/iceberg-cdc/python/check.py @@ -0,0 +1,25 @@ +from pyspark.sql import SparkSession +import configparser +import psycopg2 + +def check_spark_table(args): + expect_row_count = 0 + rw_config = args['risingwave'] + with psycopg2.connect(database=rw_config['db'], user=rw_config['user'], host=rw_config['host'], + port=rw_config['port']) as conn: + with conn.cursor() as cursor: + cursor.execute("SELECT COUNT(*) FROM user_behaviors") + expect_row_count = cursor.fetchone()[0] + print(f"expect_row_count is {expect_row_count}") + spark_config = args['spark'] + spark = SparkSession.builder.remote(spark_config['url']).getOrCreate() + actual_row_count = spark.sql("SELECT COUNT(*) FROM s1.t1").collect()[0][0] + print(f"actual_row_count is {actual_row_count}") + assert actual_row_count==expect_row_count + + +if __name__ == "__main__": + config = configparser.ConfigParser() + config.read("config.ini") + print({section: dict(config[section]) for section in config.sections()}) + check_spark_table(config) diff --git a/integration_tests/iceberg-cdc/python/config.ini b/integration_tests/iceberg-cdc/python/config.ini new file mode 100644 index 0000000000000..bd95eddc5b80e --- /dev/null +++ b/integration_tests/iceberg-cdc/python/config.ini @@ -0,0 +1,8 @@ +[spark] +url=sc://localhost:15002 + +[risingwave] +db=dev +user=root +host=127.0.0.1 +port=4566 diff --git a/integration_tests/iceberg-cdc/python/init.py b/integration_tests/iceberg-cdc/python/init.py new file mode 100644 index 0000000000000..289fa2f161889 --- /dev/null +++ b/integration_tests/iceberg-cdc/python/init.py @@ -0,0 +1,103 @@ +from pyspark.sql import SparkSession +import configparser +import psycopg2 + + +def init_spark_table(args): + spark_config = args['spark'] + spark = SparkSession.builder.remote(spark_config['url']).getOrCreate() + + init_table_sqls = [ + "CREATE SCHEMA IF NOT EXISTS s1", + "DROP TABLE IF EXISTS s1.t1", + """ + CREATE TABLE s1.t1 + ( + user_id string, + target_id string, + target_type string, + event_timestamp string, + behavior_type string, + parent_target_type string, + parent_target_id string + ) USING iceberg + TBLPROPERTIES ('format-version'='2'); + """, + ] + + for sql in init_table_sqls: + print(f"Executing sql: {sql}") + spark.sql(sql) + + +def init_risingwave_mv(args): + rw_config = args['risingwave'] + sqls = [ + "set streaming_parallelism = 4", + """ + CREATE TABLE user_behaviors ( + user_id VARCHAR, + target_id VARCHAR, + target_type VARCHAR, + event_timestamp VARCHAR, + behavior_type VARCHAR, + parent_target_type VARCHAR, + parent_target_id VARCHAR, + PRIMARY KEY(user_id, target_id, event_timestamp) + ) with ( + connector = 'mysql-cdc', + hostname = 'mysql', + port = '3306', + username = 'root', + password = '123456', + database.name = 'mydb', + table.name = 'user_behaviors', + server.id = '1' + ); + """, + # f""" + # CREATE SINK s1 + # AS SELECT * FROM user_behaviors + # WITH ( + # connector='iceberg', + # type='upsert', + # primary_key = 'user_id, target_id, event_timestamp', + # catalog.type = 'storage', + # s3.endpoint = 'http://minio-0:9301', + # s3.access.key = 'hummockadmin', + # s3.secret.key = 'hummockadmin', + # database.name='demo', + # table.name='s1.t1',warehouse.path = 's3://hummock001/icebergdata/demo',s3.region = 'us-east-1' + # ); + # """ + f""" + CREATE SINK s1 + AS SELECT * FROM user_behaviors + WITH ( + connector='iceberg', + type='upsert', + primary_key = 'user_id, target_id, event_timestamp', + catalog.type = 'rest', + catalog.uri = 'http://rest:8181', + s3.endpoint = 'http://minio-0:9301', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + database.name='demo', + table.name='s1.t1',warehouse.path = 's3://icebergdata/demo/s1/t1',s3.region = 'us-east-1' + ); + """ + ] + with psycopg2.connect(database=rw_config['db'], user=rw_config['user'], host=rw_config['host'], + port=rw_config['port']) as conn: + with conn.cursor() as cursor: + for sql in sqls: + print(f"Executing sql {sql}") + cursor.execute(sql) + + +if __name__ == "__main__": + config = configparser.ConfigParser() + config.read("config.ini") + print({section: dict(config[section]) for section in config.sections()}) + init_spark_table(config) + init_risingwave_mv(config) diff --git a/integration_tests/iceberg-cdc/python/pyproject.toml b/integration_tests/iceberg-cdc/python/pyproject.toml new file mode 100644 index 0000000000000..4c7bce1165796 --- /dev/null +++ b/integration_tests/iceberg-cdc/python/pyproject.toml @@ -0,0 +1,16 @@ +[tool.poetry] +name = "icelake-integration-tests" +version = "0.0.9" +description = "" +authors = ["Renjie Liu "] +readme = "README.md" +packages = [{include = "icelake_integration_tests"}] + +[tool.poetry.dependencies] +python = "^3.11" +pyspark = { version = "3.4.1", extras = ["sql", "connect"] } +psycopg2-binary = "^2.9" + +[build-system] +requires = ["poetry-core"] +build-backend = "poetry.core.masonry.api" diff --git a/integration_tests/iceberg-cdc/run_test.sh b/integration_tests/iceberg-cdc/run_test.sh new file mode 100755 index 0000000000000..2d8b691bc7284 --- /dev/null +++ b/integration_tests/iceberg-cdc/run_test.sh @@ -0,0 +1,19 @@ +#!/bin/bash + +# Start test environment. +docker-compose up -d --wait + +# To avoid exiting by unhealth, set it after start environment. +set -ex + +# Generate data +docker build -t iceberg-cdc-datagen ../datagen +timeout 20 docker run --network=iceberg-cdc_default iceberg-cdc-datagen /datagen --mode clickstream --qps 1 mysql --user mysqluser --password mysqlpw --host mysql --port 3306 --db mydb & + +cd python +poetry update --quiet +# Init source, mv, and sink. +poetry run python init.py +# Wait for sink to be finished. +sleep 40; +poetry run python check.py diff --git a/integration_tests/iceberg-cdc/spark/.gitignore b/integration_tests/iceberg-cdc/spark/.gitignore new file mode 100644 index 0000000000000..51dcf07222856 --- /dev/null +++ b/integration_tests/iceberg-cdc/spark/.gitignore @@ -0,0 +1,3 @@ +derby.log +metastore_db +.ivy \ No newline at end of file diff --git a/integration_tests/iceberg-cdc/spark/spark-connect-server.sh b/integration_tests/iceberg-cdc/spark/spark-connect-server.sh new file mode 100755 index 0000000000000..7c1cd64f1a2f2 --- /dev/null +++ b/integration_tests/iceberg-cdc/spark/spark-connect-server.sh @@ -0,0 +1,21 @@ +#!/bin/bash + +set -ex + +JARS=$(find /opt/spark/deps -type f -name "*.jar" | tr '\n' ':') + +/opt/spark/sbin/start-connect-server.sh \ + --master local[3] \ + --driver-class-path $JARS \ + --conf spark.driver.bindAddress=0.0.0.0 \ + --conf spark.sql.catalog.demo=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \ + --conf spark.sql.catalog.demo.catalog-impl=org.apache.iceberg.rest.RESTCatalog \ + --conf spark.sql.catalog.demo.uri=http://rest:8181 \ + --conf spark.sql.catalog.demo.s3.endpoint=http://minio-0:9301 \ + --conf spark.sql.catalog.demo.s3.path.style.access=true \ + --conf spark.sql.catalog.demo.s3.access.key=hummockadmin \ + --conf spark.sql.catalog.demo.s3.secret.key=hummockadmin \ + --conf spark.sql.defaultCatalog=demo + +tail -f /opt/spark/logs/spark*.out diff --git a/integration_tests/scripts/run_demos.py b/integration_tests/scripts/run_demos.py index 28623f7ddc4a7..da2519e18db44 100644 --- a/integration_tests/scripts/run_demos.py +++ b/integration_tests/scripts/run_demos.py @@ -42,6 +42,13 @@ def run_demo(demo: str, format: str, wait_time = 40): run_sql_file(sql_file, demo_dir) sleep(10) +def iceberg_cdc_demo(): + demo = "iceberg-cdc" + file_dir = dirname(abspath(__file__)) + project_dir = dirname(file_dir) + demo_dir = os.path.join(project_dir, demo) + print("Running demo: iceberg-cdc") + subprocess.run(["bash","./run_test.sh"], cwd=demo_dir, check=True) def run_iceberg_demo(): demo = "iceberg-sink" @@ -149,5 +156,7 @@ def run_clickhouse_demo(): run_iceberg_demo() elif args.case == "clickhouse-sink": run_clickhouse_demo() +elif args.case == "iceberg-cdc": + iceberg_cdc_demo() else: run_demo(args.case, args.format) From 2d428b153e85973faf8f849cd95e8aaa0690a624 Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Wed, 25 Oct 2023 14:03:30 +0800 Subject: [PATCH 18/48] feat(sink): Optimizing ClickHouse Upsert (#12259) --- integration_tests/clickhouse-sink/README.md | 2 + src/connector/src/sink/clickhouse.rs | 410 ++++++++++++-------- 2 files changed, 249 insertions(+), 163 deletions(-) diff --git a/integration_tests/clickhouse-sink/README.md b/integration_tests/clickhouse-sink/README.md index 607621faefeae..a383f3fba5ee4 100644 --- a/integration_tests/clickhouse-sink/README.md +++ b/integration_tests/clickhouse-sink/README.md @@ -23,6 +23,8 @@ docker compose exec clickhouse-server bash /opt/clickhouse/clickhouse-sql/run-sq - create_mv.sql - create_sink.sql +We only support `upsert` with clickhouse' `CollapsingMergeTree` and `VersionedCollapsingMergeTree` + 4. Execute a simple query: ```sh diff --git a/src/connector/src/sink/clickhouse.rs b/src/connector/src/sink/clickhouse.rs index f4fdf9b761f38..fb06baf42920c 100644 --- a/src/connector/src/sink/clickhouse.rs +++ b/src/connector/src/sink/clickhouse.rs @@ -11,18 +11,17 @@ // 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 core::fmt::Debug; use std::collections::{HashMap, HashSet}; use std::time::Duration; use anyhow::anyhow; -use clickhouse::{Client, Client as ClickHouseClient, Row as ClickHouseRow}; +use clickhouse::{Client as ClickHouseClient, Row as ClickHouseRow}; use itertools::Itertools; -use risingwave_common::array::{Op, RowRef, StreamChunk}; +use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::catalog::Schema; use risingwave_common::row::Row; -use risingwave_common::types::{DataType, ScalarRefImpl, Serial}; +use risingwave_common::types::{DataType, Decimal, ScalarRefImpl, Serial}; use serde::ser::{SerializeSeq, SerializeStruct}; use serde::Serialize; use serde_derive::Deserialize; @@ -38,6 +37,10 @@ use crate::sink::{ Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; +const QUERY_ENGINE: &str = + "select distinct ?fields from system.tables where database = ? and table = ?"; +const QUERY_COLUMN: &str = + "select distinct ?fields from system.columns where database = ? and table = ? order by ?"; pub const CLICKHOUSE_SINK: &str = "clickhouse"; const BUFFER_SIZE: usize = 1024; @@ -55,6 +58,75 @@ pub struct ClickHouseCommon { pub table: String, } +#[allow(clippy::enum_variant_names)] +#[derive(Debug)] +enum ClickHouseEngine { + MergeTree, + ReplacingMergeTree, + SummingMergeTree, + AggregatingMergeTree, + CollapsingMergeTree(String), + VersionedCollapsingMergeTree(String), + GraphiteMergeTree, +} +impl ClickHouseEngine { + pub fn is_collapsing_engine(&self) -> bool { + matches!( + self, + ClickHouseEngine::CollapsingMergeTree(_) + | ClickHouseEngine::VersionedCollapsingMergeTree(_) + ) + } + + pub fn get_sign_name(&self) -> Option { + match self { + ClickHouseEngine::CollapsingMergeTree(sign_name) => Some(sign_name.to_string()), + ClickHouseEngine::VersionedCollapsingMergeTree(sign_name) => { + Some(sign_name.to_string()) + } + _ => None, + } + } + + pub fn from_query_engine(engine_name: &ClickhouseQueryEngine) -> Result { + match engine_name.engine.as_str() { + "MergeTree" => Ok(ClickHouseEngine::MergeTree), + "ReplacingMergeTree" => Ok(ClickHouseEngine::ReplacingMergeTree), + "SummingMergeTree" => Ok(ClickHouseEngine::SummingMergeTree), + "AggregatingMergeTree" => Ok(ClickHouseEngine::AggregatingMergeTree), + "VersionedCollapsingMergeTree" => { + let sign_name = engine_name + .create_table_query + .split("VersionedCollapsingMergeTree(") + .last() + .ok_or_else(|| SinkError::ClickHouse("must have last".to_string()))? + .split(',') + .next() + .ok_or_else(|| SinkError::ClickHouse("must have next".to_string()))? + .to_string(); + Ok(ClickHouseEngine::VersionedCollapsingMergeTree(sign_name)) + } + "CollapsingMergeTree" => { + let sign_name = engine_name + .create_table_query + .split("CollapsingMergeTree(") + .last() + .ok_or_else(|| SinkError::ClickHouse("must have last".to_string()))? + .split(')') + .next() + .ok_or_else(|| SinkError::ClickHouse("must have next".to_string()))? + .to_string(); + Ok(ClickHouseEngine::CollapsingMergeTree(sign_name)) + } + "GraphiteMergeTree" => Ok(ClickHouseEngine::GraphiteMergeTree), + _ => Err(SinkError::ClickHouse(format!( + "Cannot find clickhouse engine {:?}", + engine_name.engine + ))), + } + } +} + const POOL_IDLE_TIMEOUT: Duration = Duration::from_secs(5); impl ClickHouseCommon { @@ -198,9 +270,7 @@ impl ClickHouseSink { } risingwave_common::types::DataType::Float32 => Ok(ck_column.r#type.contains("Float32")), risingwave_common::types::DataType::Float64 => Ok(ck_column.r#type.contains("Float64")), - risingwave_common::types::DataType::Decimal => { - Err(SinkError::ClickHouse("can not support Decimal".to_string())) - } + risingwave_common::types::DataType::Decimal => Ok(ck_column.r#type.contains("Decimal")), risingwave_common::types::DataType::Date => Ok(ck_column.r#type.contains("Date32")), risingwave_common::types::DataType::Varchar => Ok(ck_column.r#type.contains("String")), risingwave_common::types::DataType::Time => Err(SinkError::ClickHouse( @@ -232,7 +302,7 @@ impl ClickHouseSink { Ok(ck_column.r#type.contains("UInt64") | ck_column.r#type.contains("Int64")) } risingwave_common::types::DataType::Int256 => Err(SinkError::ClickHouse( - "clickhouse can not support Interval".to_string(), + "clickhouse can not support Int256".to_string(), )), }; if !is_match? { @@ -264,20 +334,15 @@ impl Sink for ClickHouseSink { // check reachability let client = self.config.common.build_client()?; - let query_column = "select distinct ?fields from system.columns where database = ? and table = ? order by ?".to_string(); - let clickhouse_column = client - .query(&query_column) - .bind(self.config.common.database.clone()) - .bind(self.config.common.table.clone()) - .bind("position") - .fetch_all::() - .await?; - if clickhouse_column.is_empty() { - return Err(SinkError::ClickHouse(format!( - "table {:?}.{:?} is not find in clickhouse", - self.config.common.database, self.config.common.table - ))); + + let (clickhouse_column, clickhouse_engine) = + query_column_engine_from_ck(client, &self.config).await?; + + if !self.is_append_only && !clickhouse_engine.is_collapsing_engine() { + return Err(SinkError::ClickHouse( + "If you want to use upsert, please modify your engine is `VersionedCollapsingMergeTree` or `CollapsingMergeTree` in ClickHouse".to_owned())); } + self.check_column_name_and_type(&clickhouse_column)?; if !self.is_append_only { self.check_pk_match(&clickhouse_column)?; @@ -300,17 +365,20 @@ pub struct ClickHouseSinkWriter { pub config: ClickHouseConfig, schema: Schema, pk_indices: Vec, - client: Client, + client: ClickHouseClient, is_append_only: bool, // Save some features of the clickhouse column type column_correct_vec: Vec, - clickhouse_fields_name: Vec, + rw_fields_name_after_calibration: Vec, + clickhouse_engine: ClickHouseEngine, } #[derive(Debug)] struct ClickHouseSchemaFeature { can_null: bool, // Time accuracy in clickhouse for rw and ck conversions accuracy_time: u8, + + accuracy_decimal: (u8, u8), } impl ClickHouseSinkWriter { @@ -320,25 +388,23 @@ impl ClickHouseSinkWriter { pk_indices: Vec, is_append_only: bool, ) -> Result { - if !is_append_only { - tracing::warn!("Update and delete are not recommended because of their impact on clickhouse performance."); - } let client = config.common.build_client()?; - let query_column = "select distinct ?fields from system.columns where database = ? and table = ? order by position".to_string(); - let clickhouse_column = client - .query(&query_column) - .bind(config.common.database.clone()) - .bind(config.common.table.clone()) - .fetch_all::() - .await?; + + let (clickhouse_column, clickhouse_engine) = + query_column_engine_from_ck(client.clone(), &config).await?; + let column_correct_vec: Result> = clickhouse_column .iter() .map(Self::build_column_correct_vec) .collect(); - let clickhouse_fields_name = build_fields_name_type_from_schema(&schema)? + let mut rw_fields_name_after_calibration = build_fields_name_type_from_schema(&schema)? .iter() .map(|(a, _)| a.clone()) .collect_vec(); + + if let Some(sign) = clickhouse_engine.get_sign_name() { + rw_fields_name_after_calibration.push(sign); + } Ok(Self { config, schema, @@ -346,7 +412,8 @@ impl ClickHouseSinkWriter { client, is_append_only, column_correct_vec: column_correct_vec?, - clickhouse_fields_name, + rw_fields_name_after_calibration, + clickhouse_engine, }) } @@ -368,138 +435,84 @@ impl ClickHouseSinkWriter { } else { 0_u8 }; + let accuracy_decimal = if ck_column.r#type.contains("Decimal(") { + let decimal_all = ck_column + .r#type + .split("Decimal(") + .last() + .ok_or_else(|| SinkError::ClickHouse("must have last".to_string()))? + .split(')') + .next() + .ok_or_else(|| SinkError::ClickHouse("must have next".to_string()))? + .split(", ") + .collect_vec(); + let length = decimal_all + .first() + .ok_or_else(|| SinkError::ClickHouse("must have next".to_string()))? + .parse::() + .map_err(|e| SinkError::ClickHouse(format!("clickhouse sink error {}", e)))?; + + if length > 38 { + return Err(SinkError::ClickHouse( + "RW don't support Decimal256".to_string(), + )); + } + + let scale = decimal_all + .last() + .ok_or_else(|| SinkError::ClickHouse("must have next".to_string()))? + .parse::() + .map_err(|e| SinkError::ClickHouse(format!("clickhouse sink error {}", e)))?; + (length, scale) + } else { + (0_u8, 0_u8) + }; Ok(ClickHouseSchemaFeature { can_null, accuracy_time, + accuracy_decimal, }) } - async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { + async fn write(&mut self, chunk: StreamChunk) -> Result<()> { let mut insert = self.client.insert_with_fields_name( &self.config.common.table, - self.clickhouse_fields_name.clone(), + self.rw_fields_name_after_calibration.clone(), )?; for (op, row) in chunk.rows() { - if op != Op::Insert { - tracing::warn!( - "append only click house sink receive an {:?} which will be ignored.", - op - ); - continue; - } let mut clickhouse_filed_vec = vec![]; for (index, data) in row.iter().enumerate() { clickhouse_filed_vec.extend(ClickHouseFieldWithNull::from_scalar_ref( data, &self.column_correct_vec, index, - true, )?); } - let clickhouse_column = ClickHouseColumn { - row: clickhouse_filed_vec, - }; - insert.write(&clickhouse_column).await?; - } - insert.end().await?; - Ok(()) - } - - async fn upsert(&mut self, chunk: StreamChunk) -> Result<()> { - let get_pk_names_and_data = |row: RowRef<'_>, index: usize| { - let pk_names = self - .schema - .names() - .iter() - .cloned() - .enumerate() - .filter(|(index, _)| self.pk_indices.contains(index)) - .map(|(_, b)| b) - .collect_vec(); - let mut pk_data = vec![]; - for pk_index in &self.pk_indices { - if let ClickHouseFieldWithNull::WithoutSome(v) = - ClickHouseFieldWithNull::from_scalar_ref( - row.datum_at(*pk_index), - &self.column_correct_vec, - index, - false, - )? - .pop() - .unwrap() - { - pk_data.push(v) - } else { - return Err(SinkError::ClickHouse("pk can not be null".to_string())); - } - } - Ok((pk_names, pk_data)) - }; - - for (index, (op, row)) in chunk.rows().enumerate() { match op { - Op::Insert => { - let mut insert = self.client.insert_with_fields_name( - &self.config.common.table, - self.clickhouse_fields_name.clone(), - )?; - let mut clickhouse_filed_vec = vec![]; - for (index, data) in row.iter().enumerate() { - clickhouse_filed_vec.extend(ClickHouseFieldWithNull::from_scalar_ref( - data, - &self.column_correct_vec, - index, - true, - )?); + Op::Insert | Op::UpdateInsert => { + if self.clickhouse_engine.get_sign_name().is_some() { + clickhouse_filed_vec.push(ClickHouseFieldWithNull::WithoutSome( + ClickHouseField::Int8(1), + )); } - let clickhouse_column = ClickHouseColumn { - row: clickhouse_filed_vec, - }; - insert.write(&clickhouse_column).await?; - insert.end().await?; } - Op::Delete => { - let (delete_pk_names, delete_pk_data) = get_pk_names_and_data(row, index)?; - self.client - .delete(&self.config.common.table, delete_pk_names) - .delete(delete_pk_data) - .await?; - } - Op::UpdateDelete => continue, - Op::UpdateInsert => { - let (update_pk_names, update_pk_data) = get_pk_names_and_data(row, index)?; - let mut clickhouse_update_filed_vec = vec![]; - for (index, data) in row.iter().enumerate() { - if !self.pk_indices.contains(&index) { - clickhouse_update_filed_vec.extend( - ClickHouseFieldWithNull::from_scalar_ref( - data, - &self.column_correct_vec, - index, - false, - )?, - ); - } + Op::Delete | Op::UpdateDelete => { + if !self.clickhouse_engine.is_collapsing_engine() { + return Err(SinkError::ClickHouse( + "Clickhouse engine don't support upsert".to_string(), + )); } - // Get the names of the columns excluding pk, and use them to update. - let fields_name_update = self - .clickhouse_fields_name - .iter() - .filter(|n| !update_pk_names.contains(n)) - .map(|s| s.to_string()) - .collect_vec(); - - let update = self.client.update( - &self.config.common.table, - update_pk_names, - fields_name_update.clone(), - ); - update - .update_fields(clickhouse_update_filed_vec, update_pk_data) - .await?; + clickhouse_filed_vec.push(ClickHouseFieldWithNull::WithoutSome( + ClickHouseField::Int8(-1), + )) } } + let clickhouse_column = ClickHouseColumn { + row: clickhouse_filed_vec, + }; + insert.write(&clickhouse_column).await?; } + insert.end().await?; Ok(()) } } @@ -510,11 +523,7 @@ impl AsyncTruncateSinkWriter for ClickHouseSinkWriter { chunk: StreamChunk, _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, ) -> Result<()> { - if self.is_append_only { - self.append_only(chunk).await - } else { - self.upsert(chunk).await - } + self.write(chunk).await } } @@ -525,6 +534,48 @@ struct SystemColumn { is_in_primary_key: u8, } +#[derive(ClickHouseRow, Deserialize)] +struct ClickhouseQueryEngine { + name: String, + engine: String, + create_table_query: String, +} + +async fn query_column_engine_from_ck( + client: ClickHouseClient, + config: &ClickHouseConfig, +) -> Result<(Vec, ClickHouseEngine)> { + let query_engine = QUERY_ENGINE; + let query_column = QUERY_COLUMN; + + let clickhouse_engine = client + .query(query_engine) + .bind(config.common.database.clone()) + .bind(config.common.table.clone()) + .fetch_all::() + .await?; + let mut clickhouse_column = client + .query(query_column) + .bind(config.common.database.clone()) + .bind(config.common.table.clone()) + .bind("position") + .fetch_all::() + .await?; + if clickhouse_engine.is_empty() || clickhouse_column.is_empty() { + return Err(SinkError::ClickHouse(format!( + "table {:?}.{:?} is not find in clickhouse", + config.common.database, config.common.table + ))); + } + + let clickhouse_engine = ClickHouseEngine::from_query_engine(clickhouse_engine.get(0).unwrap())?; + + if let Some(sign) = &clickhouse_engine.get_sign_name() { + clickhouse_column.retain(|a| sign.ne(&a.name)) + } + Ok((clickhouse_column, clickhouse_engine)) +} + /// Serialize this structure to simulate the `struct` call clickhouse interface #[derive(ClickHouseRow, Debug)] struct ClickHouseColumn { @@ -543,6 +594,26 @@ enum ClickHouseField { String(String), Bool(bool), List(Vec), + Int8(i8), + Decimal(ClickHouseDecimal), +} +#[derive(Debug)] +enum ClickHouseDecimal { + Decimal32(i32), + Decimal64(i64), + Decimal128(i128), +} +impl Serialize for ClickHouseDecimal { + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + match self { + ClickHouseDecimal::Decimal32(v) => serializer.serialize_i32(*v), + ClickHouseDecimal::Decimal64(v) => serializer.serialize_i64(*v), + ClickHouseDecimal::Decimal128(v) => serializer.serialize_i128(*v), + } + } } /// Enum that support clickhouse nullable @@ -558,7 +629,6 @@ impl ClickHouseFieldWithNull { data: Option>, clickhouse_schema_feature_vec: &Vec, clickhouse_schema_feature_index: usize, - is_insert: bool, ) -> Result> { let clickhouse_schema_feature = clickhouse_schema_feature_vec .get(clickhouse_schema_feature_index) @@ -586,8 +656,29 @@ impl ClickHouseFieldWithNull { ScalarRefImpl::Float64(v) => ClickHouseField::Float64(v.into_inner()), ScalarRefImpl::Utf8(v) => ClickHouseField::String(v.to_string()), ScalarRefImpl::Bool(v) => ClickHouseField::Bool(v), - ScalarRefImpl::Decimal(_) => { - return Err(SinkError::ClickHouse("can not support Decimal".to_string())) + ScalarRefImpl::Decimal(d) => { + if let Decimal::Normalized(d) = d { + let scale = + clickhouse_schema_feature.accuracy_decimal.1 as i32 - d.scale() as i32; + + let scale = if scale < 0 { + d.mantissa() / 10_i128.pow(scale.unsigned_abs()) + } else { + d.mantissa() * 10_i128.pow(scale as u32) + }; + + if clickhouse_schema_feature.accuracy_decimal.0 <= 9 { + ClickHouseField::Decimal(ClickHouseDecimal::Decimal32(scale as i32)) + } else if clickhouse_schema_feature.accuracy_decimal.0 <= 18 { + ClickHouseField::Decimal(ClickHouseDecimal::Decimal64(scale as i64)) + } else { + ClickHouseField::Decimal(ClickHouseDecimal::Decimal128(scale)) + } + } else { + return Err(SinkError::ClickHouse( + "clickhouse can not support Decimal NAN,-INF and INF".to_string(), + )); + } } ScalarRefImpl::Interval(_) => { return Err(SinkError::ClickHouse( @@ -604,14 +695,9 @@ impl ClickHouseFieldWithNull { )) } ScalarRefImpl::Timestamp(v) => { - if is_insert { - let time = v.get_timestamp_nanos() - / 10_i32.pow((9 - clickhouse_schema_feature.accuracy_time).into()) as i64; - ClickHouseField::Int64(time) - } else { - let time = v.truncate_micros().to_string(); - ClickHouseField::String(time) - } + let time = v.get_timestamp_nanos() + / 10_i32.pow((9 - clickhouse_schema_feature.accuracy_time).into()) as i64; + ClickHouseField::Int64(time) } ScalarRefImpl::Timestamptz(_) => { return Err(SinkError::ClickHouse( @@ -630,7 +716,6 @@ impl ClickHouseFieldWithNull { field, clickhouse_schema_feature_vec, clickhouse_schema_feature_index + index, - is_insert, )?; struct_vec.push(ClickHouseFieldWithNull::WithoutSome(ClickHouseField::List( a, @@ -645,7 +730,6 @@ impl ClickHouseFieldWithNull { i, clickhouse_schema_feature_vec, clickhouse_schema_feature_index, - is_insert, )?) } return Ok(vec![ClickHouseFieldWithNull::WithoutSome( @@ -658,9 +742,7 @@ impl ClickHouseFieldWithNull { )) } }; - // Insert needs to be serialized with `Some`, update doesn't need to be serialized with - // `Some` - let data = if is_insert && clickhouse_schema_feature.can_null { + let data = if clickhouse_schema_feature.can_null { vec![ClickHouseFieldWithNull::WithSome(data)] } else { vec![ClickHouseFieldWithNull::WithoutSome(data)] @@ -690,6 +772,8 @@ impl Serialize for ClickHouseField { } s.end() } + ClickHouseField::Decimal(v) => v.serialize(serializer), + ClickHouseField::Int8(v) => serializer.serialize_i8(*v), } } } From 39f71a171b8a4cece71902dde973ff58d4215fcf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 25 Oct 2023 06:26:27 +0000 Subject: [PATCH 19/48] chore(deps): Bump the arrow group with 6 updates (#13012) Signed-off-by: dependabot[bot] Signed-off-by: Runji Wang Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Runji Wang --- .github/dependabot.yml | 1 + Cargo.lock | 61 ++++++++++++++++++++++++------------------ Cargo.toml | 14 +++++----- 3 files changed, 43 insertions(+), 33 deletions(-) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index b2d58279b5290..51242d0425e28 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -16,6 +16,7 @@ updates: arrow: patterns: - "arrow*" + - "parquet" aws: patterns: - "aws*" diff --git a/Cargo.lock b/Cargo.lock index 99a5a675c0342..3c9dbe9602508 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -243,9 +243,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow-arith" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc1d4e368e87ad9ee64f28b9577a3834ce10fe2703a26b28417d485bbbdff956" +checksum = "c5c3d17fc5b006e7beeaebfb1d2edfc92398b981f82d9744130437909b72a468" dependencies = [ "arrow-array", "arrow-buffer", @@ -258,9 +258,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d02efa7253ede102d45a4e802a129e83bcc3f49884cab795b1ac223918e4318d" +checksum = "55705ada5cdde4cb0f202ffa6aa756637e33fea30e13d8d0d0fd6a24ffcee1e3" dependencies = [ "ahash 0.8.3", "arrow-buffer", @@ -274,9 +274,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fda119225204141138cb0541c692fbfef0e875ba01bfdeaed09e9d354f9d6195" +checksum = "a722f90a09b94f295ab7102542e97199d3500128843446ef63e410ad546c5333" dependencies = [ "bytes", "half 2.3.1", @@ -285,9 +285,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d825d51b9968868d50bc5af92388754056796dbc62a4e25307d588a1fc84dee" +checksum = "af01fc1a06f6f2baf31a04776156d47f9f31ca5939fe6d00cd7a059f95a46ff1" dependencies = [ "arrow-array", "arrow-buffer", @@ -302,9 +302,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "475a4c3699c8b4095ca61cecf15da6f67841847a5f5aac983ccb9a377d02f73a" +checksum = "d0a547195e607e625e7fafa1a7269b8df1a4a612c919efd9b26bd86e74538f3a" dependencies = [ "arrow-buffer", "arrow-schema", @@ -314,9 +314,9 @@ dependencies = [ [[package]] name = "arrow-flight" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd938ea4a0e8d0db2b9f47ebba792f73f6188f4289707caeaf93a3be705e5ed5" +checksum = "c58645809ced5acd6243e89a63ae8535a2ab50d780affcd7efe8c7473a0da661" dependencies = [ "arrow-array", "arrow-buffer", @@ -334,9 +334,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1248005c8ac549f869b7a840859d942bf62471479c1a2d82659d453eebcd166a" +checksum = "e36bf091502ab7e37775ff448413ef1ffff28ff93789acb669fffdd51b394d51" dependencies = [ "arrow-array", "arrow-buffer", @@ -348,9 +348,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03b87aa408ea6a6300e49eb2eba0c032c88ed9dc19e0a9948489c55efdca71f4" +checksum = "4502123d2397319f3a13688432bc678c61cb1582f2daa01253186da650bf5841" dependencies = [ "arrow-array", "arrow-buffer", @@ -363,9 +363,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "114a348ab581e7c9b6908fcab23cb39ff9f060eb19e72b13f8fb8eaa37f65d22" +checksum = "249fc5a07906ab3f3536a6e9f118ec2883fbcde398a97a5ba70053f0276abda4" dependencies = [ "ahash 0.8.3", "arrow-array", @@ -378,15 +378,15 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d1d179c117b158853e0101bfbed5615e86fe97ee356b4af901f1c5001e1ce4b" +checksum = "9d7a8c3f97f5ef6abd862155a6f39aaba36b029322462d72bbcfa69782a50614" [[package]] name = "arrow-select" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5c71e003202e67e9db139e5278c79f5520bb79922261dfe140e4637ee8b6108" +checksum = "f868f4a5001429e20f7c1994b5cd1aa68b82e3db8cf96c559cdb56dc8be21410" dependencies = [ "ahash 0.8.3", "arrow-array", @@ -4440,6 +4440,15 @@ dependencies = [ "libc", ] +[[package]] +name = "lz4_flex" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3ea9b256699eda7b0387ffbc776dd625e28bde3918446381781245b7a50349d8" +dependencies = [ + "twox-hash", +] + [[package]] name = "lzma-sys" version = "0.1.20" @@ -5618,9 +5627,9 @@ dependencies = [ [[package]] name = "parquet" -version = "47.0.0" +version = "48.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0463cc3b256d5f50408c49a4be3a16674f4c8ceef60941709620a062b1f6bf4d" +checksum = "239229e6a668ab50c61de3dce61cf0fa1069345f7aa0f4c934491f92205a4945" dependencies = [ "ahash 0.8.3", "arrow-array", @@ -5637,7 +5646,7 @@ dependencies = [ "flate2", "futures", "hashbrown 0.14.0", - "lz4", + "lz4_flex", "num", "num-bigint", "paste", @@ -5646,7 +5655,7 @@ dependencies = [ "thrift", "tokio", "twox-hash", - "zstd 0.12.4", + "zstd 0.13.0", ] [[package]] diff --git a/Cargo.toml b/Cargo.toml index f8a9b7d0e2fa5..f0dd2d0443b9e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -112,13 +112,13 @@ tonic = { package = "madsim-tonic", version = "0.4.0" } tonic-build = { package = "madsim-tonic-build", version = "0.4.2" } prost = { version = "0.12" } icelake = { git = "https://github.com/icelake-io/icelake", rev = "16dab0e36ab337e58ee8002d828def2d212fa116" } -arrow-array = "47" -arrow-cast = "47" -arrow-schema = "47" -arrow-buffer = "47" -arrow-flight = "47" -arrow-select = "47" -arrow-ord = "47" +arrow-array = "48" +arrow-cast = "48" +arrow-schema = "48" +arrow-buffer = "48" +arrow-flight = "48" +arrow-select = "48" +arrow-ord = "48" tikv-jemalloc-ctl = { git = "https://github.com/risingwavelabs/jemallocator.git", rev = "64a2d9" } tikv-jemallocator = { git = "https://github.com/risingwavelabs/jemallocator.git", features = [ "profiling", From 726964aa564de584547af297c780812af2a5afa0 Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Wed, 25 Oct 2023 15:22:50 +0800 Subject: [PATCH 20/48] fix(batch): avoid empty chunk in source executor (#13046) --- src/batch/src/executor/source.rs | 5 ++++- src/common/src/util/chunk_coalesce.rs | 7 ++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/batch/src/executor/source.rs b/src/batch/src/executor/source.rs index 8bf9fc5b7e610..ae3fc7056a6a6 100644 --- a/src/batch/src/executor/source.rs +++ b/src/batch/src/executor/source.rs @@ -159,7 +159,10 @@ impl SourceExecutor { for chunk in stream { match chunk { Ok(chunk) => { - yield covert_stream_chunk_to_batch_chunk(chunk.chunk)?; + let data_chunk = covert_stream_chunk_to_batch_chunk(chunk.chunk)?; + if data_chunk.capacity() > 0 { + yield data_chunk; + } } Err(e) => { return Err(e); diff --git a/src/common/src/util/chunk_coalesce.rs b/src/common/src/util/chunk_coalesce.rs index 9a41fc83e8f0e..3bd56b19e434d 100644 --- a/src/common/src/util/chunk_coalesce.rs +++ b/src/common/src/util/chunk_coalesce.rs @@ -285,7 +285,12 @@ impl SlicedDataChunk { } pub fn with_offset_checked(data_chunk: DataChunk, offset: usize) -> Self { - assert!(offset < data_chunk.capacity()); + assert!( + offset < data_chunk.capacity(), + "offset {}, data_chunk capacity {}", + offset, + data_chunk.capacity() + ); Self { data_chunk, offset } } From e48547dac8c7083a13d9537f9f16e4cc855bc4e7 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Wed, 25 Oct 2023 16:41:16 +0800 Subject: [PATCH 21/48] refactor(type): switch jsonb to flat representation (#12952) Signed-off-by: Runji Wang --- Cargo.lock | 14 ++ src/common/Cargo.toml | 1 + src/common/src/array/jsonb_array.rs | 128 ++++------- src/common/src/array/proto_reader.rs | 4 +- src/common/src/array/value_reader.rs | 15 +- src/common/src/test_utils/rand_array.rs | 2 +- src/common/src/types/jsonb.rs | 261 ++++++++++++++--------- src/common/src/types/mod.rs | 4 +- src/expr/impl/Cargo.toml | 1 + src/expr/impl/benches/expr.rs | 31 ++- src/expr/impl/src/aggregate/jsonb_agg.rs | 150 +++++++++++-- src/expr/impl/src/scalar/jsonb_concat.rs | 28 +-- src/expr/macro/src/gen.rs | 86 +++++--- src/expr/macro/src/lib.rs | 2 + src/expr/macro/src/parse.rs | 30 ++- 15 files changed, 472 insertions(+), 285 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3c9dbe9602508..cf1f1f0e493e9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4117,6 +4117,18 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "jsonbb" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44376417b2ff0cd879b5c84976fa9e0855c316321b4e0502e33e52963bf84f74" +dependencies = [ + "bytes", + "serde", + "serde_json", + "smallvec", +] + [[package]] name = "jsonschema-transpiler" version = "1.10.0" @@ -7073,6 +7085,7 @@ dependencies = [ "hytra", "itertools 0.11.0", "itoa", + "jsonbb", "libc", "lru 0.7.6", "mach2", @@ -7450,6 +7463,7 @@ dependencies = [ "futures-util", "hex", "itertools 0.11.0", + "jsonbb", "madsim-tokio", "md5", "num-traits", diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index ddd1fe5a33cdb..168ba836d4c1b 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -49,6 +49,7 @@ hyper = "0.14" hytra = { workspace = true } itertools = "0.11" itoa = "1.0" +jsonbb = "0.1" lru = { git = "https://github.com/risingwavelabs/lru-rs.git", rev = "cb2d7c7" } memcomparable = { version = "0.2", features = ["decimal"] } num-integer = "0.1" diff --git a/src/common/src/array/jsonb_array.rs b/src/common/src/array/jsonb_array.rs index 0e9ba7c48511d..3c4ca23fff04e 100644 --- a/src/common/src/array/jsonb_array.rs +++ b/src/common/src/array/jsonb_array.rs @@ -12,36 +12,35 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::mem::size_of; - use risingwave_pb::data::{PbArray, PbArrayType}; -use serde_json::Value; -use super::{Array, ArrayBuilder}; +use super::{Array, ArrayBuilder, ArrayImpl, ArrayResult}; use crate::buffer::{Bitmap, BitmapBuilder}; use crate::estimate_size::EstimateSize; -use crate::types::{DataType, JsonbRef, JsonbVal, F32, F64}; -use crate::util::iter_util::ZipEqFast; +use crate::types::{DataType, JsonbRef, JsonbVal, Scalar}; #[derive(Debug)] pub struct JsonbArrayBuilder { bitmap: BitmapBuilder, - data: Vec, + builder: jsonbb::Builder, } #[derive(Debug, Clone, PartialEq, Eq)] pub struct JsonbArray { bitmap: Bitmap, - data: Vec, + /// Elements are stored as a single JSONB array value. + data: jsonbb::Value, } impl ArrayBuilder for JsonbArrayBuilder { type ArrayType = JsonbArray; fn new(capacity: usize) -> Self { + let mut builder = jsonbb::Builder::with_capacity(capacity); + builder.begin_array(); Self { bitmap: BitmapBuilder::with_capacity(capacity), - data: Vec::with_capacity(capacity), + builder, } } @@ -54,13 +53,15 @@ impl ArrayBuilder for JsonbArrayBuilder { match value { Some(x) => { self.bitmap.append_n(n, true); - self.data - .extend(std::iter::repeat(x).take(n).map(|x| x.0.clone())); + for _ in 0..n { + self.builder.add_value(x.0); + } } None => { self.bitmap.append_n(n, false); - self.data - .extend(std::iter::repeat(*JsonbVal::dummy().0).take(n)); + for _ in 0..n { + self.builder.add_null(); + } } } } @@ -69,29 +70,44 @@ impl ArrayBuilder for JsonbArrayBuilder { for bit in other.bitmap.iter() { self.bitmap.append(bit); } - self.data.extend_from_slice(&other.data); + for value in other.data.as_array().unwrap().iter() { + self.builder.add_value(value); + } } fn pop(&mut self) -> Option<()> { - self.data.pop().map(|_| self.bitmap.pop().unwrap()) + self.bitmap.pop()?; + self.builder.pop(); + Some(()) } fn len(&self) -> usize { self.bitmap.len() } - fn finish(self) -> Self::ArrayType { + fn finish(mut self) -> Self::ArrayType { + self.builder.end_array(); Self::ArrayType { bitmap: self.bitmap.finish(), - data: self.data, + data: self.builder.finish(), } } } -impl JsonbArrayBuilder { - pub fn append_move(&mut self, value: JsonbVal) { - self.bitmap.append(true); - self.data.push(*value.0); +impl JsonbArray { + /// Loads a `JsonbArray` from a protobuf array. + /// + /// See also `JsonbArray::to_protobuf`. + pub fn from_protobuf(array: &PbArray) -> ArrayResult { + ensure!( + array.values.len() == 1, + "Must have exactly 1 buffer in a jsonb array" + ); + let arr = JsonbArray { + bitmap: array.get_null_bitmap()?.into(), + data: jsonbb::Value::from_bytes(&array.values[0].body), + }; + Ok(arr.into()) } } @@ -101,52 +117,23 @@ impl Array for JsonbArray { type RefItem<'a> = JsonbRef<'a>; unsafe fn raw_value_at_unchecked(&self, idx: usize) -> Self::RefItem<'_> { - JsonbRef(self.data.get_unchecked(idx)) + JsonbRef(self.data.as_array().unwrap().get(idx).unwrap()) } fn len(&self) -> usize { - self.data.len() + self.bitmap.len() } fn to_protobuf(&self) -> PbArray { - // The memory layout contains `serde_json::Value` trees, but in protobuf we transmit this as - // variable length bytes in value encoding. That is, one buffer of length n+1 containing - // start and end offsets into the 2nd buffer containing all value bytes concatenated. - use risingwave_pb::common::buffer::CompressionType; use risingwave_pb::common::Buffer; - let mut offset_buffer = - Vec::::with_capacity((1 + self.data.len()) * std::mem::size_of::()); - let mut data_buffer = Vec::::with_capacity(self.data.len()); - - let mut offset = 0; - for (v, not_null) in self.data.iter().zip_eq_fast(self.null_bitmap().iter()) { - if !not_null { - continue; - } - let d = JsonbRef(v).value_serialize(); - offset_buffer.extend_from_slice(&(offset as u64).to_be_bytes()); - data_buffer.extend_from_slice(&d); - offset += d.len(); - } - offset_buffer.extend_from_slice(&(offset as u64).to_be_bytes()); - - let values = vec![ - Buffer { - compression: CompressionType::None as i32, - body: offset_buffer, - }, - Buffer { - compression: CompressionType::None as i32, - body: data_buffer, - }, - ]; - - let null_bitmap = self.null_bitmap().to_protobuf(); PbArray { - null_bitmap: Some(null_bitmap), - values, + null_bitmap: Some(self.null_bitmap().to_protobuf()), + values: vec![Buffer { + compression: CompressionType::None as i32, + body: self.data.as_bytes().to_vec(), + }], array_type: PbArrayType::Jsonb as i32, struct_array_data: None, list_array_data: None, @@ -176,7 +163,7 @@ impl FromIterator> for JsonbArray { let mut builder = ::Builder::new(iter.size_hint().0); for i in iter { match i { - Some(x) => builder.append_move(x), + Some(x) => builder.append(Some(x.as_scalar_ref())), None => builder.append(None), } } @@ -190,31 +177,8 @@ impl FromIterator for JsonbArray { } } -// TODO: We need to fix this later. impl EstimateSize for JsonbArray { fn estimated_heap_size(&self) -> usize { - self.bitmap.estimated_heap_size() + self.data.capacity() * size_of::() - } -} - -impl From for Value { - fn from(v: F32) -> Value { - serde_json::Number::from_f64(v.0 as f64) - .expect("todo: convert Inf/NaN to jsonb") - .into() - } -} - -impl From for Value { - fn from(v: F64) -> Value { - serde_json::Number::from_f64(v.0) - .expect("todo: convert Inf/NaN to jsonb") - .into() - } -} - -impl From> for Value { - fn from(v: JsonbRef<'_>) -> Value { - v.0.clone() + self.bitmap.estimated_heap_size() + self.data.capacity() } } diff --git a/src/common/src/array/proto_reader.rs b/src/common/src/array/proto_reader.rs index 55d505343dadd..4ca6bf7b70d05 100644 --- a/src/common/src/array/proto_reader.rs +++ b/src/common/src/array/proto_reader.rs @@ -52,9 +52,7 @@ impl ArrayImpl { PbArrayType::Timestamp => read_timestamp_array(array, cardinality)?, PbArrayType::Timestamptz => read_timestamptz_array(array, cardinality)?, PbArrayType::Interval => read_interval_array(array, cardinality)?, - PbArrayType::Jsonb => { - read_string_array::(array, cardinality)? - } + PbArrayType::Jsonb => JsonbArray::from_protobuf(array)?, PbArrayType::Struct => StructArray::from_protobuf(array)?, PbArrayType::List => ListArray::from_protobuf(array)?, PbArrayType::Unspecified => unreachable!(), diff --git a/src/common/src/array/value_reader.rs b/src/common/src/array/value_reader.rs index 96ed7c31b88aa..45db47f23242b 100644 --- a/src/common/src/array/value_reader.rs +++ b/src/common/src/array/value_reader.rs @@ -19,8 +19,7 @@ use byteorder::{BigEndian, ReadBytesExt}; use super::ArrayResult; use crate::array::{ - ArrayBuilder, BytesArrayBuilder, JsonbArrayBuilder, PrimitiveArrayItemType, Serial, - Utf8ArrayBuilder, + ArrayBuilder, BytesArrayBuilder, PrimitiveArrayItemType, Serial, Utf8ArrayBuilder, }; use crate::types::{Decimal, F32, F64}; @@ -89,15 +88,3 @@ impl VarSizedValueReader for BytesValueReader { Ok(()) } } - -pub struct JsonbValueReader; - -impl VarSizedValueReader for JsonbValueReader { - fn read(buf: &[u8], builder: &mut JsonbArrayBuilder) -> ArrayResult<()> { - let Some(v) = super::JsonbVal::value_deserialize(buf) else { - bail!("failed to read jsonb from bytes"); - }; - builder.append_move(v); - Ok(()) - } -} diff --git a/src/common/src/test_utils/rand_array.rs b/src/common/src/test_utils/rand_array.rs index 70d0cb73d4dfa..f2dd8ad42854b 100644 --- a/src/common/src/test_utils/rand_array.rs +++ b/src/common/src/test_utils/rand_array.rs @@ -135,7 +135,7 @@ impl RandValue for Int256 { impl RandValue for JsonbVal { fn rand_value(_rand: &mut R) -> Self { - JsonbVal::dummy() + JsonbVal::null() } } diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index 7f4c002037060..590b693e47891 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -15,23 +15,21 @@ use std::fmt; use std::hash::Hash; -use postgres_types::{FromSql as _, ToSql as _, Type}; -use serde_json::Value; +use bytes::Buf; +use jsonbb::{Value, ValueRef}; use crate::estimate_size::EstimateSize; -use crate::types::{Scalar, ScalarRef}; +use crate::types::{Scalar, ScalarRef, F32, F64}; -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct JsonbVal(pub(crate) Box); // The `Box` is just to keep `size_of::` smaller. +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct JsonbVal(pub(crate) Value); -#[derive(Debug, Copy, Clone, PartialEq, Eq)] -pub struct JsonbRef<'a>(pub(crate) &'a Value); +#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)] +pub struct JsonbRef<'a>(pub(crate) ValueRef<'a>); impl EstimateSize for JsonbVal { fn estimated_heap_size(&self) -> usize { - // https://github.com/risingwavelabs/risingwave/issues/8957 - // FIXME: correctly handle jsonb size - 0 + self.0.capacity() } } @@ -63,7 +61,7 @@ impl<'a> ScalarRef<'a> for JsonbRef<'a> { type ScalarType = JsonbVal; fn to_owned_scalar(&self) -> Self::ScalarType { - JsonbVal(self.0.clone().into()) + JsonbVal(self.0.into()) } fn hash_scalar(&self, state: &mut H) { @@ -71,22 +69,6 @@ impl<'a> ScalarRef<'a> for JsonbRef<'a> { } } -impl Hash for JsonbRef<'_> { - fn hash(&self, state: &mut H) { - // We do not intend to support hashing `jsonb` type. - // Before #7981 is done, we do not panic but just hash its string representation. - // Note that `serde_json` without feature `preserve_order` uses `BTreeMap` for json object. - // So its string form always have keys sorted. - self.0.to_string().hash(state) - } -} - -impl Hash for JsonbVal { - fn hash(&self, state: &mut H) { - self.0.to_string().hash(state) - } -} - impl PartialOrd for JsonbVal { fn partial_cmp(&self, other: &Self) -> Option { Some(self.cmp(other)) @@ -160,9 +142,7 @@ impl crate::types::to_binary::ToBinary for JsonbRef<'_> { &self, _ty: &crate::types::DataType, ) -> crate::error::Result> { - let mut output = bytes::BytesMut::new(); - self.0.to_sql(&Type::JSONB, &mut output).unwrap(); - Ok(Some(output.freeze())) + Ok(Some(self.value_serialize().into())) } } @@ -170,43 +150,130 @@ impl std::str::FromStr for JsonbVal { type Err = ::Err; fn from_str(s: &str) -> Result { - let v: Value = s.parse()?; - Ok(Self(v.into())) + Ok(Self(s.parse()?)) } } impl JsonbVal { - /// Constructs a value without specific meaning. Usually used as a lightweight placeholder. - pub fn dummy() -> Self { - Self(Value::Null.into()) + /// Returns a jsonb `null`. + pub fn null() -> Self { + Self(Value::null()) + } + + /// Returns an empty array `[]`. + pub fn empty_array() -> Self { + Self(Value::array([])) + } + + /// Returns an empty array `{}`. + pub fn empty_object() -> Self { + Self(Value::object([])) } + /// Deserialize from a memcomparable encoding. pub fn memcmp_deserialize( deserializer: &mut memcomparable::Deserializer, ) -> memcomparable::Result { - let v: Value = ::deserialize(deserializer)? + let v = ::deserialize(deserializer)? .parse() .map_err(|_| memcomparable::Error::Message("invalid json".into()))?; - Ok(Self(v.into())) + Ok(Self(v)) + } + + /// Deserialize from a pgwire "BINARY" encoding. + pub fn value_deserialize(mut buf: &[u8]) -> Option { + if buf.is_empty() || buf.get_u8() != 1 { + return None; + } + Value::from_text(buf).ok().map(Self) + } + + /// Convert the value to a [`serde_json::Value`]. + pub fn take(self) -> serde_json::Value { + self.0.into() + } +} + +impl From for JsonbVal { + fn from(v: serde_json::Value) -> Self { + Self(v.into()) + } +} + +impl From for JsonbVal { + fn from(v: bool) -> Self { + Self(v.into()) + } +} + +impl From for JsonbVal { + fn from(v: i16) -> Self { + Self(v.into()) } +} - pub fn value_deserialize(buf: &[u8]) -> Option { - let v = Value::from_sql(&Type::JSONB, buf).ok()?; - Some(Self(v.into())) +impl From for JsonbVal { + fn from(v: i32) -> Self { + Self(v.into()) } +} - pub fn take(mut self) -> Value { - self.0.take() +impl From for JsonbVal { + fn from(v: i64) -> Self { + Self(v.into()) } +} - pub fn as_serde_mut(&mut self) -> &mut Value { - &mut self.0 +impl From for JsonbVal { + fn from(v: F32) -> Self { + if v.0 == f32::INFINITY { + Self("Infinity".into()) + } else if v.0 == f32::NEG_INFINITY { + Self("-Infinity".into()) + } else if v.0.is_nan() { + Self("NaN".into()) + } else { + Self(v.0.into()) + } + } +} + +// NOTE: Infinite or NaN values are not JSON numbers. They are stored as strings in Postgres. +impl From for JsonbVal { + fn from(v: F64) -> Self { + if v.0 == f64::INFINITY { + Self("Infinity".into()) + } else if v.0 == f64::NEG_INFINITY { + Self("-Infinity".into()) + } else if v.0.is_nan() { + Self("NaN".into()) + } else { + Self(v.0.into()) + } + } +} + +impl From<&str> for JsonbVal { + fn from(v: &str) -> Self { + Self(v.into()) + } +} + +impl From> for JsonbVal { + fn from(v: JsonbRef<'_>) -> Self { + Self(v.0.to_owned()) } } impl From for JsonbVal { fn from(v: Value) -> Self { - Self(v.into()) + Self(v) + } +} + +impl<'a> From> for ValueRef<'a> { + fn from(v: JsonbRef<'a>) -> Self { + v.0 } } @@ -221,49 +288,52 @@ impl<'a> JsonbRef<'a> { serde::Serialize::serialize(&s, serializer) } + /// Serialize to a pgwire "BINARY" encoding. pub fn value_serialize(&self) -> Vec { + use std::io::Write; // Reuse the pgwire "BINARY" encoding for jsonb type. // It is not truly binary, but one byte of version `1u8` followed by string form. // This version number helps us maintain compatibility when we switch to more efficient // encoding later. - let mut output = bytes::BytesMut::new(); - self.0.to_sql(&Type::JSONB, &mut output).unwrap(); - output.freeze().into() + let mut buf = Vec::with_capacity(self.0.capacity()); + buf.push(1); + write!(&mut buf, "{}", self.0).unwrap(); + buf } + /// Returns true if this is a jsonb `null`. pub fn is_jsonb_null(&self) -> bool { - matches!(self.0, Value::Null) + self.0.as_null().is_some() } + /// Returns the type name of this jsonb. + /// + /// Possible values are: `null`, `boolean`, `number`, `string`, `array`, `object`. pub fn type_name(&self) -> &'static str { match self.0 { - Value::Null => "null", - Value::Bool(_) => "boolean", - Value::Number(_) => "number", - Value::String(_) => "string", - Value::Array(_) => "array", - Value::Object(_) => "object", + ValueRef::Null => "null", + ValueRef::Bool(_) => "boolean", + ValueRef::Number(_) => "number", + ValueRef::String(_) => "string", + ValueRef::Array(_) => "array", + ValueRef::Object(_) => "object", } } + /// Returns the length of this json array. pub fn array_len(&self) -> Result { - match self.0 { - Value::Array(v) => Ok(v.len()), - _ => Err(format!( - "cannot get array length of a jsonb {}", - self.type_name() - )), - } + let array = self + .0 + .as_array() + .ok_or_else(|| format!("cannot get array length of a jsonb {}", self.type_name()))?; + Ok(array.len()) } + /// If the JSON is a boolean, returns the associated bool. pub fn as_bool(&self) -> Result { - match self.0 { - Value::Bool(v) => Ok(*v), - _ => Err(format!( - "cannot cast jsonb {} to type boolean", - self.type_name() - )), - } + self.0 + .as_bool() + .ok_or_else(|| format!("cannot cast jsonb {} to type boolean", self.type_name())) } /// Attempt to read jsonb as a JSON number. @@ -271,13 +341,11 @@ impl<'a> JsonbRef<'a> { /// According to RFC 8259, only number within IEEE 754 binary64 (double precision) has good /// interoperability. We do not support arbitrary precision like PostgreSQL `numeric` right now. pub fn as_number(&self) -> Result { - match self.0 { - Value::Number(v) => v.as_f64().ok_or_else(|| "jsonb number out of range".into()), - _ => Err(format!( - "cannot cast jsonb {} to type number", - self.type_name() - )), - } + self.0 + .as_number() + .ok_or_else(|| format!("cannot cast jsonb {} to type number", self.type_name()))? + .as_f64() + .ok_or_else(|| "jsonb number out of range".into()) } /// This is part of the `->>` or `#>>` syntax to access a child as string. @@ -291,9 +359,9 @@ impl<'a> JsonbRef<'a> { /// * Jsonb string is displayed with quotes but treated as its inner value here. pub fn force_str(&self, writer: &mut W) -> std::fmt::Result { match self.0 { - Value::String(v) => writer.write_str(v), - Value::Null => Ok(()), - Value::Bool(_) | Value::Number(_) | Value::Array(_) | Value::Object(_) => { + ValueRef::String(v) => writer.write_str(v), + ValueRef::Null => Ok(()), + ValueRef::Bool(_) | ValueRef::Number(_) | ValueRef::Array(_) | ValueRef::Object(_) => { use crate::types::to_text::ToText as _; self.write_with_type(&crate::types::DataType::Jsonb, writer) } @@ -316,38 +384,33 @@ impl<'a> JsonbRef<'a> { /// Returns an iterator over the elements if this is an array. pub fn array_elements(self) -> Result>, String> { - match &self.0 { - Value::Array(array) => Ok(array.iter().map(Self)), - _ => Err(format!( - "cannot extract elements from a jsonb {}", - self.type_name() - )), - } + let array = self + .0 + .as_array() + .ok_or_else(|| format!("cannot extract elements from a jsonb {}", self.type_name()))?; + Ok(array.iter().map(Self)) } /// Returns an iterator over the keys if this is an object. pub fn object_keys(self) -> Result, String> { - match &self.0 { - Value::Object(object) => Ok(object.keys().map(|s| s.as_str())), - _ => Err(format!( + let object = self.0.as_object().ok_or_else(|| { + format!( "cannot call jsonb_object_keys on a jsonb {}", self.type_name() - )), - } + ) + })?; + Ok(object.keys()) } /// Returns an iterator over the key-value pairs if this is an object. pub fn object_key_values( self, ) -> Result)>, String> { - match &self.0 { - Value::Object(object) => Ok(object.iter().map(|(k, v)| (k.as_str(), Self(v)))), - _ => Err(format!("cannot deconstruct a jsonb {}", self.type_name())), - } - } - - pub fn value(&self) -> &'a Value { - self.0 + let object = self + .0 + .as_object() + .ok_or_else(|| format!("cannot deconstruct a jsonb {}", self.type_name()))?; + Ok(object.iter().map(|(k, v)| (k, Self(v)))) } } diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index 83d281c5238e6..386f63280a557 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -416,7 +416,7 @@ impl DataType { DataType::Timestamptz => ScalarImpl::Timestamptz(Timestamptz::MIN), DataType::Decimal => ScalarImpl::Decimal(Decimal::NegativeInf), DataType::Interval => ScalarImpl::Interval(Interval::MIN), - DataType::Jsonb => ScalarImpl::Jsonb(JsonbVal::dummy()), // NOT `min` #7981 + DataType::Jsonb => ScalarImpl::Jsonb(JsonbVal::null()), // NOT `min` #7981 DataType::Struct(data_types) => ScalarImpl::Struct(StructValue::new( data_types .types() @@ -1303,7 +1303,7 @@ mod tests { ScalarImpl::Interval(Interval::from_month_day_usec(2, 3, 3333)), DataType::Interval, ), - DataTypeName::Jsonb => (ScalarImpl::Jsonb(JsonbVal::dummy()), DataType::Jsonb), + DataTypeName::Jsonb => (ScalarImpl::Jsonb(JsonbVal::null()), DataType::Jsonb), DataTypeName::Struct => ( ScalarImpl::Struct(StructValue::new(vec![ ScalarImpl::Int64(233).into(), diff --git a/src/expr/impl/Cargo.toml b/src/expr/impl/Cargo.toml index 81cd685c4dc27..cc0229f83ebab 100644 --- a/src/expr/impl/Cargo.toml +++ b/src/expr/impl/Cargo.toml @@ -29,6 +29,7 @@ futures-async-stream = { workspace = true } futures-util = "0.3" hex = "0.4" itertools = "0.11" +jsonbb = "0.1" md5 = "0.7" num-traits = "0.2" regex = "1" diff --git a/src/expr/impl/benches/expr.rs b/src/expr/impl/benches/expr.rs index 1e84d8d8e4825..010508c8de45e 100644 --- a/src/expr/impl/benches/expr.rs +++ b/src/expr/impl/benches/expr.rs @@ -170,10 +170,7 @@ fn bench_expr(c: &mut Criterion) { // 25: serial array SerialArray::from_iter((1..=CHUNK_SIZE).map(|i| Serial::from(i as i64))).into_ref(), // 26: jsonb array - JsonbArray::from_iter( - (1..=CHUNK_SIZE).map(|i| JsonbVal::from(serde_json::Value::Number(i.into()))), - ) - .into_ref(), + JsonbArray::from_iter((1..=CHUNK_SIZE).map(|i| JsonbVal::from(i as i64))).into_ref(), // 27: int256 array Int256Array::from_iter((1..=CHUNK_SIZE).map(|_| Int256::from(1))).into_ref(), // 28: extract field for interval @@ -279,16 +276,16 @@ fn bench_expr(c: &mut Criterion) { 'sig: for sig in sigs { if (sig.inputs_type.iter()) .chain([&sig.ret_type]) - .any(|t| !t.is_exact()) + .any(|t| !t.is_exact() || t.as_exact().is_array()) { - // TODO: support struct and list + // TODO: support struct and array println!("todo: {sig:?}"); continue; } if [ - "date_trunc(varchar, timestamptz) -> timestamptz", - "to_timestamp1(varchar, varchar) -> timestamptz", - "to_char(timestamptz, varchar) -> varchar", + "date_trunc(character varying, timestamp with time zone) -> timestamp with time zone", + "to_timestamp1(character varying, character varying) -> timestamp with time zone", + "to_char(timestamp with time zone, character varying) -> character varying", ] .contains(&format!("{sig:?}").as_str()) { @@ -376,6 +373,13 @@ fn bench_expr(c: &mut Criterion) { args: match sig.inputs_type.as_slice() { [] => AggArgs::None, [t] => AggArgs::Unary(t.as_exact().clone(), input_index_for_type(t.as_exact())), + [t1, t2] => AggArgs::Binary( + [t1.as_exact().clone(), t2.as_exact().clone()], + [ + input_index_for_type(t1.as_exact()), + input_index_for_type(t2.as_exact()), + ], + ), _ => { println!("todo: {sig:?}"); continue; @@ -393,6 +397,15 @@ fn bench_expr(c: &mut Criterion) { continue; } }; + let input = match sig.inputs_type.as_slice() { + [] => input.project(&[]), + [t] => input.project(&[input_index_for_type(t.as_exact())]), + [t1, t2] => input.project(&[ + input_index_for_type(t1.as_exact()), + input_index_for_type(t2.as_exact()), + ]), + _ => unreachable!(), + }; c.bench_function(&format!("{sig:?}"), |bencher| { bencher .to_async(FuturesExecutor) diff --git a/src/expr/impl/src/aggregate/jsonb_agg.rs b/src/expr/impl/src/aggregate/jsonb_agg.rs index 8385e2c6a060b..96f5e50da85e3 100644 --- a/src/expr/impl/src/aggregate/jsonb_agg.rs +++ b/src/expr/impl/src/aggregate/jsonb_agg.rs @@ -12,22 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::JsonbVal; +use risingwave_common::estimate_size::EstimateSize; +use risingwave_common::types::{JsonbRef, JsonbVal, ScalarImpl, F32, F64}; +use risingwave_expr::aggregate::AggStateDyn; use risingwave_expr::{aggregate, ExprError, Result}; -use serde_json::Value; #[aggregate("jsonb_agg(boolean) -> jsonb")] #[aggregate("jsonb_agg(*int) -> jsonb")] #[aggregate("jsonb_agg(*float) -> jsonb")] #[aggregate("jsonb_agg(varchar) -> jsonb")] #[aggregate("jsonb_agg(jsonb) -> jsonb")] -fn jsonb_agg(state: Option, input: Option>) -> JsonbVal { - let mut jsonb = state.unwrap_or_else(|| Value::Array(Vec::with_capacity(1)).into()); - match jsonb.as_serde_mut() { - Value::Array(a) => a.push(input.map_or(Value::Null, Into::into)), - _ => unreachable!("invalid jsonb state"), - }; - jsonb +fn jsonb_agg(state: &mut JsonbArrayState, input: Option) { + match input { + Some(input) => input.add_to(&mut state.0), + None => state.0.add_null(), + } } #[aggregate("jsonb_object_agg(varchar, boolean) -> jsonb")] @@ -36,15 +35,130 @@ fn jsonb_agg(state: Option, input: Option>) -> JsonbV #[aggregate("jsonb_object_agg(varchar, varchar) -> jsonb")] #[aggregate("jsonb_object_agg(varchar, jsonb) -> jsonb")] fn jsonb_object_agg( - state: Option, + state: &mut JsonbObjectState, key: Option<&str>, - value: Option>, -) -> Result { + value: Option, +) -> Result<()> { let key = key.ok_or(ExprError::FieldNameNull)?; - let mut jsonb = state.unwrap_or_else(|| Value::Object(Default::default()).into()); - match jsonb.as_serde_mut() { - Value::Object(map) => map.insert(key.into(), value.map_or(Value::Null, Into::into)), - _ => unreachable!("invalid jsonb state"), - }; - Ok(jsonb) + state.0.add_string(key); + match value { + Some(value) => value.add_to(&mut state.0), + None => state.0.add_null(), + } + Ok(()) +} + +#[derive(Debug)] +struct JsonbArrayState(jsonbb::Builder); + +impl EstimateSize for JsonbArrayState { + fn estimated_heap_size(&self) -> usize { + self.0.capacity() + } +} + +impl AggStateDyn for JsonbArrayState {} + +/// Creates an initial state. +impl Default for JsonbArrayState { + fn default() -> Self { + let mut builder = jsonbb::Builder::default(); + builder.begin_array(); + Self(builder) + } +} + +/// Finishes aggregation and returns the result. +impl From<&JsonbArrayState> for ScalarImpl { + fn from(builder: &JsonbArrayState) -> Self { + // TODO: avoid clone + let mut builder = builder.0.clone(); + builder.end_array(); + let jsonb: JsonbVal = builder.finish().into(); + jsonb.into() + } +} + +#[derive(Debug)] +struct JsonbObjectState(jsonbb::Builder); + +impl EstimateSize for JsonbObjectState { + fn estimated_heap_size(&self) -> usize { + self.0.capacity() + } +} + +impl AggStateDyn for JsonbObjectState {} + +/// Creates an initial state. +impl Default for JsonbObjectState { + fn default() -> Self { + let mut builder = jsonbb::Builder::default(); + builder.begin_object(); + Self(builder) + } +} + +/// Finishes aggregation and returns the result. +impl From<&JsonbObjectState> for ScalarImpl { + fn from(builder: &JsonbObjectState) -> Self { + // TODO: avoid clone + let mut builder = builder.0.clone(); + builder.end_object(); + let jsonb: JsonbVal = builder.finish().into(); + jsonb.into() + } +} + +/// Values that can be converted to JSON. +trait ToJson { + fn add_to(self, builder: &mut jsonbb::Builder); +} + +impl ToJson for bool { + fn add_to(self, builder: &mut jsonbb::Builder) { + builder.add_bool(self); + } +} + +impl ToJson for i16 { + fn add_to(self, builder: &mut jsonbb::Builder) { + builder.add_i64(self as _); + } +} + +impl ToJson for i32 { + fn add_to(self, builder: &mut jsonbb::Builder) { + builder.add_i64(self as _); + } +} + +impl ToJson for i64 { + fn add_to(self, builder: &mut jsonbb::Builder) { + builder.add_i64(self); + } +} + +impl ToJson for F32 { + fn add_to(self, builder: &mut jsonbb::Builder) { + builder.add_f64(self.0 as f64); + } +} + +impl ToJson for F64 { + fn add_to(self, builder: &mut jsonbb::Builder) { + builder.add_f64(self.0); + } +} + +impl ToJson for &str { + fn add_to(self, builder: &mut jsonbb::Builder) { + builder.add_string(self); + } +} + +impl ToJson for JsonbRef<'_> { + fn add_to(self, builder: &mut jsonbb::Builder) { + builder.add_value(self.into()); + } } diff --git a/src/expr/impl/src/scalar/jsonb_concat.rs b/src/expr/impl/src/scalar/jsonb_concat.rs index 6277db8d5b981..db469457bb135 100644 --- a/src/expr/impl/src/scalar/jsonb_concat.rs +++ b/src/expr/impl/src/scalar/jsonb_concat.rs @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use jsonbb::{Value, ValueRef}; use risingwave_common::types::{JsonbRef, JsonbVal}; use risingwave_expr::function; -use serde_json::{json, Value}; /// Concatenates the two jsonbs. /// @@ -59,43 +59,35 @@ use serde_json::{json, Value}; /// ``` #[function("jsonb_cat(jsonb, jsonb) -> jsonb")] pub fn jsonb_cat(left: JsonbRef<'_>, right: JsonbRef<'_>) -> JsonbVal { - let left_val = left.value().clone(); - let right_val = right.value().clone(); - match (left_val, right_val) { + match (left.into(), right.into()) { // left and right are object based. // This would have left:{'a':1}, right:{'b':2} -> {'a':1,'b':2} - (Value::Object(mut left_map), Value::Object(right_map)) => { - left_map.extend(right_map); - JsonbVal::from(Value::Object(left_map)) + (ValueRef::Object(left), ValueRef::Object(right)) => { + JsonbVal::from(Value::object(left.iter().chain(right.iter()))) } // left and right are array-based. // This would merge both arrays into one array. // This would have left:[1,2], right:[3,4] -> [1,2,3,4] - (Value::Array(mut left_arr), Value::Array(right_arr)) => { - left_arr.extend(right_arr); - JsonbVal::from(Value::Array(left_arr)) + (ValueRef::Array(left), ValueRef::Array(right)) => { + JsonbVal::from(Value::array(left.iter().chain(right.iter()))) } // One operand is an array, and the other is a single element. // This would insert the non-array value as another element into the array // Eg left:[1,2] right: {'a':1} -> [1,2,{'a':1}] - (Value::Array(mut left_arr), single_val) => { - left_arr.push(single_val); - JsonbVal::from(Value::Array(left_arr)) - } + (ValueRef::Array(left), value) => JsonbVal::from(Value::array(left.iter().chain([value]))), // One operand is an array, and the other is a single element. // This would insert the non-array value as another element into the array // Eg left:{'a':1} right:[1,2] -> [{'a':1},1,2] - (single_val, Value::Array(mut right_arr)) => { - right_arr.insert(0, single_val); - JsonbVal::from(Value::Array(right_arr)) + (value, ValueRef::Array(right)) => { + JsonbVal::from(Value::array([value].into_iter().chain(right.iter()))) } // Both are non-array inputs. // Both elements would be placed together in an array // Eg left:1 right: 2 -> [1,2] - (left, right) => JsonbVal::from(json!([left, right])), + (left, right) => JsonbVal::from(Value::array([left, right])), } } diff --git a/src/expr/macro/src/gen.rs b/src/expr/macro/src/gen.rs index 9155853df5b7b..454d2a3169137 100644 --- a/src/expr/macro/src/gen.rs +++ b/src/expr/macro/src/gen.rs @@ -579,9 +579,13 @@ impl FunctionAttr { /// Generate build function for aggregate function. fn generate_agg_build_fn(&self, user_fn: &AggregateFnOrImpl) -> Result { - let state_type: TokenStream2 = match &self.state { - Some(state) if state == "ref" => types::ref_type(&self.ret).parse().unwrap(), - Some(state) if state != "ref" => types::owned_type(state).parse().unwrap(), + // If the first argument of the aggregate function is of type `&mut T`, + // we assume it is a user defined state type. + let custom_state = user_fn.accumulate().first_mut_ref_arg.as_ref(); + let state_type: TokenStream2 = match (custom_state, &self.state) { + (Some(s), _) => s.parse().unwrap(), + (_, Some(state)) if state == "ref" => types::ref_type(&self.ret).parse().unwrap(), + (_, Some(state)) if state != "ref" => types::owned_type(state).parse().unwrap(), _ => types::owned_type(&self.ret).parse().unwrap(), }; let let_arrays = self @@ -603,24 +607,37 @@ impl FunctionAttr { quote! { let #v = unsafe { #a.value_at_unchecked(row_id) }; } }) .collect_vec(); - let let_state = match &self.state { - Some(s) if s == "ref" => { - quote! { state0.as_ref().map(|x| x.as_scalar_ref_impl().try_into().unwrap()) } - } - _ => quote! { state0.take().map(|s| s.try_into().unwrap()) }, + let downcast_state = if custom_state.is_some() { + quote! { let mut state: &mut #state_type = state0.downcast_mut(); } + } else if let Some(s) = &self.state && s == "ref" { + quote! { let mut state: Option<#state_type> = state0.as_datum_mut().as_ref().map(|x| x.as_scalar_ref_impl().try_into().unwrap()); } + } else { + quote! { let mut state: Option<#state_type> = state0.as_datum_mut().take().map(|s| s.try_into().unwrap()); } }; - let assign_state = match &self.state { - Some(s) if s == "ref" => quote! { state.map(|x| x.to_owned_scalar().into()) }, - _ => quote! { state.map(|s| s.into()) }, + let restore_state = if custom_state.is_some() { + quote! {} + } else if let Some(s) = &self.state && s == "ref" { + quote! { *state0.as_datum_mut() = state.map(|x| x.to_owned_scalar().into()); } + } else { + quote! { *state0.as_datum_mut() = state.map(|s| s.into()); } }; - let create_state = self.init_state.as_ref().map(|state| { + let create_state = if custom_state.is_some() { + quote! { + fn create_state(&self) -> AggregateState { + AggregateState::Any(Box::<#state_type>::default()) + } + } + } else if let Some(state) = &self.init_state { let state: TokenStream2 = state.parse().unwrap(); quote! { fn create_state(&self) -> AggregateState { AggregateState::Datum(Some(#state.into())) } } - }); + } else { + // by default: `AggregateState::Datum(None)` + quote! {} + }; let args = (0..self.args.len()).map(|i| format_ident!("v{i}")); let args = quote! { #(#args,)* }; let panic_on_retract = { @@ -703,17 +720,23 @@ impl FunctionAttr { _ => todo!("multiple arguments are not supported for non-option function"), } } - let get_result = match user_fn { - AggregateFnOrImpl::Impl(impl_) if impl_.finalize.is_some() => { - quote! { - let state = match state { - Some(s) => s.as_scalar_ref_impl().try_into().unwrap(), - None => return Ok(None), - }; - Ok(Some(self.function.finalize(state).into())) - } + let update_state = if custom_state.is_some() { + quote! { _ = #next_state; } + } else { + quote! { state = #next_state; } + }; + let get_result = if custom_state.is_some() { + quote! { Ok(Some(state.downcast_ref::<#state_type>().into())) } + } else if let AggregateFnOrImpl::Impl(impl_) = user_fn && impl_.finalize.is_some() { + quote! { + let state = match state.as_datum() { + Some(s) => s.as_scalar_ref_impl().try_into().unwrap(), + None => return Ok(None), + }; + Ok(Some(self.function.finalize(state).into())) } - _ => quote! { Ok(state.clone()) }, + } else { + quote! { Ok(state.as_datum().clone()) } }; let function_field = match user_fn { AggregateFnOrImpl::Fn(_) => quote! {}, @@ -768,27 +791,25 @@ impl FunctionAttr { async fn update(&self, state0: &mut AggregateState, input: &StreamChunk) -> Result<()> { #(#let_arrays)* - let state0 = state0.as_datum_mut(); - let mut state: Option<#state_type> = #let_state; + #downcast_state for row_id in input.visibility().iter_ones() { let op = unsafe { *input.ops().get_unchecked(row_id) }; #(#let_values)* - state = #next_state; + #update_state } - *state0 = #assign_state; + #restore_state Ok(()) } async fn update_range(&self, state0: &mut AggregateState, input: &StreamChunk, range: Range) -> Result<()> { assert!(range.end <= input.capacity()); #(#let_arrays)* - let state0 = state0.as_datum_mut(); - let mut state: Option<#state_type> = #let_state; + #downcast_state if input.is_compacted() { for row_id in range { let op = unsafe { *input.ops().get_unchecked(row_id) }; #(#let_values)* - state = #next_state; + #update_state } } else { for row_id in input.visibility().iter_ones() { @@ -799,15 +820,14 @@ impl FunctionAttr { } let op = unsafe { *input.ops().get_unchecked(row_id) }; #(#let_values)* - state = #next_state; + #update_state } } - *state0 = #assign_state; + #restore_state Ok(()) } async fn get_result(&self, state: &AggregateState) -> Result { - let state = state.as_datum(); #get_result } } diff --git a/src/expr/macro/src/lib.rs b/src/expr/macro/src/lib.rs index 363fc958b557d..50a99cf3fda22 100644 --- a/src/expr/macro/src/lib.rs +++ b/src/expr/macro/src/lib.rs @@ -522,6 +522,8 @@ struct UserFunctionAttr { retract: bool, /// The argument type are `Option`s. arg_option: bool, + /// If the first argument type is `&mut T`, then `Some(T)`. + first_mut_ref_arg: Option, /// The return type kind. return_type_kind: ReturnTypeKind, /// The kind of inner type `T` in `impl Iterator` diff --git a/src/expr/macro/src/parse.rs b/src/expr/macro/src/parse.rs index 24cc6942afcee..8e2e8c6d0b2f1 100644 --- a/src/expr/macro/src/parse.rs +++ b/src/expr/macro/src/parse.rs @@ -123,6 +123,7 @@ impl From<&syn::Signature> for UserFunctionAttr { context: sig.inputs.iter().any(arg_is_context), retract: last_arg_is_retract(sig), arg_option: args_contain_option(sig), + first_mut_ref_arg: first_mut_ref_arg(sig), return_type_kind, iterator_item_kind, core_return_type, @@ -223,18 +224,15 @@ fn last_arg_is_retract(sig: &syn::Signature) -> bool { /// Check if any argument is `Option`. fn args_contain_option(sig: &syn::Signature) -> bool { - if sig.inputs.is_empty() { - return false; - } for arg in &sig.inputs { let syn::FnArg::Typed(arg) = arg else { - return false; + continue; }; let syn::Type::Path(path) = arg.ty.as_ref() else { - return false; + continue; }; let Some(seg) = path.path.segments.last() else { - return false; + continue; }; if seg.ident == "Option" { return true; @@ -243,6 +241,26 @@ fn args_contain_option(sig: &syn::Signature) -> bool { false } +/// Returns `T` if the first argument (except `self`) is `&mut T`. +fn first_mut_ref_arg(sig: &syn::Signature) -> Option { + let arg = match sig.inputs.first()? { + syn::FnArg::Typed(arg) => arg, + syn::FnArg::Receiver(_) => match sig.inputs.iter().nth(1)? { + syn::FnArg::Typed(arg) => arg, + _ => return None, + }, + }; + let syn::Type::Reference(syn::TypeReference { + elem, + mutability: Some(_), + .. + }) = arg.ty.as_ref() + else { + return None; + }; + Some(elem.to_token_stream().to_string()) +} + /// Check the return type. fn check_type(ty: &syn::Type) -> (ReturnTypeKind, &syn::Type) { if let Some(inner) = strip_outer_type(ty, "Result") { From 09a67abbefb5c7a0c53596e0a5cf557a2fae3664 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 25 Oct 2023 16:49:08 +0800 Subject: [PATCH 22/48] fix: `WAIT` should return error if timeout (#13045) --- src/meta/service/src/ddl_service.rs | 2 +- src/meta/src/rpc/ddl_controller.rs | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 061ff93589163..fac8f89e17b11 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -734,7 +734,7 @@ impl DdlService for DdlServiceImpl { } async fn wait(&self, _request: Request) -> Result, Status> { - self.ddl_controller.wait().await; + self.ddl_controller.wait().await?; Ok(Response::new(WaitResponse {})) } } diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 36615bd93b757..9886121bc03c2 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -1097,7 +1097,7 @@ impl DdlController { } } - pub async fn wait(&self) { + pub async fn wait(&self) -> MetaResult<()> { for _ in 0..30 * 60 { if self .catalog_manager @@ -1105,9 +1105,10 @@ impl DdlController { .await .is_empty() { - break; + return Ok(()); } sleep(Duration::from_secs(1)).await; } + Err(MetaError::cancelled("timeout".into())) } } From 7f82929b38131f43b1290e51b0722c972fa3b6c2 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 25 Oct 2023 16:57:45 +0800 Subject: [PATCH 23/48] fix(meta): persist internal tables of `CREATE TABLE` (#13039) --- src/meta/src/manager/catalog/mod.rs | 33 ++++++++++++----------------- src/meta/src/rpc/ddl_controller.rs | 1 + 2 files changed, 15 insertions(+), 19 deletions(-) diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index f988646428aac..15e74e4c2ac9e 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -647,7 +647,8 @@ impl CatalogManager { self.start_create_table_procedure_with_source(source, table) .await } else { - self.start_create_table_procedure(table, vec![]).await + self.start_create_table_procedure(table, internal_tables) + .await } } } @@ -765,7 +766,9 @@ impl CatalogManager { /// 2. Not belonging to a background stream job. /// Clean up these hanging tables by the id. pub async fn clean_dirty_tables(&self, fragment_manager: FragmentManagerRef) -> MetaResult<()> { - let creating_tables: Vec = self.list_persisted_creating_tables().await; + let core = &mut *self.core.lock().await; + let database_core = &mut core.database; + let creating_tables: Vec
= database_core.list_persisted_creating_tables(); tracing::debug!( "creating_tables ids: {:#?}", creating_tables.iter().map(|t| t.id).collect_vec() @@ -839,14 +842,13 @@ impl CatalogManager { } } - let core = &mut *self.core.lock().await; - let database_core = &mut core.database; let tables = &mut database_core.tables; let mut tables = BTreeMapTransaction::new(tables); for table in &tables_to_clean { - tracing::debug!("cleaning table_id: {}", table.id); - let table = tables.remove(table.id); - assert!(table.is_some()) + let table_id = table.id; + tracing::debug!("cleaning table_id: {}", table_id); + let table = tables.remove(table_id); + assert!(table.is_some(), "table_id {} missing", table_id) } commit_meta!(self, tables)?; @@ -929,14 +931,8 @@ impl CatalogManager { ); return Ok(()); }; - table - }; - - tracing::trace!("cleanup tables for {}", table.id); - { - let core = &mut self.core.lock().await; - let database_core = &mut core.database; + tracing::trace!("cleanup tables for {}", table.id); let mut table_ids = vec![table.id]; table_ids.extend(internal_table_ids); @@ -944,10 +940,11 @@ impl CatalogManager { let mut tables = BTreeMapTransaction::new(tables); for table_id in table_ids { let res = tables.remove(table_id); - assert!(res.is_some()); + assert!(res.is_some(), "table_id {} missing", table_id); } commit_meta!(self, tables)?; - } + table + }; { let core = &mut self.core.lock().await; @@ -1984,9 +1981,7 @@ impl CatalogManager { let table_key = (table.database_id, table.schema_id, table.name.clone()); assert!( !database_core.sources.contains_key(&source.id) - && !database_core.tables.contains_key(&table.id) - && database_core.has_in_progress_creation(&source_key) - && database_core.has_in_progress_creation(&table_key), + && !database_core.tables.contains_key(&table.id), "table and source must be in creating procedure" ); diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 9886121bc03c2..5f40d9a561f4e 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -431,6 +431,7 @@ impl DdlController { let env = StreamEnvironment::from_protobuf(fragment_graph.get_env().unwrap()); + // Persist tables tracing::debug!(id = stream_job.id(), "preparing stream job"); let fragment_graph = self .prepare_stream_job(&mut stream_job, fragment_graph) From 7f791d65f10ff304e10ba3a6d58bb6bb3a33bad9 Mon Sep 17 00:00:00 2001 From: August Date: Wed, 25 Oct 2023 20:29:16 +0800 Subject: [PATCH 24/48] feat: move model_v2 and model_migration into a separate crates (#13058) --- .licenserc.yaml | 2 +- Cargo.lock | 34 ++- Cargo.toml | 5 +- src/meta/Cargo.toml | 3 +- src/meta/model_v2/Cargo.toml | 26 ++ src/meta/model_v2/migration/Cargo.toml | 22 ++ .../{src => }/model_v2/migration/README.md | 0 .../{src => }/model_v2/migration/src/lib.rs | 0 .../migration/src/m20230908_072257_init.rs | 0 .../migration/src/m20231008_020431_hummock.rs | 0 .../{src => }/model_v2/migration/src/main.rs | 2 +- .../{src/model_v2 => model_v2/src}/README.md | 2 +- .../{src/model_v2 => model_v2/src}/actor.rs | 2 +- .../{src/model_v2 => model_v2/src}/cluster.rs | 0 .../src}/compaction_config.rs | 0 .../src}/compaction_status.rs | 0 .../src}/compaction_task.rs | 0 .../model_v2 => model_v2/src}/connection.rs | 2 +- .../model_v2 => model_v2/src}/database.rs | 13 +- .../model_v2 => model_v2/src}/fragment.rs | 2 +- .../model_v2 => model_v2/src}/function.rs | 2 +- .../src}/hummock_pinned_snapshot.rs | 0 .../src}/hummock_pinned_version.rs | 0 .../src}/hummock_version_delta.rs | 0 .../src}/hummock_version_stats.rs | 0 .../{src/model_v2 => model_v2/src}/index.rs | 2 +- .../model_v2/mod.rs => model_v2/src/lib.rs} | 2 - .../{src/model_v2 => model_v2/src}/object.rs | 2 +- .../src}/object_dependency.rs | 2 +- .../{src/model_v2 => model_v2/src}/prelude.rs | 0 .../{src/model_v2 => model_v2/src}/schema.rs | 13 +- .../{src/model_v2 => model_v2/src}/sink.rs | 2 +- .../{src/model_v2 => model_v2/src}/source.rs | 2 +- .../src}/system_parameter.rs | 0 .../{src/model_v2 => model_v2/src}/table.rs | 2 +- .../{src/model_v2 => model_v2/src}/user.rs | 2 +- .../src}/user_privilege.rs | 2 +- .../{src/model_v2 => model_v2/src}/view.rs | 2 +- src/meta/model_v2/src/worker.rs | 128 ++++++++ .../src}/worker_property.rs | 2 +- src/meta/node/Cargo.toml | 2 +- src/meta/node/src/server.rs | 2 +- src/meta/service/Cargo.toml | 1 + src/meta/service/src/telemetry_service.rs | 2 +- src/meta/src/controller/catalog.rs | 14 +- src/meta/src/controller/cluster.rs | 64 +--- src/meta/src/controller/mod.rs | 26 +- src/meta/src/controller/system_param.rs | 4 +- src/meta/src/controller/utils.rs | 14 +- src/meta/src/lib.rs | 1 - src/meta/src/manager/env.rs | 2 +- src/meta/src/manager/mod.rs | 3 +- src/meta/src/model_v2/ext/hummock.rs | 61 ---- src/meta/src/model_v2/ext/mod.rs | 16 - src/meta/src/model_v2/migration/Cargo.toml | 17 -- src/meta/src/model_v2/trx.rs | 276 ------------------ src/meta/src/model_v2/worker.rs | 67 ----- 57 files changed, 276 insertions(+), 576 deletions(-) create mode 100644 src/meta/model_v2/Cargo.toml create mode 100644 src/meta/model_v2/migration/Cargo.toml rename src/meta/{src => }/model_v2/migration/README.md (100%) rename src/meta/{src => }/model_v2/migration/src/lib.rs (100%) rename src/meta/{src => }/model_v2/migration/src/m20230908_072257_init.rs (100%) rename src/meta/{src => }/model_v2/migration/src/m20231008_020431_hummock.rs (100%) rename src/meta/{src => }/model_v2/migration/src/main.rs (52%) rename src/meta/{src/model_v2 => model_v2/src}/README.md (93%) rename src/meta/{src/model_v2 => model_v2/src}/actor.rs (97%) rename src/meta/{src/model_v2 => model_v2/src}/cluster.rs (100%) rename src/meta/{src/model_v2 => model_v2/src}/compaction_config.rs (100%) rename src/meta/{src/model_v2 => model_v2/src}/compaction_status.rs (100%) rename src/meta/{src/model_v2 => model_v2/src}/compaction_task.rs (100%) rename src/meta/{src/model_v2 => model_v2/src}/connection.rs (97%) rename src/meta/{src/model_v2 => model_v2/src}/database.rs (81%) rename src/meta/{src/model_v2 => model_v2/src}/fragment.rs (98%) rename src/meta/{src/model_v2 => model_v2/src}/function.rs (97%) rename src/meta/{src/model_v2 => model_v2/src}/hummock_pinned_snapshot.rs (100%) rename src/meta/{src/model_v2 => model_v2/src}/hummock_pinned_version.rs (100%) rename src/meta/{src/model_v2 => model_v2/src}/hummock_version_delta.rs (100%) rename src/meta/{src/model_v2 => model_v2/src}/hummock_version_stats.rs (100%) rename src/meta/{src/model_v2 => model_v2/src}/index.rs (96%) rename src/meta/{src/model_v2/mod.rs => model_v2/src/lib.rs} (99%) rename src/meta/{src/model_v2 => model_v2/src}/object.rs (98%) rename src/meta/{src/model_v2 => model_v2/src}/object_dependency.rs (97%) rename src/meta/{src/model_v2 => model_v2/src}/prelude.rs (100%) rename src/meta/{src/model_v2 => model_v2/src}/schema.rs (81%) rename src/meta/{src/model_v2 => model_v2/src}/sink.rs (99%) rename src/meta/{src/model_v2 => model_v2/src}/source.rs (99%) rename src/meta/{src/model_v2 => model_v2/src}/system_parameter.rs (100%) rename src/meta/{src/model_v2 => model_v2/src}/table.rs (99%) rename src/meta/{src/model_v2 => model_v2/src}/user.rs (97%) rename src/meta/{src/model_v2 => model_v2/src}/user_privilege.rs (97%) rename src/meta/{src/model_v2 => model_v2/src}/view.rs (97%) create mode 100644 src/meta/model_v2/src/worker.rs rename src/meta/{src/model_v2 => model_v2/src}/worker_property.rs (97%) delete mode 100644 src/meta/src/model_v2/ext/hummock.rs delete mode 100644 src/meta/src/model_v2/ext/mod.rs delete mode 100644 src/meta/src/model_v2/migration/Cargo.toml delete mode 100644 src/meta/src/model_v2/trx.rs delete mode 100644 src/meta/src/model_v2/worker.rs diff --git a/.licenserc.yaml b/.licenserc.yaml index c1745a4d1ad74..7b49108b6b2f3 100644 --- a/.licenserc.yaml +++ b/.licenserc.yaml @@ -17,6 +17,6 @@ header: - "**/*.d.ts" - "src/sqlparser/**/*.rs" - "java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/*.java" - - "src/meta/src/model_v2/migration/**/*.rs" + - "src/meta/model_v2/migration/**/*.rs" comment: on-failure diff --git a/Cargo.lock b/Cargo.lock index cf1f1f0e493e9..19e79820fb5e9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4779,15 +4779,6 @@ dependencies = [ "syn 1.0.109", ] -[[package]] -name = "model_migration" -version = "0.1.0" -dependencies = [ - "async-std", - "sea-orm-migration", - "uuid", -] - [[package]] name = "moka" version = "0.12.0" @@ -7695,7 +7686,6 @@ dependencies = [ "maplit", "memcomparable", "mime_guess", - "model_migration", "num-integer", "num-traits", "parking_lot 0.12.1", @@ -7709,6 +7699,8 @@ dependencies = [ "risingwave_common_heap_profiling", "risingwave_connector", "risingwave_hummock_sdk", + "risingwave_meta_model_migration", + "risingwave_meta_model_v2", "risingwave_object_store", "risingwave_pb", "risingwave_rpc_client", @@ -7730,6 +7722,25 @@ dependencies = [ "workspace-hack", ] +[[package]] +name = "risingwave_meta_model_migration" +version = "1.3.0-alpha" +dependencies = [ + "async-std", + "sea-orm-migration", + "uuid", +] + +[[package]] +name = "risingwave_meta_model_v2" +version = "1.3.0-alpha" +dependencies = [ + "risingwave_pb", + "sea-orm", + "serde", + "serde_json", +] + [[package]] name = "risingwave_meta_node" version = "1.3.0-alpha" @@ -7742,13 +7753,13 @@ dependencies = [ "madsim-etcd-client", "madsim-tokio", "madsim-tonic", - "model_migration", "prometheus-http-query", "regex", "risingwave_common", "risingwave_common_heap_profiling", "risingwave_common_service", "risingwave_meta", + "risingwave_meta_model_migration", "risingwave_meta_service", "risingwave_pb", "risingwave_rpc_client", @@ -7772,6 +7783,7 @@ dependencies = [ "risingwave_common", "risingwave_connector", "risingwave_meta", + "risingwave_meta_model_v2", "risingwave_pb", "sea-orm", "sync-point", diff --git a/Cargo.toml b/Cargo.toml index f0dd2d0443b9e..ac533e733f7a8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -19,9 +19,10 @@ members = [ "src/java_binding", "src/jni_core", "src/meta", + "src/meta/model_v2", + "src/meta/model_v2/migration", "src/meta/node", "src/meta/service", - "src/meta/src/model_v2/migration", "src/object_store", "src/prost", "src/prost/helpers", @@ -143,6 +144,8 @@ risingwave_hummock_test = { path = "./src/storage/hummock_test" } risingwave_hummock_trace = { path = "./src/storage/hummock_trace" } risingwave_meta = { path = "./src/meta" } risingwave_meta_service = { path = "./src/meta/service" } +risingwave_meta_model_migration = { path = "src/meta/model_v2/migration" } +risingwave_meta_model_v2 = { path = "./src/meta/model_v2" } risingwave_meta_node = { path = "./src/meta/node" } risingwave_object_store = { path = "./src/object_store" } risingwave_pb = { path = "./src/prost" } diff --git a/src/meta/Cargo.toml b/src/meta/Cargo.toml index 3e96dfcc7be2f..f37c909546594 100644 --- a/src/meta/Cargo.toml +++ b/src/meta/Cargo.toml @@ -36,7 +36,6 @@ hyper = "0.14" itertools = "0.11" memcomparable = { version = "0.2" } mime_guess = "2" -model_migration = { path = "src/model_v2/migration" } num-integer = "0.1" num-traits = "0.2" parking_lot = { version = "0.12", features = ["arc_lock"] } @@ -50,6 +49,8 @@ risingwave_common = { workspace = true } risingwave_common_heap_profiling = { workspace = true } risingwave_connector = { workspace = true } risingwave_hummock_sdk = { workspace = true } +risingwave_meta_model_migration = { workspace = true } +risingwave_meta_model_v2 = { workspace = true } risingwave_object_store = { workspace = true } risingwave_pb = { workspace = true } risingwave_rpc_client = { workspace = true } diff --git a/src/meta/model_v2/Cargo.toml b/src/meta/model_v2/Cargo.toml new file mode 100644 index 0000000000000..1d9992da8a832 --- /dev/null +++ b/src/meta/model_v2/Cargo.toml @@ -0,0 +1,26 @@ +[package] +name = "risingwave_meta_model_v2" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +keywords = { workspace = true } +license = { workspace = true } +repository = { workspace = true } + +[package.metadata.cargo-machete] +ignored = ["workspace-hack"] + +[package.metadata.cargo-udeps.ignore] +normal = ["workspace-hack"] + +[dependencies] +risingwave_pb = { workspace = true } +sea-orm = { version = "0.12.0", features = [ + "sqlx-mysql", + "sqlx-postgres", + "sqlx-sqlite", + "runtime-tokio-native-tls", + "macros", +] } +serde = { version = "1", features = ["derive"] } +serde_json = "1" diff --git a/src/meta/model_v2/migration/Cargo.toml b/src/meta/model_v2/migration/Cargo.toml new file mode 100644 index 0000000000000..4745125140a22 --- /dev/null +++ b/src/meta/model_v2/migration/Cargo.toml @@ -0,0 +1,22 @@ +[package] +name = "risingwave_meta_model_migration" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +keywords = { workspace = true } +license = { workspace = true } +repository = { workspace = true } + +[package.metadata.cargo-machete] +ignored = ["workspace-hack"] + +[package.metadata.cargo-udeps.ignore] +normal = ["workspace-hack"] + +[dependencies] +async-std = { version = "1", features = ["attributes", "tokio1"] } +uuid = { version = "1", features = ["v4"] } + +[dependencies.sea-orm-migration] +version = "0.12.0" +features = ["sqlx-mysql", "sqlx-postgres", "sqlx-sqlite", "runtime-tokio-native-tls", "with-uuid"] diff --git a/src/meta/src/model_v2/migration/README.md b/src/meta/model_v2/migration/README.md similarity index 100% rename from src/meta/src/model_v2/migration/README.md rename to src/meta/model_v2/migration/README.md diff --git a/src/meta/src/model_v2/migration/src/lib.rs b/src/meta/model_v2/migration/src/lib.rs similarity index 100% rename from src/meta/src/model_v2/migration/src/lib.rs rename to src/meta/model_v2/migration/src/lib.rs diff --git a/src/meta/src/model_v2/migration/src/m20230908_072257_init.rs b/src/meta/model_v2/migration/src/m20230908_072257_init.rs similarity index 100% rename from src/meta/src/model_v2/migration/src/m20230908_072257_init.rs rename to src/meta/model_v2/migration/src/m20230908_072257_init.rs diff --git a/src/meta/src/model_v2/migration/src/m20231008_020431_hummock.rs b/src/meta/model_v2/migration/src/m20231008_020431_hummock.rs similarity index 100% rename from src/meta/src/model_v2/migration/src/m20231008_020431_hummock.rs rename to src/meta/model_v2/migration/src/m20231008_020431_hummock.rs diff --git a/src/meta/src/model_v2/migration/src/main.rs b/src/meta/model_v2/migration/src/main.rs similarity index 52% rename from src/meta/src/model_v2/migration/src/main.rs rename to src/meta/model_v2/migration/src/main.rs index 9354e45ecd198..9be884a68a11d 100644 --- a/src/meta/src/model_v2/migration/src/main.rs +++ b/src/meta/model_v2/migration/src/main.rs @@ -2,5 +2,5 @@ use sea_orm_migration::prelude::*; #[async_std::main] async fn main() { - cli::run_cli(model_migration::Migrator).await; + cli::run_cli(risingwave_meta_model_migration::Migrator).await; } diff --git a/src/meta/src/model_v2/README.md b/src/meta/model_v2/src/README.md similarity index 93% rename from src/meta/src/model_v2/README.md rename to src/meta/model_v2/src/README.md index 25c22a4f566e1..48095d3e6d67f 100644 --- a/src/meta/src/model_v2/README.md +++ b/src/meta/model_v2/src/README.md @@ -1,6 +1,6 @@ # How to define changes between versions and generate migration and model files -- Generate a new migration file and apply it to the database, check [migration](./migration/README.md) for more details. Let's take a local PG database as an example(`postgres://postgres:@localhost:5432/postgres`): +- Generate a new migration file and apply it to the database, check [migration](../migration/README.md) for more details. Let's take a local PG database as an example(`postgres://postgres:@localhost:5432/postgres`): ```sh export DATABASE_URL=postgres://postgres:@localhost:5432/postgres; cargo run -- generate MIGRATION_NAME diff --git a/src/meta/src/model_v2/actor.rs b/src/meta/model_v2/src/actor.rs similarity index 97% rename from src/meta/src/model_v2/actor.rs rename to src/meta/model_v2/src/actor.rs index 8fecb3046b1bc..79a70e3f65e95 100644 --- a/src/meta/src/model_v2/actor.rs +++ b/src/meta/model_v2/src/actor.rs @@ -14,7 +14,7 @@ use sea_orm::entity::prelude::*; -use crate::model_v2::I32Array; +use crate::I32Array; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "actor")] diff --git a/src/meta/src/model_v2/cluster.rs b/src/meta/model_v2/src/cluster.rs similarity index 100% rename from src/meta/src/model_v2/cluster.rs rename to src/meta/model_v2/src/cluster.rs diff --git a/src/meta/src/model_v2/compaction_config.rs b/src/meta/model_v2/src/compaction_config.rs similarity index 100% rename from src/meta/src/model_v2/compaction_config.rs rename to src/meta/model_v2/src/compaction_config.rs diff --git a/src/meta/src/model_v2/compaction_status.rs b/src/meta/model_v2/src/compaction_status.rs similarity index 100% rename from src/meta/src/model_v2/compaction_status.rs rename to src/meta/model_v2/src/compaction_status.rs diff --git a/src/meta/src/model_v2/compaction_task.rs b/src/meta/model_v2/src/compaction_task.rs similarity index 100% rename from src/meta/src/model_v2/compaction_task.rs rename to src/meta/model_v2/src/compaction_task.rs diff --git a/src/meta/src/model_v2/connection.rs b/src/meta/model_v2/src/connection.rs similarity index 97% rename from src/meta/src/model_v2/connection.rs rename to src/meta/model_v2/src/connection.rs index 0096603c843a3..8cff6b2a6025b 100644 --- a/src/meta/src/model_v2/connection.rs +++ b/src/meta/model_v2/src/connection.rs @@ -17,7 +17,7 @@ use risingwave_pb::catalog::PbConnection; use sea_orm::entity::prelude::*; use sea_orm::ActiveValue; -use crate::model_v2::{ConnectionId, PrivateLinkService}; +use crate::{ConnectionId, PrivateLinkService}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "connection")] diff --git a/src/meta/src/model_v2/database.rs b/src/meta/model_v2/src/database.rs similarity index 81% rename from src/meta/src/model_v2/database.rs rename to src/meta/model_v2/src/database.rs index 909c12eceac5a..95ff3a8aee8e6 100644 --- a/src/meta/src/model_v2/database.rs +++ b/src/meta/model_v2/src/database.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_pb::catalog::PbDatabase; use sea_orm::entity::prelude::*; +use sea_orm::ActiveValue; -use crate::model_v2::DatabaseId; +use crate::DatabaseId; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "database")] @@ -44,3 +46,12 @@ impl Related for Entity { } impl ActiveModelBehavior for ActiveModel {} + +impl From for ActiveModel { + fn from(db: PbDatabase) -> Self { + Self { + database_id: ActiveValue::Set(db.id), + name: ActiveValue::Set(db.name), + } + } +} diff --git a/src/meta/src/model_v2/fragment.rs b/src/meta/model_v2/src/fragment.rs similarity index 98% rename from src/meta/src/model_v2/fragment.rs rename to src/meta/model_v2/src/fragment.rs index 9263dd99eabb8..c590a58da771e 100644 --- a/src/meta/src/model_v2/fragment.rs +++ b/src/meta/model_v2/src/fragment.rs @@ -14,7 +14,7 @@ use sea_orm::entity::prelude::*; -use crate::model_v2::I32Array; +use crate::I32Array; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "fragment")] diff --git a/src/meta/src/model_v2/function.rs b/src/meta/model_v2/src/function.rs similarity index 97% rename from src/meta/src/model_v2/function.rs rename to src/meta/model_v2/src/function.rs index 663f8e2284fd7..4126dddc0f5ee 100644 --- a/src/meta/src/model_v2/function.rs +++ b/src/meta/model_v2/src/function.rs @@ -17,7 +17,7 @@ use risingwave_pb::catalog::PbFunction; use sea_orm::entity::prelude::*; use sea_orm::ActiveValue; -use crate::model_v2::{DataType, DataTypeArray, FunctionId}; +use crate::{DataType, DataTypeArray, FunctionId}; #[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum)] #[sea_orm(rs_type = "String", db_type = "String(None)")] diff --git a/src/meta/src/model_v2/hummock_pinned_snapshot.rs b/src/meta/model_v2/src/hummock_pinned_snapshot.rs similarity index 100% rename from src/meta/src/model_v2/hummock_pinned_snapshot.rs rename to src/meta/model_v2/src/hummock_pinned_snapshot.rs diff --git a/src/meta/src/model_v2/hummock_pinned_version.rs b/src/meta/model_v2/src/hummock_pinned_version.rs similarity index 100% rename from src/meta/src/model_v2/hummock_pinned_version.rs rename to src/meta/model_v2/src/hummock_pinned_version.rs diff --git a/src/meta/src/model_v2/hummock_version_delta.rs b/src/meta/model_v2/src/hummock_version_delta.rs similarity index 100% rename from src/meta/src/model_v2/hummock_version_delta.rs rename to src/meta/model_v2/src/hummock_version_delta.rs diff --git a/src/meta/src/model_v2/hummock_version_stats.rs b/src/meta/model_v2/src/hummock_version_stats.rs similarity index 100% rename from src/meta/src/model_v2/hummock_version_stats.rs rename to src/meta/model_v2/src/hummock_version_stats.rs diff --git a/src/meta/src/model_v2/index.rs b/src/meta/model_v2/src/index.rs similarity index 96% rename from src/meta/src/model_v2/index.rs rename to src/meta/model_v2/src/index.rs index 3b80632e2cfc3..c85a896914240 100644 --- a/src/meta/src/model_v2/index.rs +++ b/src/meta/model_v2/src/index.rs @@ -14,7 +14,7 @@ use sea_orm::entity::prelude::*; -use crate::model_v2::{ExprNodeArray, I32Array, IndexId, JobStatus, TableId}; +use crate::{ExprNodeArray, I32Array, IndexId, JobStatus, TableId}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "index")] diff --git a/src/meta/src/model_v2/mod.rs b/src/meta/model_v2/src/lib.rs similarity index 99% rename from src/meta/src/model_v2/mod.rs rename to src/meta/model_v2/src/lib.rs index 1c2f928063fff..5fe23bcaa280c 100644 --- a/src/meta/src/model_v2/mod.rs +++ b/src/meta/model_v2/src/lib.rs @@ -27,7 +27,6 @@ pub mod compaction_status; pub mod compaction_task; pub mod connection; pub mod database; -pub mod ext; pub mod fragment; pub mod function; pub mod hummock_pinned_snapshot; @@ -42,7 +41,6 @@ pub mod sink; pub mod source; pub mod system_parameter; pub mod table; -pub mod trx; pub mod user; pub mod user_privilege; pub mod view; diff --git a/src/meta/src/model_v2/object.rs b/src/meta/model_v2/src/object.rs similarity index 98% rename from src/meta/src/model_v2/object.rs rename to src/meta/model_v2/src/object.rs index 5048f93a483d9..39506777068a3 100644 --- a/src/meta/src/model_v2/object.rs +++ b/src/meta/model_v2/src/object.rs @@ -14,7 +14,7 @@ use sea_orm::entity::prelude::*; -use crate::model_v2::{DatabaseId, ObjectId, SchemaId, UserId}; +use crate::{DatabaseId, ObjectId, SchemaId, UserId}; #[derive(Clone, Debug, PartialEq, Eq, Copy, EnumIter, DeriveActiveEnum)] #[sea_orm(rs_type = "String", db_type = "String(None)")] diff --git a/src/meta/src/model_v2/object_dependency.rs b/src/meta/model_v2/src/object_dependency.rs similarity index 97% rename from src/meta/src/model_v2/object_dependency.rs rename to src/meta/model_v2/src/object_dependency.rs index 53800112a7370..52ca229c6997a 100644 --- a/src/meta/src/model_v2/object_dependency.rs +++ b/src/meta/model_v2/src/object_dependency.rs @@ -14,7 +14,7 @@ use sea_orm::entity::prelude::*; -use crate::model_v2::{ObjectId, UserId}; +use crate::{ObjectId, UserId}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "object_dependency")] diff --git a/src/meta/src/model_v2/prelude.rs b/src/meta/model_v2/src/prelude.rs similarity index 100% rename from src/meta/src/model_v2/prelude.rs rename to src/meta/model_v2/src/prelude.rs diff --git a/src/meta/src/model_v2/schema.rs b/src/meta/model_v2/src/schema.rs similarity index 81% rename from src/meta/src/model_v2/schema.rs rename to src/meta/model_v2/src/schema.rs index 2c28665fd06f0..0af2d7fc020c9 100644 --- a/src/meta/src/model_v2/schema.rs +++ b/src/meta/model_v2/src/schema.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_pb::catalog::PbSchema; use sea_orm::entity::prelude::*; +use sea_orm::ActiveValue; -use crate::model_v2::SchemaId; +use crate::SchemaId; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "schema")] @@ -43,3 +45,12 @@ impl Related for Entity { } impl ActiveModelBehavior for ActiveModel {} + +impl From for ActiveModel { + fn from(schema: PbSchema) -> Self { + Self { + schema_id: ActiveValue::Set(schema.id), + name: ActiveValue::Set(schema.name), + } + } +} diff --git a/src/meta/src/model_v2/sink.rs b/src/meta/model_v2/src/sink.rs similarity index 99% rename from src/meta/src/model_v2/sink.rs rename to src/meta/model_v2/src/sink.rs index bef46f1d7195f..21ac172246703 100644 --- a/src/meta/src/model_v2/sink.rs +++ b/src/meta/model_v2/src/sink.rs @@ -15,7 +15,7 @@ use risingwave_pb::catalog::PbSinkType; use sea_orm::entity::prelude::*; -use crate::model_v2::{ +use crate::{ ColumnCatalogArray, ColumnOrderArray, ConnectionId, I32Array, JobStatus, Property, SinkFormatDesc, SinkId, }; diff --git a/src/meta/src/model_v2/source.rs b/src/meta/model_v2/src/source.rs similarity index 99% rename from src/meta/src/model_v2/source.rs rename to src/meta/model_v2/src/source.rs index 2ad1de7914d96..620d002c27b55 100644 --- a/src/meta/src/model_v2/source.rs +++ b/src/meta/model_v2/src/source.rs @@ -14,7 +14,7 @@ use sea_orm::entity::prelude::*; -use crate::model_v2::{ +use crate::{ ColumnCatalogArray, ConnectionId, I32Array, Property, SourceId, StreamSourceInfo, TableId, WatermarkDescArray, }; diff --git a/src/meta/src/model_v2/system_parameter.rs b/src/meta/model_v2/src/system_parameter.rs similarity index 100% rename from src/meta/src/model_v2/system_parameter.rs rename to src/meta/model_v2/src/system_parameter.rs diff --git a/src/meta/src/model_v2/table.rs b/src/meta/model_v2/src/table.rs similarity index 99% rename from src/meta/src/model_v2/table.rs rename to src/meta/model_v2/src/table.rs index 08caee7009f8f..a335f41023442 100644 --- a/src/meta/src/model_v2/table.rs +++ b/src/meta/model_v2/src/table.rs @@ -16,7 +16,7 @@ use risingwave_pb::catalog::table::PbTableType; use risingwave_pb::catalog::PbHandleConflictBehavior; use sea_orm::entity::prelude::*; -use crate::model_v2::{ +use crate::{ Cardinality, ColumnCatalogArray, ColumnOrderArray, CreateType, I32Array, JobStatus, Property, SourceId, TableId, TableVersion, }; diff --git a/src/meta/src/model_v2/user.rs b/src/meta/model_v2/src/user.rs similarity index 97% rename from src/meta/src/model_v2/user.rs rename to src/meta/model_v2/src/user.rs index 0e7ab4dd17876..e9cd36f75fb43 100644 --- a/src/meta/src/model_v2/user.rs +++ b/src/meta/model_v2/src/user.rs @@ -14,7 +14,7 @@ use sea_orm::entity::prelude::*; -use crate::model_v2::UserId; +use crate::UserId; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "user")] diff --git a/src/meta/src/model_v2/user_privilege.rs b/src/meta/model_v2/src/user_privilege.rs similarity index 97% rename from src/meta/src/model_v2/user_privilege.rs rename to src/meta/model_v2/src/user_privilege.rs index 335f716cec1c8..7e12af225ed02 100644 --- a/src/meta/src/model_v2/user_privilege.rs +++ b/src/meta/model_v2/src/user_privilege.rs @@ -14,7 +14,7 @@ use sea_orm::entity::prelude::*; -use crate::model_v2::{ObjectId, UserId}; +use crate::{ObjectId, UserId}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "user_privilege")] diff --git a/src/meta/src/model_v2/view.rs b/src/meta/model_v2/src/view.rs similarity index 97% rename from src/meta/src/model_v2/view.rs rename to src/meta/model_v2/src/view.rs index 8f7d22408d3f2..0de9ea64a616e 100644 --- a/src/meta/src/model_v2/view.rs +++ b/src/meta/model_v2/src/view.rs @@ -16,7 +16,7 @@ use risingwave_pb::catalog::PbView; use sea_orm::entity::prelude::*; use sea_orm::ActiveValue; -use crate::model_v2::{FieldArray, Property, ViewId}; +use crate::{FieldArray, Property, ViewId}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "view")] diff --git a/src/meta/model_v2/src/worker.rs b/src/meta/model_v2/src/worker.rs new file mode 100644 index 0000000000000..d164fba62b41e --- /dev/null +++ b/src/meta/model_v2/src/worker.rs @@ -0,0 +1,128 @@ +// 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 risingwave_pb::common::worker_node::PbState; +use risingwave_pb::common::{PbWorkerNode, PbWorkerType}; +use sea_orm::entity::prelude::*; +use sea_orm::ActiveValue; + +use crate::{TransactionId, WorkerId}; + +#[derive(Clone, Debug, Hash, PartialEq, Eq, EnumIter, DeriveActiveEnum)] +#[sea_orm(rs_type = "String", db_type = "String(None)")] +pub enum WorkerType { + #[sea_orm(string_value = "FRONTEND")] + Frontend, + #[sea_orm(string_value = "COMPUTE_NODE")] + ComputeNode, + #[sea_orm(string_value = "RISE_CTL")] + RiseCtl, + #[sea_orm(string_value = "COMPACTOR")] + Compactor, + #[sea_orm(string_value = "META")] + Meta, +} + +impl From for WorkerType { + fn from(worker_type: PbWorkerType) -> Self { + match worker_type { + PbWorkerType::Unspecified => unreachable!("unspecified worker type"), + PbWorkerType::Frontend => Self::Frontend, + PbWorkerType::ComputeNode => Self::ComputeNode, + PbWorkerType::RiseCtl => Self::RiseCtl, + PbWorkerType::Compactor => Self::Compactor, + PbWorkerType::Meta => Self::Meta, + } + } +} + +impl From for PbWorkerType { + fn from(worker_type: WorkerType) -> Self { + match worker_type { + WorkerType::Frontend => Self::Frontend, + WorkerType::ComputeNode => Self::ComputeNode, + WorkerType::RiseCtl => Self::RiseCtl, + WorkerType::Compactor => Self::Compactor, + WorkerType::Meta => Self::Meta, + } + } +} + +#[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum)] +#[sea_orm(rs_type = "String", db_type = "String(None)")] +pub enum WorkerStatus { + #[sea_orm(string_value = "STARTING")] + Starting, + #[sea_orm(string_value = "RUNNING")] + Running, +} + +impl From for WorkerStatus { + fn from(state: PbState) -> Self { + match state { + PbState::Unspecified => unreachable!("unspecified worker status"), + PbState::Starting => Self::Starting, + PbState::Running => Self::Running, + } + } +} + +impl From for PbState { + fn from(status: WorkerStatus) -> Self { + match status { + WorkerStatus::Starting => Self::Starting, + WorkerStatus::Running => Self::Running, + } + } +} + +impl From<&PbWorkerNode> for ActiveModel { + fn from(worker: &PbWorkerNode) -> Self { + let host = worker.host.clone().unwrap(); + Self { + worker_id: ActiveValue::Set(worker.id), + worker_type: ActiveValue::Set(worker.r#type().into()), + host: ActiveValue::Set(host.host), + port: ActiveValue::Set(host.port), + status: ActiveValue::Set(worker.state().into()), + ..Default::default() + } + } +} + +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] +#[sea_orm(table_name = "worker")] +pub struct Model { + #[sea_orm(primary_key)] + pub worker_id: WorkerId, + pub worker_type: WorkerType, + pub host: String, + pub port: i32, + pub status: WorkerStatus, + pub transaction_id: Option, +} + +#[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] +pub enum Relation { + #[sea_orm(has_many = "super::worker_property::Entity")] + WorkerProperty, +} + +impl Related for Entity { + fn to() -> RelationDef { + Relation::WorkerProperty.def() + } +} + +impl ActiveModelBehavior for ActiveModel {} diff --git a/src/meta/src/model_v2/worker_property.rs b/src/meta/model_v2/src/worker_property.rs similarity index 97% rename from src/meta/src/model_v2/worker_property.rs rename to src/meta/model_v2/src/worker_property.rs index 8521cbed15ce2..0512ea97e5be3 100644 --- a/src/meta/src/model_v2/worker_property.rs +++ b/src/meta/model_v2/src/worker_property.rs @@ -14,7 +14,7 @@ use sea_orm::entity::prelude::*; -use crate::model_v2::{I32Array, WorkerId}; +use crate::{I32Array, WorkerId}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "worker_property")] diff --git a/src/meta/node/Cargo.toml b/src/meta/node/Cargo.toml index 8c2a5aeadbe41..84793a74591c8 100644 --- a/src/meta/node/Cargo.toml +++ b/src/meta/node/Cargo.toml @@ -20,13 +20,13 @@ either = "1" etcd-client = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } itertools = "0.11" -model_migration = { path = "../src/model_v2/migration" } prometheus-http-query = "0.7" regex = "1" risingwave_common = { workspace = true } risingwave_common_heap_profiling = { workspace = true } risingwave_common_service = { workspace = true } risingwave_meta = { workspace = true } +risingwave_meta_model_migration = { workspace = true } risingwave_meta_service = { workspace = true } risingwave_pb = { workspace = true } risingwave_rpc_client = { workspace = true } diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index d922f1c37e033..d8d8525aca235 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -19,7 +19,6 @@ use either::Either; use etcd_client::ConnectOptions; use futures::future::join_all; use itertools::Itertools; -use model_migration::{Migrator, MigratorTrait}; use regex::Regex; use risingwave_common::monitor::connection::{RouterExt, TcpConfig}; use risingwave_common::telemetry::manager::TelemetryManager; @@ -28,6 +27,7 @@ use risingwave_common_service::metrics_manager::MetricsManager; use risingwave_common_service::tracing::TracingExtractLayer; use risingwave_meta::rpc::intercept::MetricsMiddlewareLayer; use risingwave_meta::rpc::ElectionClientRef; +use risingwave_meta_model_migration::{Migrator, MigratorTrait}; use risingwave_meta_service::backup_service::BackupServiceImpl; use risingwave_meta_service::cloud_service::CloudServiceImpl; use risingwave_meta_service::cluster_service::ClusterServiceImpl; diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml index 1760ccd56a85a..87b293f64a5e6 100644 --- a/src/meta/service/Cargo.toml +++ b/src/meta/service/Cargo.toml @@ -23,6 +23,7 @@ regex = "1" risingwave_common = { workspace = true } risingwave_connector = { workspace = true } risingwave_meta = { workspace = true } +risingwave_meta_model_v2 = { workspace = true } risingwave_pb = { workspace = true } sea-orm = { version = "0.12.0", features = [ "sqlx-mysql", diff --git a/src/meta/service/src/telemetry_service.rs b/src/meta/service/src/telemetry_service.rs index 7c413406f13e5..42200e10a4eeb 100644 --- a/src/meta/service/src/telemetry_service.rs +++ b/src/meta/service/src/telemetry_service.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_meta_model_v2::prelude::Cluster; use risingwave_pb::meta::telemetry_info_service_server::TelemetryInfoService; use risingwave_pb::meta::{GetTelemetryInfoRequest, TelemetryInfoResponse}; use sea_orm::EntityTrait; @@ -19,7 +20,6 @@ use tonic::{Request, Response, Status}; use crate::controller::SqlMetaStore; use crate::model::ClusterId; -use crate::model_v2::prelude::Cluster; use crate::storage::MetaStoreRef; use crate::MetaResult; diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index cb37307384aa2..daaa9b684850c 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -17,6 +17,13 @@ use std::iter; use itertools::Itertools; use risingwave_common::bail; use risingwave_common::catalog::{DEFAULT_SCHEMA_NAME, SYSTEM_SCHEMAS}; +use risingwave_meta_model_v2::object::ObjectType; +use risingwave_meta_model_v2::prelude::*; +use risingwave_meta_model_v2::{ + connection, database, function, index, object, object_dependency, schema, sink, source, table, + view, ConnectionId, DatabaseId, FunctionId, ObjectId, PrivateLinkService, SchemaId, SourceId, + TableId, UserId, +}; use risingwave_pb::catalog::{ PbConnection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, }; @@ -40,13 +47,6 @@ use crate::controller::utils::{ }; use crate::controller::ObjectModel; use crate::manager::{MetaSrvEnv, NotificationVersion}; -use crate::model_v2::object::ObjectType; -use crate::model_v2::prelude::*; -use crate::model_v2::{ - connection, database, function, index, object, object_dependency, schema, sink, source, table, - view, ConnectionId, DatabaseId, FunctionId, ObjectId, PrivateLinkService, SchemaId, SourceId, - TableId, UserId, -}; use crate::rpc::ddl_controller::DropMode; use crate::{MetaError, MetaResult}; diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index ca29380a49fca..392a0def5d53f 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -22,6 +22,9 @@ use std::time::{Duration, SystemTime}; use itertools::Itertools; use risingwave_common::hash::ParallelUnitId; use risingwave_hummock_sdk::HummockSstableObjectId; +use risingwave_meta_model_v2::prelude::{Worker, WorkerProperty}; +use risingwave_meta_model_v2::worker::{WorkerStatus, WorkerType}; +use risingwave_meta_model_v2::{worker, worker_property, I32Array, TransactionId, WorkerId}; use risingwave_pb::common::worker_node::{PbProperty, PbState}; use risingwave_pb::common::{ HostAddress, ParallelUnit, PbHostAddress, PbParallelUnit, PbWorkerNode, PbWorkerType, @@ -39,10 +42,7 @@ use tokio::sync::oneshot::Sender; use tokio::sync::{RwLock, RwLockReadGuard}; use tokio::task::JoinHandle; -use crate::manager::prelude::{Worker, WorkerProperty}; use crate::manager::{LocalNotification, MetaSrvEnv, WorkerKey}; -use crate::model_v2::worker::{WorkerStatus, WorkerType}; -use crate::model_v2::{worker, worker_property, I32Array, TransactionId, WorkerId}; use crate::{MetaError, MetaResult}; pub type ClusterControllerRef = Arc; @@ -89,64 +89,6 @@ impl From for PbWorkerNode { } } -impl From for WorkerType { - fn from(worker_type: PbWorkerType) -> Self { - match worker_type { - PbWorkerType::Unspecified => unreachable!("unspecified worker type"), - PbWorkerType::Frontend => Self::Frontend, - PbWorkerType::ComputeNode => Self::ComputeNode, - PbWorkerType::RiseCtl => Self::RiseCtl, - PbWorkerType::Compactor => Self::Compactor, - PbWorkerType::Meta => Self::Meta, - } - } -} - -impl From for PbWorkerType { - fn from(worker_type: WorkerType) -> Self { - match worker_type { - WorkerType::Frontend => Self::Frontend, - WorkerType::ComputeNode => Self::ComputeNode, - WorkerType::RiseCtl => Self::RiseCtl, - WorkerType::Compactor => Self::Compactor, - WorkerType::Meta => Self::Meta, - } - } -} - -impl From for WorkerStatus { - fn from(state: PbState) -> Self { - match state { - PbState::Unspecified => unreachable!("unspecified worker status"), - PbState::Starting => Self::Starting, - PbState::Running => Self::Running, - } - } -} - -impl From for PbState { - fn from(status: WorkerStatus) -> Self { - match status { - WorkerStatus::Starting => Self::Starting, - WorkerStatus::Running => Self::Running, - } - } -} - -impl From<&PbWorkerNode> for worker::ActiveModel { - fn from(worker: &PbWorkerNode) -> Self { - let host = worker.host.clone().unwrap(); - Self { - worker_id: ActiveValue::Set(worker.id), - worker_type: ActiveValue::Set(worker.r#type().into()), - host: ActiveValue::Set(host.host), - port: ActiveValue::Set(host.port), - status: ActiveValue::Set(worker.state().into()), - ..Default::default() - } - } -} - impl ClusterController { pub async fn new(env: MetaSrvEnv, max_heartbeat_interval: Duration) -> MetaResult { let meta_store = env diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 07793e30a17fe..d9193acd5591f 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -14,6 +14,9 @@ use anyhow::anyhow; use risingwave_common::util::epoch::Epoch; +use risingwave_meta_model_v2::{ + connection, database, index, object, schema, sink, source, table, view, +}; use risingwave_pb::catalog::connection::PbInfo as PbConnectionInfo; use risingwave_pb::catalog::source::PbOptionalAssociatedTableId; use risingwave_pb::catalog::table::{PbOptionalAssociatedSourceId, PbTableType}; @@ -21,9 +24,8 @@ use risingwave_pb::catalog::{ PbConnection, PbCreateType, PbDatabase, PbHandleConflictBehavior, PbIndex, PbSchema, PbSink, PbSinkType, PbSource, PbStreamJobStatus, PbTable, PbView, }; -use sea_orm::{ActiveValue, DatabaseConnection, ModelTrait}; +use sea_orm::{DatabaseConnection, ModelTrait}; -use crate::model_v2::{connection, database, index, object, schema, sink, source, table, view}; use crate::MetaError; #[allow(dead_code)] @@ -56,7 +58,7 @@ impl SqlMetaStore { #[cfg(any(test, feature = "test"))] #[cfg(not(madsim))] pub async fn for_test() -> Self { - use model_migration::{Migrator, MigratorTrait}; + use risingwave_meta_model_migration::{Migrator, MigratorTrait}; let conn = sea_orm::Database::connect("sqlite::memory:").await.unwrap(); Migrator::up(&conn, None).await.unwrap(); Self { conn } @@ -75,24 +77,6 @@ impl From> for PbDatabase { } } -impl From for database::ActiveModel { - fn from(db: PbDatabase) -> Self { - Self { - database_id: ActiveValue::Set(db.id), - name: ActiveValue::Set(db.name), - } - } -} - -impl From for schema::ActiveModel { - fn from(schema: PbSchema) -> Self { - Self { - schema_id: ActiveValue::Set(schema.id), - name: ActiveValue::Set(schema.name), - } - } -} - impl From> for PbSchema { fn from(value: ObjectModel) -> Self { Self { diff --git a/src/meta/src/controller/system_param.rs b/src/meta/src/controller/system_param.rs index 0656da5ea9a46..5c9761a9a119d 100644 --- a/src/meta/src/controller/system_param.rs +++ b/src/meta/src/controller/system_param.rs @@ -21,6 +21,8 @@ use risingwave_common::system_param::{ check_missing_params, derive_missing_fields, set_system_param, }; use risingwave_common::{for_all_params, key_of}; +use risingwave_meta_model_v2::prelude::SystemParameter; +use risingwave_meta_model_v2::system_parameter; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::PbSystemParams; use sea_orm::{ActiveModelTrait, ActiveValue, DatabaseConnection, EntityTrait, TransactionTrait}; @@ -31,8 +33,6 @@ use tracing::info; use crate::controller::SqlMetaStore; use crate::manager::{LocalNotification, NotificationManagerRef}; -use crate::model_v2::prelude::SystemParameter; -use crate::model_v2::system_parameter; use crate::{MetaError, MetaResult}; pub type SystemParamsControllerRef = Arc; diff --git a/src/meta/src/controller/utils.rs b/src/meta/src/controller/utils.rs index d36918db3820d..2dbd89ac92423 100644 --- a/src/meta/src/controller/utils.rs +++ b/src/meta/src/controller/utils.rs @@ -13,7 +13,13 @@ // limitations under the License. use anyhow::anyhow; -use model_migration::WithQuery; +use risingwave_meta_model_migration::WithQuery; +use risingwave_meta_model_v2::object::ObjectType; +use risingwave_meta_model_v2::prelude::*; +use risingwave_meta_model_v2::{ + connection, function, index, object, object_dependency, schema, sink, source, table, view, + DataTypeArray, DatabaseId, ObjectId, SchemaId, UserId, +}; use risingwave_pb::catalog::{PbConnection, PbFunction}; use sea_orm::sea_query::{ Alias, CommonTableExpression, Expr, Query, QueryStatementBuilder, SelectStatement, UnionType, @@ -24,12 +30,6 @@ use sea_orm::{ Order, PaginatorTrait, QueryFilter, QuerySelect, RelationTrait, Statement, }; -use crate::model_v2::object::ObjectType; -use crate::model_v2::prelude::*; -use crate::model_v2::{ - connection, function, index, object, object_dependency, schema, sink, source, table, view, - DataTypeArray, DatabaseId, ObjectId, SchemaId, UserId, -}; use crate::{MetaError, MetaResult}; /// This function will construct a query using recursive cte to find all objects[(id, `obj_type`)] that are used by the given object. diff --git a/src/meta/src/lib.rs b/src/meta/src/lib.rs index f549578f079c6..95b4ce7ead72d 100644 --- a/src/meta/src/lib.rs +++ b/src/meta/src/lib.rs @@ -42,7 +42,6 @@ pub mod error; pub mod hummock; pub mod manager; pub mod model; -pub mod model_v2; pub mod rpc; pub mod serving; pub mod storage; diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index 16a4bcb248b23..28d8200c73ea5 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -16,6 +16,7 @@ use std::ops::Deref; use std::sync::Arc; use risingwave_common::config::{CompactionConfig, DefaultParallelism}; +use risingwave_meta_model_v2::prelude::Cluster; use risingwave_pb::meta::SystemParams; use risingwave_rpc_client::{ConnectorClient, StreamClientPool, StreamClientPoolRef}; use sea_orm::EntityTrait; @@ -28,7 +29,6 @@ use crate::manager::{ NotificationManagerRef, }; use crate::model::ClusterId; -use crate::model_v2::prelude::Cluster; use crate::storage::MetaStoreRef; #[cfg(any(test, feature = "test"))] use crate::storage::{MemStore, MetaStoreBoxExt}; diff --git a/src/meta/src/manager/mod.rs b/src/meta/src/manager/mod.rs index 35642ed0ec143..e7e5208856bc3 100644 --- a/src/meta/src/manager/mod.rs +++ b/src/meta/src/manager/mod.rs @@ -28,7 +28,6 @@ pub use env::{MetaSrvEnv, *}; pub use id::*; pub use idle::*; pub use notification::{LocalNotification, MessageStatus, NotificationManagerRef, *}; +pub use risingwave_meta_model_v2::prelude; pub use streaming_job::*; pub use system_param::*; - -pub use super::model_v2::prelude; diff --git a/src/meta/src/model_v2/ext/hummock.rs b/src/meta/src/model_v2/ext/hummock.rs deleted file mode 100644 index 77111e2e7d202..0000000000000 --- a/src/meta/src/model_v2/ext/hummock.rs +++ /dev/null @@ -1,61 +0,0 @@ -// 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 risingwave_pb::hummock::HummockPinnedVersion; -use sea_orm::sea_query::OnConflict; -use sea_orm::ActiveValue::{Set, Unchanged}; -use sea_orm::EntityTrait; - -use crate::model::{MetadataModelResult, Transactional}; -use crate::model_v2::hummock_pinned_version; -use crate::model_v2::trx::Transaction; - -#[async_trait::async_trait] -impl Transactional for HummockPinnedVersion { - async fn upsert_in_transaction( - &self, - trx: &mut crate::model_v2::trx::Transaction, - ) -> MetadataModelResult<()> { - // TODO: error type conversion - // TODO: integer type conversion - let m = hummock_pinned_version::ActiveModel { - context_id: Unchanged(self.context_id.try_into().unwrap()), - min_pinned_id: Set(self.min_pinned_id.try_into().unwrap()), - }; - hummock_pinned_version::Entity::insert(m) - .on_conflict( - OnConflict::column(hummock_pinned_version::Column::ContextId) - .update_columns([hummock_pinned_version::Column::MinPinnedId]) - .to_owned(), - ) - .exec(trx) - .await - .unwrap(); - Ok(()) - } - - async fn delete_in_transaction( - &self, - trx: &mut crate::model_v2::trx::Transaction, - ) -> MetadataModelResult<()> { - // TODO: error type conversion - // TODO: integer type conversion - let id: i32 = self.context_id.try_into().unwrap(); - hummock_pinned_version::Entity::delete_by_id(id) - .exec(trx) - .await - .unwrap(); - Ok(()) - } -} diff --git a/src/meta/src/model_v2/ext/mod.rs b/src/meta/src/model_v2/ext/mod.rs deleted file mode 100644 index 47a5ce8623dc4..0000000000000 --- a/src/meta/src/model_v2/ext/mod.rs +++ /dev/null @@ -1,16 +0,0 @@ -// 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. - -mod hummock; -pub use hummock::*; diff --git a/src/meta/src/model_v2/migration/Cargo.toml b/src/meta/src/model_v2/migration/Cargo.toml deleted file mode 100644 index d5d51d77da909..0000000000000 --- a/src/meta/src/model_v2/migration/Cargo.toml +++ /dev/null @@ -1,17 +0,0 @@ -[package] -name = "model_migration" -version = "0.1.0" -edition = "2021" -publish = false - -[lib] -name = "model_migration" -path = "src/lib.rs" - -[dependencies] -async-std = { version = "1", features = ["attributes", "tokio1"] } -uuid = { version = "1", features = ["v4"] } - -[dependencies.sea-orm-migration] -version = "0.12.0" -features = ["sqlx-mysql", "sqlx-postgres", "sqlx-sqlite", "runtime-tokio-native-tls", "with-uuid"] diff --git a/src/meta/src/model_v2/trx.rs b/src/meta/src/model_v2/trx.rs deleted file mode 100644 index 4bfe6d0261de4..0000000000000 --- a/src/meta/src/model_v2/trx.rs +++ /dev/null @@ -1,276 +0,0 @@ -// 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. - -pub type Transaction = sea_orm::DatabaseTransaction; - -#[cfg(not(madsim))] -#[cfg(test)] -mod tests { - use std::collections::BTreeMap; - - use risingwave_pb::hummock::HummockPinnedVersion; - use sea_orm::{EntityTrait, TransactionTrait}; - - use crate::controller::SqlMetaStore; - use crate::model::{BTreeMapTransaction, ValTransaction, VarTransaction}; - use crate::model_v2::hummock_pinned_version::Model as HummockPinnedVersionModel; - use crate::model_v2::prelude::HummockPinnedVersion as HummockPinnedVersionEntity; - use crate::model_v2::trx::Transaction; - - #[tokio::test] - async fn test_simple_var_transaction_commit() { - let store = SqlMetaStore::for_test().await; - let db = &store.conn; - let mut kv = HummockPinnedVersion { - context_id: 1, - min_pinned_id: 2, - }; - let mut num_txn = VarTransaction::<'_, Transaction, _>::new(&mut kv); - num_txn.min_pinned_id = 3; - assert_eq!(num_txn.min_pinned_id, 3); - let mut txn = db.begin().await.unwrap(); - num_txn.apply_to_txn(&mut txn).await.unwrap(); - txn.commit().await.unwrap(); - let db_val = HummockPinnedVersionEntity::find_by_id(1) - .one(db) - .await - .unwrap() - .unwrap(); - assert_eq!(db_val.min_pinned_id, 3); - num_txn.commit(); - assert_eq!(kv.min_pinned_id, 3); - } - - #[test] - fn test_simple_var_transaction_abort() { - let mut kv = HummockPinnedVersion { - context_id: 1, - min_pinned_id: 11, - }; - let mut num_txn = VarTransaction::<'_, Transaction, _>::new(&mut kv); - num_txn.min_pinned_id = 2; - num_txn.abort(); - assert_eq!(11, kv.min_pinned_id); - } - - #[tokio::test] - async fn test_tree_map_transaction_commit() { - let mut map: BTreeMap = BTreeMap::new(); - // to remove - map.insert( - 1, - HummockPinnedVersion { - context_id: 1, - min_pinned_id: 11, - }, - ); - // to-remove-after-modify - map.insert( - 2, - HummockPinnedVersion { - context_id: 2, - min_pinned_id: 22, - }, - ); - // first - map.insert( - 3, - HummockPinnedVersion { - context_id: 3, - min_pinned_id: 33, - }, - ); - - let mut map_copy = map.clone(); - let mut map_txn = BTreeMapTransaction::new(&mut map); - map_txn.remove(1); - map_txn.insert( - 2, - HummockPinnedVersion { - context_id: 2, - min_pinned_id: 0, - }, - ); - map_txn.remove(2); - // first - map_txn.insert( - 3, - HummockPinnedVersion { - context_id: 3, - min_pinned_id: 333, - }, - ); - // second - map_txn.insert( - 4, - HummockPinnedVersion { - context_id: 4, - min_pinned_id: 44, - }, - ); - assert_eq!( - &HummockPinnedVersion { - context_id: 4, - min_pinned_id: 44 - }, - map_txn.get(&4).unwrap() - ); - // third - map_txn.insert( - 5, - HummockPinnedVersion { - context_id: 5, - min_pinned_id: 55, - }, - ); - assert_eq!( - &HummockPinnedVersion { - context_id: 5, - min_pinned_id: 55 - }, - map_txn.get(&5).unwrap() - ); - - let mut third_entry = map_txn.get_mut(5).unwrap(); - third_entry.min_pinned_id = 555; - assert_eq!( - &HummockPinnedVersion { - context_id: 5, - min_pinned_id: 555 - }, - map_txn.get(&5).unwrap() - ); - - let store = SqlMetaStore::for_test().await; - let db = &store.conn; - let mut txn = db.begin().await.unwrap(); - map_txn.apply_to_txn(&mut txn).await.unwrap(); - txn.commit().await.unwrap(); - - let db_rows: Vec = - HummockPinnedVersionEntity::find().all(db).await.unwrap(); - assert_eq!(db_rows.len(), 3); - assert_eq!( - 1, - db_rows - .iter() - .filter(|m| m.context_id == 3 && m.min_pinned_id == 333) - .count() - ); - assert_eq!( - 1, - db_rows - .iter() - .filter(|m| m.context_id == 4 && m.min_pinned_id == 44) - .count() - ); - assert_eq!( - 1, - db_rows - .iter() - .filter(|m| m.context_id == 5 && m.min_pinned_id == 555) - .count() - ); - map_txn.commit(); - - // replay the change to local copy and compare - map_copy.remove(&1).unwrap(); - map_copy.insert( - 2, - HummockPinnedVersion { - context_id: 2, - min_pinned_id: 22, - }, - ); - map_copy.remove(&2).unwrap(); - map_copy.insert( - 3, - HummockPinnedVersion { - context_id: 3, - min_pinned_id: 333, - }, - ); - map_copy.insert( - 4, - HummockPinnedVersion { - context_id: 4, - min_pinned_id: 44, - }, - ); - map_copy.insert( - 5, - HummockPinnedVersion { - context_id: 5, - min_pinned_id: 555, - }, - ); - assert_eq!(map_copy, map); - } - - #[tokio::test] - async fn test_tree_map_entry_update_transaction_commit() { - let mut map: BTreeMap = BTreeMap::new(); - map.insert( - 1, - HummockPinnedVersion { - context_id: 1, - min_pinned_id: 11, - }, - ); - - let mut map_txn = BTreeMapTransaction::new(&mut map); - let mut first_entry_txn = map_txn.new_entry_txn(1).unwrap(); - first_entry_txn.min_pinned_id = 111; - - let store = SqlMetaStore::for_test().await; - let db = &store.conn; - let mut txn = db.begin().await.unwrap(); - first_entry_txn.apply_to_txn(&mut txn).await.unwrap(); - txn.commit().await.unwrap(); - first_entry_txn.commit(); - - let db_rows: Vec = - HummockPinnedVersionEntity::find().all(db).await.unwrap(); - assert_eq!(db_rows.len(), 1); - assert_eq!( - 1, - db_rows - .iter() - .filter(|m| m.context_id == 1 && m.min_pinned_id == 111) - .count() - ); - assert_eq!(111, map.get(&1).unwrap().min_pinned_id); - } - - #[tokio::test] - async fn test_tree_map_entry_insert_transaction_commit() { - let mut map: BTreeMap = BTreeMap::new(); - - let mut map_txn = BTreeMapTransaction::new(&mut map); - let first_entry_txn = map_txn.new_entry_insert_txn( - 1, - HummockPinnedVersion { - context_id: 1, - min_pinned_id: 11, - }, - ); - let store = SqlMetaStore::for_test().await; - let db = &store.conn; - let mut txn = db.begin().await.unwrap(); - first_entry_txn.apply_to_txn(&mut txn).await.unwrap(); - txn.commit().await.unwrap(); - first_entry_txn.commit(); - assert_eq!(11, map.get(&1).unwrap().min_pinned_id); - } -} diff --git a/src/meta/src/model_v2/worker.rs b/src/meta/src/model_v2/worker.rs deleted file mode 100644 index 08cdb2be34da1..0000000000000 --- a/src/meta/src/model_v2/worker.rs +++ /dev/null @@ -1,67 +0,0 @@ -// 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 sea_orm::entity::prelude::*; - -use crate::model_v2::{TransactionId, WorkerId}; - -#[derive(Clone, Debug, Hash, PartialEq, Eq, EnumIter, DeriveActiveEnum)] -#[sea_orm(rs_type = "String", db_type = "String(None)")] -pub enum WorkerType { - #[sea_orm(string_value = "FRONTEND")] - Frontend, - #[sea_orm(string_value = "COMPUTE_NODE")] - ComputeNode, - #[sea_orm(string_value = "RISE_CTL")] - RiseCtl, - #[sea_orm(string_value = "COMPACTOR")] - Compactor, - #[sea_orm(string_value = "META")] - Meta, -} - -#[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum)] -#[sea_orm(rs_type = "String", db_type = "String(None)")] -pub enum WorkerStatus { - #[sea_orm(string_value = "STARTING")] - Starting, - #[sea_orm(string_value = "RUNNING")] - Running, -} - -#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] -#[sea_orm(table_name = "worker")] -pub struct Model { - #[sea_orm(primary_key)] - pub worker_id: WorkerId, - pub worker_type: WorkerType, - pub host: String, - pub port: i32, - pub status: WorkerStatus, - pub transaction_id: Option, -} - -#[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] -pub enum Relation { - #[sea_orm(has_many = "super::worker_property::Entity")] - WorkerProperty, -} - -impl Related for Entity { - fn to() -> RelationDef { - Relation::WorkerProperty.def() - } -} - -impl ActiveModelBehavior for ActiveModel {} From b724be78a46b15d397ac509d79aabf6365d687cf Mon Sep 17 00:00:00 2001 From: jinser Date: Thu, 26 Oct 2023 00:26:15 +0800 Subject: [PATCH 25/48] feat: add `comment on` clause support (#12849) Co-authored-by: Richard Chien Co-authored-by: August --- .../batch/catalog/pg_description.slt.part | 81 +++++++++++++---- e2e_test/ddl/show.slt | 56 ++++++++---- e2e_test/extended_mode/basic.slt | 15 ++-- proto/catalog.proto | 11 +++ proto/ddl_service.proto | 10 +++ proto/plan_common.proto | 3 + src/common/src/catalog/column.rs | 8 ++ src/common/src/catalog/mod.rs | 2 + src/common/src/catalog/test_utils.rs | 1 + src/compute/tests/integration_tests.rs | 1 + src/connector/src/parser/avro/util.rs | 1 + src/connector/src/parser/protobuf/parser.rs | 1 + src/connector/src/source/manager.rs | 1 + src/frontend/src/binder/expr/mod.rs | 2 + src/frontend/src/catalog/catalog_service.rs | 10 ++- .../src/catalog/system_catalog/mod.rs | 5 ++ .../pg_catalog/pg_description.rs | 39 +++++---- .../catalog/system_catalog/rw_catalog/mod.rs | 2 + .../rw_catalog/rw_description.rs | 84 ++++++++++++++++++ src/frontend/src/catalog/table_catalog.rs | 8 ++ src/frontend/src/handler/comment.rs | 87 +++++++++++++++++++ src/frontend/src/handler/create_source.rs | 4 + src/frontend/src/handler/create_table.rs | 1 + src/frontend/src/handler/describe.rs | 77 ++++++++++------ src/frontend/src/handler/mod.rs | 6 ++ src/frontend/src/handler/util.rs | 1 + .../optimizer/plan_node/stream_materialize.rs | 1 + src/frontend/src/optimizer/plan_node/utils.rs | 1 + src/frontend/src/session.rs | 5 ++ src/frontend/src/test_utils.rs | 6 +- src/meta/service/src/ddl_service.rs | 26 +++++- src/meta/src/controller/mod.rs | 1 + src/meta/src/manager/catalog/mod.rs | 44 +++++++++- src/meta/src/rpc/ddl_controller.rs | 8 +- src/rpc_client/src/meta_client.rs | 12 ++- src/source/src/source_desc.rs | 1 + src/storage/src/filter_key_extractor.rs | 1 + .../src/delete_range_runner.rs | 1 + src/utils/pgwire/src/pg_response.rs | 1 + 39 files changed, 535 insertions(+), 90 deletions(-) create mode 100644 src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs create mode 100644 src/frontend/src/handler/comment.rs diff --git a/e2e_test/batch/catalog/pg_description.slt.part b/e2e_test/batch/catalog/pg_description.slt.part index 41ade5ffbb9b3..44ba006ea4367 100644 --- a/e2e_test/batch/catalog/pg_description.slt.part +++ b/e2e_test/batch/catalog/pg_description.slt.part @@ -1,18 +1,65 @@ -query IIITT -SELECT d.*, c.relname FROM pg_catalog.pg_description d join pg_catalog.pg_class c on d.objoid = c.oid ORDER BY d.objoid limit 15; +statement ok +create table t(a int, b text, c date); + +statement ok +comment on table t is 'Lorem ipsum'; + +statement ok +comment on column t.a is 'Praesent elementum'; + +statement ok +comment on column public.t.c is 'Nullam ultricies'; + +statement ok +comment on column public.t._row_id is 'facilisis enim'; + +query TIIIT +SELECT + c.relname, + ( + SELECT relname FROM pg_catalog.pg_class WHERE oid = d.classoid + ) AS classoid, + d.objsubid, + d.description +FROM + pg_catalog.pg_description d + JOIN pg_catalog.pg_class c + ON d.objoid = c.oid +ORDER BY d.objsubid; ---- -1 NULL 0 NULL pg_type -2 NULL 0 NULL pg_namespace -3 NULL 0 NULL pg_cast -4 NULL 0 NULL pg_matviews -5 NULL 0 NULL pg_user -6 NULL 0 NULL pg_class -7 NULL 0 NULL pg_index -8 NULL 0 NULL pg_opclass -9 NULL 0 NULL pg_collation -10 NULL 0 NULL pg_am -11 NULL 0 NULL pg_operator -12 NULL 0 NULL pg_views -13 NULL 0 NULL pg_attribute -14 NULL 0 NULL pg_database -15 NULL 0 NULL pg_description \ No newline at end of file +t rw_tables -1 facilisis enim +t rw_tables 0 Lorem ipsum +t rw_tables 1 Praesent elementum +t rw_tables 3 Nullam ultricies + +statement ok +comment on table public.t is NULL; + +statement ok +comment on column t._row_id is NULL; + +statement ok +comment on column t.c is ''; + +statement ok +comment on column public.t.b is 'Vivamus fermentum'; + +query TIIIT +SELECT + c.relname, + ( + SELECT relname FROM pg_catalog.pg_class WHERE oid = d.classoid + ) AS classoid, + d.objsubid, + d.description +FROM + pg_catalog.pg_description d + JOIN pg_catalog.pg_class c + ON d.objoid = c.oid +ORDER BY d.objsubid; +---- +t rw_tables 1 Praesent elementum +t rw_tables 2 Vivamus fermentum + +statement ok +drop table t; diff --git a/e2e_test/ddl/show.slt b/e2e_test/ddl/show.slt index 5ae7575668645..9586731207fa7 100644 --- a/e2e_test/ddl/show.slt +++ b/e2e_test/ddl/show.slt @@ -7,15 +7,28 @@ create materialized view mv3 as select sum(v1) as sum_v1 from t3; statement ok create view v3 as select sum(v2) as sum_v2 from t3; -query TTT +statement ok +comment on table t3 is 'volutpat vitae'; + +statement ok +comment on column t3.v1 is 'turpis vehicula'; + +statement ok +comment on column t3.v2 is 'Lorem ipsum dolor sit amet'; + +statement ok +comment on column public.t3._row_id is 'consectetur adipiscing elit'; + +query TTTT describe t3; ---- -v1 integer false -v2 integer false -v3 integer false -_row_id serial true -primary key _row_id NULL -distribution key _row_id NULL +v1 integer false turpis vehicula +v2 integer false Lorem ipsum dolor sit amet +v3 integer false NULL +_row_id serial true consectetur adipiscing elit +primary key _row_id NULL NULL +distribution key _row_id NULL NULL +table description t3 NULL volutpat vitae query TTT show columns from t3; @@ -33,16 +46,29 @@ show indexes from t3; ---- idx1 t3 v1 ASC, v2 ASC v3 v1 -query TTT +statement ok +comment on table public.t3 is 'consectetur turpis'; + +statement ok +comment on column t3.v1 is 'Nemo enim ipsam'; + +statement ok +comment on column t3.v2 is ''; + +statement ok +comment on column t3._row_id is NULL; + +query TTTT describe t3; ---- -v1 integer false -v2 integer false -v3 integer false -_row_id serial true -primary key _row_id NULL -distribution key _row_id NULL -idx1 index(v1 ASC, v2 ASC) include(v3) distributed by(v1) NULL +v1 integer false Nemo enim ipsam +v2 integer false NULL +v3 integer false NULL +_row_id serial true NULL +primary key _row_id NULL NULL +distribution key _row_id NULL NULL +idx1 index(v1 ASC, v2 ASC) include(v3) distributed by(v1) NULL NULL +table description t3 NULL consectetur turpis query TT show create index idx1; diff --git a/e2e_test/extended_mode/basic.slt b/e2e_test/extended_mode/basic.slt index 7869494979e47..054dae2f6a234 100644 --- a/e2e_test/extended_mode/basic.slt +++ b/e2e_test/extended_mode/basic.slt @@ -39,15 +39,16 @@ values(round(42.4382)); statement ok create table t3 (v1 int, v2 int, v3 int); -query TTT +query TTTT describe t3; ---- -v1 integer false -v2 integer false -v3 integer false -_row_id serial true -primary key _row_id NULL -distribution key _row_id NULL +v1 integer false NULL +v2 integer false NULL +v3 integer false NULL +_row_id serial true NULL +primary key _row_id NULL NULL +distribution key _row_id NULL NULL +table description t3 NULL NULL query TTT show columns from t3; diff --git a/proto/catalog.proto b/proto/catalog.proto index 2d4d51b5692b3..c966b7bbe5eb0 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -280,6 +280,9 @@ message Table { CreateType create_type = 32; + // This field is used to store the description set by the `comment on` clause. + optional string description = 33; + // Per-table catalog version, used by schema change. `None` for internal tables and tests. // Not to be confused with the global catalog version for notification service. TableVersion version = 100; @@ -317,3 +320,11 @@ message Database { string name = 2; uint32 owner = 3; } + +message Comment { + uint32 table_id = 1; + uint32 schema_id = 2; + uint32 database_id = 3; + optional uint32 column_index = 4; + optional string description = 5; +} diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 1efc933a7d033..b3c7f17509f8c 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -318,6 +318,15 @@ message WaitRequest {} message WaitResponse {} +message CommentOnRequest { + catalog.Comment comment = 1; +} + +message CommentOnResponse { + common.Status status = 1; + uint64 version = 2; +} + service DdlService { rpc CreateDatabase(CreateDatabaseRequest) returns (CreateDatabaseResponse); rpc DropDatabase(DropDatabaseRequest) returns (DropDatabaseResponse); @@ -348,4 +357,5 @@ service DdlService { rpc DropConnection(DropConnectionRequest) returns (DropConnectionResponse); rpc GetTables(GetTablesRequest) returns (GetTablesResponse); rpc Wait(WaitRequest) returns (WaitResponse); + rpc CommentOn(CommentOnRequest) returns (CommentOnResponse); } diff --git a/proto/plan_common.proto b/proto/plan_common.proto index d4c7a2e04f138..afea3aff14bc7 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -37,6 +37,9 @@ message ColumnDesc { GeneratedColumnDesc generated_column = 6; DefaultColumnDesc default_column = 7; } + + // This field is used to store the description set by the `comment on` clause. + optional string description = 8; } message ColumnCatalog { diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index cde16ef8d7652..b70084fbf864a 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -101,6 +101,7 @@ pub struct ColumnDesc { pub field_descs: Vec, pub type_name: String, pub generated_or_default_column: Option, + pub description: Option, } impl ColumnDesc { @@ -112,6 +113,7 @@ impl ColumnDesc { field_descs: vec![], type_name: String::new(), generated_or_default_column: None, + description: None, } } @@ -129,6 +131,7 @@ impl ColumnDesc { .collect_vec(), type_name: self.type_name.clone(), generated_or_default_column: self.generated_or_default_column.clone(), + description: self.description.clone(), } } @@ -172,6 +175,7 @@ impl ColumnDesc { field_descs: vec![], type_name: "".to_string(), generated_or_default_column: None, + description: None, } } @@ -192,6 +196,7 @@ impl ColumnDesc { field_descs: fields, type_name: type_name.to_string(), generated_or_default_column: None, + description: None, } } @@ -206,6 +211,7 @@ impl ColumnDesc { .map(Self::from_field_without_column_id) .collect_vec(), type_name: field.type_name.clone(), + description: None, generated_or_default_column: None, } } @@ -243,6 +249,7 @@ impl From for ColumnDesc { type_name: prost.type_name, field_descs, generated_or_default_column: prost.generated_or_default_column, + description: prost.description.clone(), } } } @@ -262,6 +269,7 @@ impl From<&ColumnDesc> for PbColumnDesc { field_descs: c.field_descs.iter().map(ColumnDesc::to_protobuf).collect(), type_name: c.type_name.clone(), generated_or_default_column: c.generated_or_default_column.clone(), + description: c.description.clone(), } } } diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index e83b4aa638907..1a46cdcf4057a 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -110,6 +110,7 @@ pub fn row_id_column_desc() -> ColumnDesc { field_descs: vec![], type_name: "".to_string(), generated_or_default_column: None, + description: None, } } @@ -131,6 +132,7 @@ pub fn offset_column_desc() -> ColumnDesc { field_descs: vec![], type_name: "".to_string(), generated_or_default_column: None, + description: None, } } diff --git a/src/common/src/catalog/test_utils.rs b/src/common/src/catalog/test_utils.rs index 2cce9b79b346e..6b524edb92430 100644 --- a/src/common/src/catalog/test_utils.rs +++ b/src/common/src/catalog/test_utils.rs @@ -57,6 +57,7 @@ impl ColumnDescTestExt for ColumnDesc { type_name: type_name.to_string(), field_descs: fields, generated_or_default_column: None, + description: None, } } } diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index 6d7e93365c275..078dfa05aa448 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -159,6 +159,7 @@ async fn test_table_materialize() -> StreamResult<()> { field_descs: vec![], type_name: "".to_string(), generated_or_default_column: None, + description: None, }) .collect_vec(); let (barrier_tx, barrier_rx) = unbounded_channel(); diff --git a/src/connector/src/parser/avro/util.rs b/src/connector/src/parser/avro/util.rs index 917b23f1af6d0..e1b63962bf23c 100644 --- a/src/connector/src/parser/avro/util.rs +++ b/src/connector/src/parser/avro/util.rs @@ -57,6 +57,7 @@ fn avro_field_to_column_desc( field_descs: vec_column, type_name: schema_name.to_string(), generated_or_default_column: None, + description: None, }) } _ => { diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index 71d122470902e..0ca0235254000 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -199,6 +199,7 @@ impl ProtobufParserConfig { field_descs, type_name: m.full_name().to_string(), generated_or_default_column: None, + description: None, }) } else { *index += 1; diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index bdd72a090f9e8..8624550274299 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -123,6 +123,7 @@ impl From<&SourceColumnDesc> for ColumnDesc { field_descs: s.fields.clone(), type_name: "".to_string(), generated_or_default_column: None, + description: None, } } } diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 6da590c2d315d..221056f3a4822 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -576,6 +576,7 @@ pub fn bind_struct_field(column_def: &StructField) -> Result { field_descs: vec![], type_name: "".to_string(), generated_or_default_column: None, + description: None, }) }) .collect::>>()? @@ -589,6 +590,7 @@ pub fn bind_struct_field(column_def: &StructField) -> Result { field_descs, type_name: "".to_string(), generated_or_default_column: None, + description: None, }) } diff --git a/src/frontend/src/catalog/catalog_service.rs b/src/frontend/src/catalog/catalog_service.rs index 8eb6b9e3e4485..be85293acd27f 100644 --- a/src/frontend/src/catalog/catalog_service.rs +++ b/src/frontend/src/catalog/catalog_service.rs @@ -21,7 +21,8 @@ use risingwave_common::error::ErrorCode::InternalError; use risingwave_common::error::{Result, RwError}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::catalog::{ - PbCreateType, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, + PbComment, PbCreateType, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, + PbView, }; use risingwave_pb::ddl_service::alter_relation_name_request::Relation; use risingwave_pb::ddl_service::create_connection_request; @@ -111,6 +112,8 @@ pub trait CatalogWriter: Send + Sync { connection: create_connection_request::Payload, ) -> Result<()>; + async fn comment_on(&self, comment: PbComment) -> Result<()>; + async fn drop_table( &self, source_id: Option, @@ -282,6 +285,11 @@ impl CatalogWriter for CatalogWriterImpl { self.wait_version(version).await } + async fn comment_on(&self, comment: PbComment) -> Result<()> { + let version = self.meta_client.comment_on(comment).await?; + self.wait_version(version).await + } + async fn drop_table( &self, source_id: Option, diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index 4cd271f0495b9..d64db79b8ced1 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -57,6 +57,9 @@ pub struct SystemTableCatalog { // owner of table, should always be default super user, keep it for compatibility. pub owner: u32, + + /// description of table, set by `comment on`. + pub description: Option, } impl SystemTableCatalog { @@ -165,6 +168,7 @@ impl From<&BuiltinTable> for SystemTableCatalog { .collect(), pk: val.pk.to_vec(), owner: DEFAULT_SUPER_USER_ID, + description: None, } } } @@ -412,6 +416,7 @@ prepare_sys_catalog! { { BuiltinCatalog::Table(&RW_HUMMOCK_BRANCHED_OBJECTS), read_hummock_branched_objects await }, { BuiltinCatalog::Table(&RW_HUMMOCK_COMPACTION_GROUP_CONFIGS), read_hummock_compaction_group_configs await }, { BuiltinCatalog::Table(&RW_HUMMOCK_META_CONFIGS), read_hummock_meta_configs await}, + { BuiltinCatalog::Table(&RW_DESCRIPTION), read_rw_description }, } #[cfg(test)] diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs index 8bc91f4a10bb9..593522ceda705 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs @@ -21,22 +21,25 @@ use crate::catalog::system_catalog::BuiltinView; /// The catalog `pg_description` stores description. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-description.html`] -pub static PG_DESCRIPTION: LazyLock = LazyLock::new(|| { - BuiltinView { - name: "pg_description", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "objoid"), - (DataType::Int32, "classoid"), - (DataType::Int32, "objsubid"), - (DataType::Varchar, "description"), - ], - sql: "SELECT id AS objoid, NULL::integer AS classoid, 0 AS objsubid, NULL AS description FROM rw_catalog.rw_tables \ - UNION ALL SELECT id AS objoid, NULL::integer AS classoid, 0 AS objsubid, NULL AS description FROM rw_catalog.rw_materialized_views \ - UNION ALL SELECT id AS objoid, NULL::integer AS classoid, 0 AS objsubid, NULL AS description FROM rw_catalog.rw_views \ - UNION ALL SELECT id AS objoid, NULL::integer AS classoid, 0 AS objsubid, NULL AS description FROM rw_catalog.rw_indexes \ - UNION ALL SELECT id AS objoid, NULL::integer AS classoid, 0 AS objsubid, NULL AS description FROM rw_catalog.rw_sources \ - UNION ALL SELECT id AS objoid, NULL::integer AS classoid, 0 AS objsubid, NULL AS description FROM rw_catalog.rw_system_tables\ - ".into(), - } +pub static PG_DESCRIPTION: LazyLock = LazyLock::new(|| BuiltinView { + name: "pg_description", + schema: PG_CATALOG_SCHEMA_NAME, + columns: &[ + (DataType::Int32, "objoid"), + (DataType::Int32, "classoid"), + (DataType::Int32, "objsubid"), + (DataType::Varchar, "description"), + ], + // objsubid = 0 => _row_id (hidden column) + // objsubid is NULL => table self + sql: "SELECT objoid, \ + classoid, \ + CASE \ + WHEN objsubid = 0 THEN -1 \ + WHEN objsubid IS NULL THEN 0 \ + ELSE objsubid \ + END AS objsubid, \ + description FROM rw_catalog.rw_description \ + WHERE description IS NOT NULL;" + .into(), }); diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs index 9f89c9eed5e81..517f40b460e94 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs @@ -17,6 +17,7 @@ mod rw_columns; mod rw_connections; mod rw_databases; mod rw_ddl_progress; +mod rw_description; mod rw_fragments; mod rw_functions; mod rw_hummock_branched_objects; @@ -50,6 +51,7 @@ pub use rw_columns::*; pub use rw_connections::*; pub use rw_databases::*; pub use rw_ddl_progress::*; +pub use rw_description::*; pub use rw_fragments::*; pub use rw_functions::*; pub use rw_hummock_branched_objects::*; diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs new file mode 100644 index 0000000000000..370dec33a2a2a --- /dev/null +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs @@ -0,0 +1,84 @@ +// 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::iter; + +use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; + +use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; + +pub const RW_DESCRIPTION: BuiltinTable = BuiltinTable { + name: "rw_description", + schema: RW_CATALOG_SCHEMA_NAME, + columns: &[ + // table_id, view_id, function_id, etc. + (DataType::Int32, "objoid"), + // rw_tables, rw_views, rw_functions, etc. + (DataType::Int32, "classoid"), + // If `objoid` is `table_id`, then non-null `objsubid` is column number. + (DataType::Int32, "objsubid"), + (DataType::Varchar, "description"), + ], + pk: &[0, 1, 2], +}; + +impl SysCatalogReaderImpl { + pub fn read_rw_description(&self) -> Result> { + let build_row = + |table_id, catalog_id, index: Option, description: Option>| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(table_id)), + Some(ScalarImpl::Int32(catalog_id)), + index.map(ScalarImpl::Int32), + description.map(ScalarImpl::Utf8), + ]) + }; + + let reader = self.catalog_reader.read_guard(); + let rw_catalog = + reader.get_schema_by_name(&self.auth_context.database, RW_CATALOG_SCHEMA_NAME)?; + let schemas = reader + .iter_schemas(&self.auth_context.database)? + .filter(|schema| schema.id() != rw_catalog.id()); + + let rw_tables_id: i32 = rw_catalog + .get_system_table_by_name("rw_tables") + .map(|st| st.id.table_id) + .unwrap_or_default() as _; + + Ok(schemas + .flat_map(|schema| { + schema.iter_table().flat_map(|table| { + iter::once(build_row( + table.id.table_id as _, + rw_tables_id, + None, + table.description.as_deref().map(Into::into), + )) + .chain(table.columns.iter().map(|col| { + build_row( + table.id.table_id as _, + rw_tables_id, + Some(col.column_id().get_id() as _), + col.column_desc.description.as_deref().map(Into::into), + ) + })) + }) + }) + .collect()) + } +} diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index b0f9088132f59..750a06da7d231 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -152,6 +152,9 @@ pub struct TableCatalog { /// Indicate whether to create table in background or foreground. pub create_type: CreateType, + + /// description of table, set by `comment on`. + pub description: Option, } // How the stream job was created will determine @@ -438,6 +441,7 @@ impl TableCatalog { cleaned_by_watermark: self.cleaned_by_watermark, stream_job_status: PbStreamJobStatus::Creating.into(), create_type: self.create_type.to_prost().into(), + description: self.description.clone(), } } @@ -551,6 +555,7 @@ impl From for TableCatalog { initialized_at_epoch: tb.initialized_at_epoch.map(Epoch::from), cleaned_by_watermark: matches!(tb.cleaned_by_watermark, true), create_type: CreateType::from_prost(create_type), + description: tb.description, } } } @@ -643,6 +648,7 @@ mod tests { cleaned_by_watermark: false, stream_job_status: PbStreamJobStatus::Creating.into(), create_type: PbCreateType::Foreground.into(), + description: Some("description".to_string()), } .into(); @@ -668,6 +674,7 @@ mod tests { ColumnDesc::new_atomic(DataType::Varchar, "zipcode", 3), ], type_name: ".test.Country".to_string(), + description: None, generated_or_default_column: None, }, is_hidden: false @@ -698,6 +705,7 @@ mod tests { initialized_at_epoch: None, cleaned_by_watermark: false, create_type: CreateType::Foreground, + description: Some("description".to_string()) } ); assert_eq!(table, TableCatalog::from(table.to_prost(0, 0))); diff --git a/src/frontend/src/handler/comment.rs b/src/frontend/src/handler/comment.rs new file mode 100644 index 0000000000000..b0ff42a790346 --- /dev/null +++ b/src/frontend/src/handler/comment.rs @@ -0,0 +1,87 @@ +// 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 pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::error::{ErrorCode, Result}; +use risingwave_pb::catalog::PbComment; +use risingwave_sqlparser::ast::{CommentObject, ObjectName}; + +use super::{HandlerArgs, RwPgResponse}; +use crate::Binder; + +pub async fn handle_comment( + handler_args: HandlerArgs, + object_type: CommentObject, + object_name: ObjectName, + comment: Option, +) -> Result { + let session = handler_args.session; + let comment = comment.filter(|s| !s.is_empty()); + + let comment = { + let mut binder = Binder::new_for_ddl(&session); + // only `Column` and `Table` object are now supported + match object_type { + CommentObject::Column => { + let [tab @ .., col] = object_name.0.as_slice() else { + return Err(ErrorCode::BindError(format!( + "Invalid column: {}", + object_name.real_value() + )) + .into()); + }; + + let (schema, table) = Binder::resolve_schema_qualified_name( + session.database(), + ObjectName(tab.to_vec()), + )?; + + let (database_id, schema_id) = + session.get_database_and_schema_id_for_create(schema)?; + let table = binder.bind_table(None, &table, None)?; + binder.bind_columns_to_context(col.real_value(), table.table_catalog.columns)?; + + let column = binder.bind_column(object_name.0.as_slice())?; + + PbComment { + table_id: table.table_id.into(), + schema_id, + database_id, + column_index: column.as_input_ref().map(|input_ref| input_ref.index as _), + description: comment, + } + } + CommentObject::Table => { + let (schema, table) = + Binder::resolve_schema_qualified_name(session.database(), object_name)?; + let (database_id, schema_id) = + session.get_database_and_schema_id_for_create(schema)?; + let table = binder.bind_table(None, &table, None)?; + + PbComment { + table_id: table.table_id.into(), + schema_id, + database_id, + column_index: None, + description: comment, + } + } + } + }; + + let catalog_writer = session.catalog_writer()?; + catalog_writer.comment_on(comment).await?; + + Ok(PgResponse::empty_result(StatementType::COMMENT)) +} diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 6f7de61285cb0..0c2398a608eb8 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -630,6 +630,7 @@ pub(crate) async fn try_bind_columns_from_source( field_descs: vec![], type_name: "".to_string(), generated_or_default_column: None, + description: None, }, is_hidden: false, }, @@ -641,6 +642,7 @@ pub(crate) async fn try_bind_columns_from_source( field_descs: vec![], type_name: "".to_string(), generated_or_default_column: None, + description: None, }, is_hidden: false, }, @@ -775,6 +777,7 @@ fn check_and_add_timestamp_column( field_descs: vec![], type_name: "".to_string(), generated_or_default_column: None, + description: None, }, is_hidden: true, @@ -792,6 +795,7 @@ fn add_upsert_default_key_column(columns: &mut Vec) { field_descs: vec![], type_name: "".to_string(), generated_or_default_column: None, + description: None, }, is_hidden: true, }; diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 19d9a2f25c4b8..bb02797c21395 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -188,6 +188,7 @@ pub fn bind_sql_columns(column_defs: &[ColumnDef]) -> Result> field_descs, type_name: "".to_string(), generated_or_default_column: None, + description: None, }, is_hidden: false, }); diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index 4100b9a20be02..cef7af9dbd324 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -12,20 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; +use std::fmt::Display; use itertools::Itertools; use pgwire::pg_field_descriptor::PgFieldDescriptor; use pgwire::pg_response::{PgResponse, StatementType}; use pgwire::types::Row; -use risingwave_common::catalog::{ColumnCatalog, ColumnDesc}; use risingwave_common::error::Result; use risingwave_common::types::DataType; use risingwave_sqlparser::ast::{display_comma_separated, ObjectName}; use super::RwPgResponse; use crate::binder::{Binder, Relation}; -use crate::catalog::{CatalogError, IndexCatalog}; +use crate::catalog::CatalogError; use crate::handler::util::col_descs_to_rows; use crate::handler::HandlerArgs; @@ -34,12 +33,9 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res let mut binder = Binder::new_for_system(&session); let relation = binder.bind_relation_by_name(table_name.clone(), None, false)?; // For Source, it doesn't have table catalog so use get source to get column descs. - let (columns, pk_columns, dist_columns, indices): ( - Vec, - Vec, - Vec, - Vec>, - ) = match relation { + + // Vec, Vec, Vec, Vec>, String, Option + let (columns, pk_columns, dist_columns, indices, relname, description) = match relation { Relation::Source(s) => { let pk_column_catalogs = s .catalog @@ -55,7 +51,14 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res .unwrap() }) .collect_vec(); - (s.catalog.columns, pk_column_catalogs, vec![], vec![]) + ( + s.catalog.columns, + pk_column_catalogs, + vec![], + vec![], + s.catalog.name, + None, // Description + ) } Relation::BaseTable(t) => { let pk_column_catalogs = t @@ -75,6 +78,8 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res pk_column_catalogs, dist_columns, t.table_indexes, + t.table_catalog.name, + t.table_catalog.description, ) } Relation::SystemTable(t) => { @@ -89,6 +94,8 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res pk_column_catalogs, vec![], vec![], + t.sys_table_catalog.name.clone(), + None, // Description ) } _ => { @@ -99,18 +106,23 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res // Convert all column descs to rows let mut rows = col_descs_to_rows(columns); + fn concat(display_elems: impl IntoIterator) -> String + where + T: Display, + { + format!( + "{}", + display_comma_separated(&display_elems.into_iter().collect::>()) + ) + } + // Convert primary key to rows if !pk_columns.is_empty() { rows.push(Row::new(vec![ Some("primary key".into()), - Some( - format!( - "{}", - display_comma_separated(&pk_columns.into_iter().map(|x| x.name).collect_vec()), - ) - .into(), - ), - None, + Some(concat(pk_columns.iter().map(|x| &x.name)).into()), + None, // Is Hidden + None, // Description ])); } @@ -118,14 +130,9 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res if !dist_columns.is_empty() { rows.push(Row::new(vec![ Some("distribution key".into()), - Some( - display_comma_separated( - &dist_columns.into_iter().map(|col| col.name).collect_vec(), - ) - .to_string() - .into(), - ), - None, + Some(concat(dist_columns.iter().map(|x| &x.name)).into()), + None, // Is Hidden + None, // Description ])); } @@ -155,10 +162,22 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res .into(), ) }, + // Is Hidden + None, + // Description + // TODO: index description None, ]) })); + rows.push(Row::new(vec![ + Some("table description".into()), + Some(relname.into()), + None, // Is Hidden + description.map(Into::into), // Description + ])); + + // TODO: table name and description as title of response // TODO: recover the original user statement Ok(PgResponse::builder(StatementType::DESCRIBE) .values( @@ -179,6 +198,11 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res DataType::Varchar.to_oid(), DataType::Varchar.type_len(), ), + PgFieldDescriptor::new( + "Description".to_owned(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), ], ) .into()) @@ -233,6 +257,7 @@ mod tests { "primary key".into() => "v3".into(), "distribution key".into() => "v3".into(), "idx1".into() => "index(v1 DESC, v2 ASC, v3 ASC) include(v4) distributed by(v1)".into(), + "table description".into() => "t".into(), }; assert_eq!(columns, expected_columns); diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 174ed23e03ec5..8275551fbc4a6 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -39,6 +39,7 @@ mod alter_system; mod alter_table_column; pub mod alter_user; pub mod cancel_job; +mod comment; pub mod create_connection; mod create_database; pub mod create_function; @@ -525,6 +526,11 @@ pub async fn handle( session, } => transaction::handle_set(handler_args, modes, snapshot, session).await, Statement::CancelJobs(jobs) => handle_cancel(handler_args, jobs).await, + Statement::Comment { + object_type, + object_name, + comment, + } => comment::handle_comment(handler_args, object_type, object_name, comment).await, _ => Err( ErrorCode::NotImplemented(format!("Unhandled statement: {}", stmt), None.into()).into(), ), diff --git a/src/frontend/src/handler/util.rs b/src/frontend/src/handler/util.rs index 66494be928d42..7c1eca3fa9bbf 100644 --- a/src/frontend/src/handler/util.rs +++ b/src/frontend/src/handler/util.rs @@ -187,6 +187,7 @@ pub fn col_descs_to_rows(columns: Vec) -> Vec { Some(c.name.into()), Some(type_name.into()), Some(col.is_hidden.to_string().into()), + c.description.map(Into::into), ]) }) .collect_vec() diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 9c87f1a34abbd..d5435e9beb397 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -250,6 +250,7 @@ impl StreamMaterialize { initialized_at_epoch: None, cleaned_by_watermark: false, create_type: CreateType::Foreground, // Will be updated in the handler itself. + description: None, }) } diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index f167d73c53a46..f05a8be162554 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -180,6 +180,7 @@ impl TableCatalogBuilder { // NOTE(kwannoel): This may not match the create type of the materialized table. // It should be ignored for internal tables. create_type: CreateType::Foreground, + description: None, } } diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 67eac0df34d05..0385bd52690ef 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -1148,6 +1148,11 @@ fn infer(bound: Option, stmt: Statement) -> Result Ok(vec![PgFieldDescriptor::new( "QUERY PLAN".to_owned(), diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index cf915ae35713d..6cca805f3caae 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -32,7 +32,7 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::backup_service::MetaSnapshotMetadata; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ - PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, Table, + PbComment, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, Table, }; use risingwave_pb::ddl_service::{create_connection_request, DdlProgress}; use risingwave_pb::hummock::write_limits::WriteLimit; @@ -318,6 +318,10 @@ impl CatalogWriter for MockCatalogWriter { unreachable!() } + async fn comment_on(&self, _comment: PbComment) -> Result<()> { + unreachable!() + } + async fn drop_table( &self, source_id: Option, diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index fac8f89e17b11..6f08ebfb18d17 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -25,7 +25,7 @@ use risingwave_pb::catalog::connection::private_link_service::{ use risingwave_pb::catalog::connection::PbPrivateLinkService; use risingwave_pb::catalog::source::OptionalAssociatedTableId; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; -use risingwave_pb::catalog::{connection, Connection, CreateType, PbSource, PbTable}; +use risingwave_pb::catalog::{connection, Comment, Connection, CreateType, PbSource, PbTable}; use risingwave_pb::ddl_service::ddl_service_server::DdlService; use risingwave_pb::ddl_service::drop_table_request::PbSourceId; use risingwave_pb::ddl_service::*; @@ -717,6 +717,30 @@ impl DdlService for DdlServiceImpl { })) } + async fn comment_on( + &self, + request: Request, + ) -> Result, Status> { + let req = request.into_inner(); + let comment = req.get_comment()?.clone(); + + let version = self + .ddl_controller + .run_command(DdlCommand::CommentOn(Comment { + table_id: comment.table_id, + schema_id: comment.schema_id, + database_id: comment.database_id, + column_index: comment.column_index, + description: comment.description, + })) + .await?; + + Ok(Response::new(CommentOnResponse { + status: None, + version, + })) + } + #[cfg_attr(coverage, coverage(off))] async fn get_tables( &self, diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index d9193acd5591f..c69c615165d11 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -131,6 +131,7 @@ impl From> for PbTable { .0 .optional_associated_source_id .map(PbOptionalAssociatedSourceId::AssociatedSourceId), + description: None, } } } diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 15e74e4c2ac9e..d2007dcab45d6 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -34,8 +34,8 @@ use risingwave_common::catalog::{ use risingwave_common::{bail, ensure}; use risingwave_pb::catalog::table::{OptionalAssociatedSourceId, TableType}; use risingwave_pb::catalog::{ - Connection, CreateType, Database, Function, Index, PbStreamJobStatus, Schema, Sink, Source, - StreamJobStatus, Table, View, + Comment, Connection, CreateType, Database, Function, Index, PbStreamJobStatus, Schema, Sink, + Source, StreamJobStatus, Table, View, }; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::user::grant_privilege::{ActionWithGrantOption, Object}; @@ -2365,6 +2365,46 @@ impl CatalogManager { Ok(()) } + pub async fn comment_on(&self, comment: Comment) -> MetaResult { + let core = &mut *self.core.lock().await; + let database_core = &mut core.database; + + database_core.ensure_database_id(comment.database_id)?; + database_core.ensure_schema_id(comment.schema_id)?; + database_core.ensure_table_id(comment.table_id)?; + + let mut tables = BTreeMapTransaction::new(&mut database_core.tables); + + // unwrap is safe because the table id was ensured before + let mut table = tables.get_mut(comment.table_id).unwrap(); + if let Some(col_idx) = comment.column_index { + let column = table + .columns + .get_mut(col_idx as usize) + .ok_or_else(|| MetaError::catalog_id_not_found("column", col_idx))?; + let column_desc = column.column_desc.as_mut().ok_or_else(|| { + anyhow!( + "column desc at index {} for table id {} not found", + col_idx, + comment.table_id + ) + })?; + column_desc.description = comment.description; + } else { + table.description = comment.description; + } + + let new_table = table.clone(); + + commit_meta!(self, tables)?; + + let version = self + .notify_frontend_relation_info(Operation::Update, RelationInfo::Table(new_table)) + .await; + + Ok(version) + } + pub async fn list_connections(&self) -> Vec { self.core.lock().await.database.list_connections() } diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 5f40d9a561f4e..8f6e7c0be6915 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -24,7 +24,7 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::epoch::Epoch; use risingwave_pb::catalog::connection::private_link_service::PbPrivateLinkProvider; use risingwave_pb::catalog::{ - connection, Connection, CreateType, Database, Function, Schema, Source, Table, View, + connection, Comment, Connection, CreateType, Database, Function, Schema, Source, Table, View, }; use risingwave_pb::ddl_service::alter_relation_name_request::Relation; use risingwave_pb::ddl_service::DdlProgress; @@ -103,6 +103,7 @@ pub enum DdlCommand { AlterSourceColumn(Source), CreateConnection(Connection), DropConnection(ConnectionId), + CommentOn(Comment), } #[derive(Clone)] @@ -260,6 +261,7 @@ impl DdlController { ctrl.drop_connection(connection_id).await } DdlCommand::AlterSourceColumn(source) => ctrl.alter_source_column(source).await, + DdlCommand::CommentOn(comment) => ctrl.comment_on(comment).await, } } .in_current_span(); @@ -1112,4 +1114,8 @@ impl DdlController { } Err(MetaError::cancelled("timeout".into())) } + + async fn comment_on(&self, comment: Comment) -> MetaResult { + self.catalog_manager.comment_on(comment).await + } } diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 95b746ea33e6c..b8603fbe46e62 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -40,7 +40,8 @@ use risingwave_hummock_sdk::{ use risingwave_pb::backup_service::backup_service_client::BackupServiceClient; use risingwave_pb::backup_service::*; use risingwave_pb::catalog::{ - Connection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, Table, + Connection, PbComment, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, + PbView, Table, }; use risingwave_pb::cloud_service::cloud_service_client::CloudServiceClient; use risingwave_pb::cloud_service::*; @@ -407,6 +408,14 @@ impl MetaClient { Ok((resp.table_id.into(), resp.version)) } + pub async fn comment_on(&self, comment: PbComment) -> Result { + let request = CommentOnRequest { + comment: Some(comment), + }; + let resp = self.inner.comment_on(request).await?; + Ok(resp.version) + } + pub async fn alter_relation_name( &self, relation: Relation, @@ -1724,6 +1733,7 @@ macro_rules! for_all_meta_rpc { ,{ ddl_client, create_connection, CreateConnectionRequest, CreateConnectionResponse } ,{ ddl_client, list_connections, ListConnectionsRequest, ListConnectionsResponse } ,{ ddl_client, drop_connection, DropConnectionRequest, DropConnectionResponse } + ,{ ddl_client, comment_on, CommentOnRequest, CommentOnResponse } ,{ ddl_client, get_tables, GetTablesRequest, GetTablesResponse } ,{ ddl_client, wait, WaitRequest, WaitResponse } ,{ hummock_client, unpin_version_before, UnpinVersionBeforeRequest, UnpinVersionBeforeResponse } diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 161bbc41ceb63..e6646c03282a0 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -197,6 +197,7 @@ pub mod test_utils { field_descs: vec![], type_name: "".to_string(), generated_or_default_column: None, + description: None, } .to_protobuf(), ), diff --git a/src/storage/src/filter_key_extractor.rs b/src/storage/src/filter_key_extractor.rs index b5a79a6f6b42f..6538042566537 100644 --- a/src/storage/src/filter_key_extractor.rs +++ b/src/storage/src/filter_key_extractor.rs @@ -551,6 +551,7 @@ mod tests { cleaned_by_watermark: false, stream_job_status: PbStreamJobStatus::Created.into(), create_type: PbCreateType::Foreground.into(), + description: None, } } diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index 346cf2fe6acf8..db64dc6334c04 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -154,6 +154,7 @@ async fn compaction_test( cleaned_by_watermark: false, stream_job_status: PbStreamJobStatus::Created.into(), create_type: PbCreateType::Foreground.into(), + description: None, }; let mut delete_range_table = delete_key_table.clone(); delete_range_table.id = 2; diff --git a/src/utils/pgwire/src/pg_response.rs b/src/utils/pgwire/src/pg_response.rs index eeec929732f50..802f651ce4298 100644 --- a/src/utils/pgwire/src/pg_response.rs +++ b/src/utils/pgwire/src/pg_response.rs @@ -53,6 +53,7 @@ pub enum StatementType { CREATE_INDEX, CREATE_FUNCTION, CREATE_CONNECTION, + COMMENT, DESCRIBE, GRANT_PRIVILEGE, DROP_TABLE, From 90fb4a3478ec90c1497732af3e75afea6d22ae17 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 26 Oct 2023 04:25:11 +0000 Subject: [PATCH 26/48] chore(deps): Bump comfy-table from 7.0.1 to 7.1.0 (#13049) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 19e79820fb5e9..3f7df00f7648d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1825,13 +1825,13 @@ dependencies = [ [[package]] name = "comfy-table" -version = "7.0.1" +version = "7.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ab77dbd8adecaf3f0db40581631b995f312a8a5ae3aa9993188bb8f23d83a5b" +checksum = "7c64043d6c7b7a4c58e39e7efccfdea7b93d885a795d0c054a69dbbf4dd52686" dependencies = [ - "crossterm 0.26.1", - "strum 0.24.1", - "strum_macros 0.24.3", + "crossterm 0.27.0", + "strum 0.25.0", + "strum_macros 0.25.2", "unicode-width", ] @@ -2140,17 +2140,14 @@ dependencies = [ [[package]] name = "crossterm" -version = "0.26.1" +version = "0.27.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a84cda67535339806297f1b331d6dd6320470d2a0fe65381e79ee9e156dd3d13" +checksum = "f476fe445d41c9e991fd07515a6f463074b782242ccf4a5b7b1d1012e70824df" dependencies = [ - "bitflags 1.3.2", + "bitflags 2.4.0", "crossterm_winapi", "libc", - "mio", "parking_lot 0.12.1", - "signal-hook", - "signal-hook-mio", "winapi", ] From 942e99dc74228659ff106589634ddc6196009191 Mon Sep 17 00:00:00 2001 From: Yufan Song <33971064+yufansong@users.noreply.github.com> Date: Wed, 25 Oct 2023 22:10:31 -0700 Subject: [PATCH 27/48] fix(nats-connector): change stream into optional string, add replace stream name logic (#13024) --- src/connector/src/common.rs | 13 ++++++++----- src/connector/src/source/nats/mod.rs | 3 +++ src/connector/src/source/nats/source/reader.rs | 6 +++++- 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/src/connector/src/common.rs b/src/connector/src/common.rs index 2af396f5c33b4..c9a55dfe15086 100644 --- a/src/connector/src/common.rs +++ b/src/connector/src/common.rs @@ -417,8 +417,6 @@ pub struct UpsertMessage<'a> { pub struct NatsCommon { #[serde(rename = "server_url")] pub server_url: String, - #[serde(rename = "stream")] - pub stream: String, #[serde(rename = "subject")] pub subject: String, #[serde(rename = "connect_mode")] @@ -505,14 +503,18 @@ impl NatsCommon { pub(crate) async fn build_consumer( &self, + stream: String, split_id: String, start_sequence: NatsOffset, ) -> anyhow::Result< async_nats::jetstream::consumer::Consumer, > { let context = self.build_context().await?; - let stream = self.build_or_get_stream(context.clone()).await?; - let subject_name = self.subject.replace(',', "-"); + let stream = self.build_or_get_stream(context.clone(), stream).await?; + let subject_name = self + .subject + .replace(',', "-") + .replace(['.', '>', '*', ' ', '\t'], "_"); let name = format!("risingwave-consumer-{}-{}", subject_name, split_id); let mut config = jetstream::consumer::pull::Config { ack_policy: jetstream::consumer::AckPolicy::None, @@ -545,10 +547,11 @@ impl NatsCommon { pub(crate) async fn build_or_get_stream( &self, jetstream: jetstream::Context, + stream: String, ) -> anyhow::Result { let subjects: Vec = self.subject.split(',').map(|s| s.to_string()).collect(); let mut config = jetstream::stream::Config { - name: self.stream.clone(), + name: stream, max_bytes: 1000000, subjects, ..Default::default() diff --git a/src/connector/src/source/nats/mod.rs b/src/connector/src/source/nats/mod.rs index 3e8cc57bc1da8..f209086693fd3 100644 --- a/src/connector/src/source/nats/mod.rs +++ b/src/connector/src/source/nats/mod.rs @@ -35,6 +35,9 @@ pub struct NatsProperties { #[serde(rename = "scan.startup.timestamp_millis")] pub start_time: Option, + + #[serde(rename = "stream")] + pub stream: String, } impl NatsProperties {} diff --git a/src/connector/src/source/nats/source/reader.rs b/src/connector/src/source/nats/source/reader.rs index 6e22748bcf468..7f9a5718f95d4 100644 --- a/src/connector/src/source/nats/source/reader.rs +++ b/src/connector/src/source/nats/source/reader.rs @@ -77,7 +77,11 @@ impl SplitReader for NatsSplitReader { let consumer = properties .common - .build_consumer(split_id.to_string(), start_position.clone()) + .build_consumer( + properties.stream.clone(), + split_id.to_string(), + start_position.clone(), + ) .await?; Ok(Self { consumer, From e2a36373fa1379667a70b60e83502627d9224c58 Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Thu, 26 Oct 2023 13:56:26 +0800 Subject: [PATCH 28/48] =?UTF-8?q?fix(DBeaver):=20Add=20the=20=E2=80=98=5Fx?= =?UTF-8?q?xx=E2=80=99=20in=20RW=E2=80=98s=20pg=5Fcatalog.pg=5Ftype=20to?= =?UTF-8?q?=20avoid=20some=20DBeaver=20errors."=20(#13052)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- e2e_test/batch/catalog/issue_10177.slt.part | 49 ++++++++++++------- e2e_test/batch/catalog/pg_type.slt.part | 49 ++++++++++++------- .../tests/testdata/output/pg_catalog.yaml | 10 ++-- .../system_catalog/pg_catalog/pg_type.rs | 4 +- .../system_catalog/rw_catalog/rw_types.rs | 18 +++++-- 5 files changed, 84 insertions(+), 46 deletions(-) diff --git a/e2e_test/batch/catalog/issue_10177.slt.part b/e2e_test/batch/catalog/issue_10177.slt.part index 771d304d7a058..8f8c40aa75c19 100644 --- a/e2e_test/batch/catalog/issue_10177.slt.part +++ b/e2e_test/batch/catalog/issue_10177.slt.part @@ -43,20 +43,35 @@ ORDER BY CASE WHEN typtype = 'd' AND elemtyptype = 'a' THEN 6 -- Domains over arrays last END; ---- -pg_catalog 1043 varchar b f NULL -pg_catalog 1082 date b f NULL -pg_catalog 1083 time b f NULL -pg_catalog 1114 timestamp b f NULL -pg_catalog 1184 timestamptz b f NULL -pg_catalog 1186 interval b f NULL -pg_catalog 1301 rw_int256 b f NULL -pg_catalog 16 bool b f NULL -pg_catalog 17 bytea b f NULL -pg_catalog 1700 numeric b f NULL -pg_catalog 20 int8 b f NULL -pg_catalog 21 int2 b f NULL -pg_catalog 23 int4 b f NULL -pg_catalog 25 text b f NULL -pg_catalog 3802 jsonb b f NULL -pg_catalog 700 float4 b f NULL -pg_catalog 701 float8 b f NULL +pg_catalog 1000 _bool b f NULL +pg_catalog 1001 _bytea b f NULL +pg_catalog 1005 _int2 b f NULL +pg_catalog 1007 _int4 b f NULL +pg_catalog 1015 _varchar b f NULL +pg_catalog 1016 _int8 b f NULL +pg_catalog 1021 _float4 b f NULL +pg_catalog 1022 _float8 b f NULL +pg_catalog 1043 varchar b f NULL +pg_catalog 1082 date b f NULL +pg_catalog 1083 time b f NULL +pg_catalog 1114 timestamp b f NULL +pg_catalog 1115 _timestamp b f NULL +pg_catalog 1182 _date b f NULL +pg_catalog 1183 _time b f NULL +pg_catalog 1184 timestamptz b f NULL +pg_catalog 1185 _timestamptz b f NULL +pg_catalog 1186 interval b f NULL +pg_catalog 1187 _interval b f NULL +pg_catalog 1231 _numeric b f NULL +pg_catalog 1301 rw_int256 b f NULL +pg_catalog 16 bool b f NULL +pg_catalog 17 bytea b f NULL +pg_catalog 1700 numeric b f NULL +pg_catalog 20 int8 b f NULL +pg_catalog 21 int2 b f NULL +pg_catalog 23 int4 b f NULL +pg_catalog 25 text b f NULL +pg_catalog 3802 jsonb b f NULL +pg_catalog 3807 _jsonb b f NULL +pg_catalog 700 float4 b f NULL +pg_catalog 701 float8 b f NULL \ No newline at end of file diff --git a/e2e_test/batch/catalog/pg_type.slt.part b/e2e_test/batch/catalog/pg_type.slt.part index e35d21291bc14..29c510a394e61 100644 --- a/e2e_test/batch/catalog/pg_type.slt.part +++ b/e2e_test/batch/catalog/pg_type.slt.part @@ -1,20 +1,35 @@ query ITITT SELECT oid, typname, typelem, typnotnull, typtype, typinput FROM pg_catalog.pg_type order by oid; ---- -16 bool 0 f b boolin -17 bytea 0 f b byteain -20 int8 0 f b int8in -21 int2 0 f b int2in -23 int4 0 f b int4in -25 text 0 f b textin -700 float4 0 f b float4in -701 float8 0 f b float8in -1043 varchar 0 f b varcharin -1082 date 0 f b date_in -1083 time 0 f b time_in -1114 timestamp 0 f b timestamp_in -1184 timestamptz 0 f b timestamptz_in -1186 interval 0 f b interval_in -1301 rw_int256 0 f b rw_int256_in -1700 numeric 0 f b numeric_in -3802 jsonb 0 f b jsonb_in +16 bool 0 f b boolin +17 bytea 0 f b byteain +20 int8 0 f b int8in +21 int2 0 f b int2in +23 int4 0 f b int4in +25 text 0 f b textin +700 float4 0 f b float4in +701 float8 0 f b float8in +1000 _bool 16 f b array_in +1001 _bytea 17 f b array_in +1005 _int2 21 f b array_in +1007 _int4 23 f b array_in +1015 _varchar 1043 f b array_in +1016 _int8 20 f b array_in +1021 _float4 700 f b array_in +1022 _float8 701 f b array_in +1043 varchar 0 f b varcharin +1082 date 0 f b date_in +1083 time 0 f b time_in +1114 timestamp 0 f b timestamp_in +1115 _timestamp 1114 f b array_in +1182 _date 1082 f b array_in +1183 _time 1083 f b array_in +1184 timestamptz 0 f b timestamptz_in +1185 _timestamptz 1184 f b array_in +1186 interval 0 f b interval_in +1187 _interval 1186 f b array_in +1231 _numeric 1700 f b array_in +1301 rw_int256 0 f b rw_int256_in +1700 numeric 0 f b numeric_in +3802 jsonb 0 f b jsonb_in +3807 _jsonb 3802 f b array_in 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 5c5a88fb472b6..5ae1827ad95a3 100644 --- a/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml +++ b/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml @@ -2,16 +2,16 @@ - sql: | select * from pg_catalog.pg_type logical_plan: |- - LogicalProject { exprs: [rw_types.id, rw_types.name, 0:Int32, 0:Int32, 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] } + 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] } └─LogicalShare { id: 3 } - └─LogicalProject { exprs: [rw_types.id, rw_types.name, 0:Int32, 0:Int32, 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] } + └─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] } + ├─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] } batch_plan: |- - BatchProject { exprs: [rw_types.id, rw_types.name, 0:Int32, 0:Int32, 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] } + 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 } - ├─BatchScan { table: rw_types, columns: [rw_types.id, rw_types.name, rw_types.input_oid], distribution: Single } + ├─BatchScan { table: rw_types, columns: [rw_types.id, rw_types.name, rw_types.input_oid, rw_types.typelem, rw_types.typarray], distribution: Single } └─BatchProject { exprs: [rw_schemas.id] } └─BatchFilter { predicate: (rw_schemas.name = 'pg_catalog':Varchar) } └─BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name], distribution: Single } diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs index af6a2968e3a8c..5b4993c2e8c9e 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs @@ -58,8 +58,8 @@ pub static PG_TYPE: LazyLock = LazyLock::new(|| BuiltinView { ], sql: "SELECT t.id AS oid, \ t.name AS typname, \ - 0 AS typelem, \ - 0 AS typarray, \ + t.typelem AS typelem, \ + t.typarray AS typarray, \ t.input_oid AS typinput, \ false AS typnotnull, \ 0 AS typbasetype, \ diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs index 02462c4cee303..9ce1d35784f54 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs @@ -27,16 +27,20 @@ macro_rules! impl_pg_type_data { ($( { $enum:ident | $oid:literal | $oid_array:literal | $name:ident | $input:ident | $len:literal } )*) => { &[ $( - ($oid, stringify!($name), stringify!($input)), + ($oid, stringify!($name), stringify!($input), 0, $oid_array), )* // Note: rw doesn't support `text` type, returning it is just a workaround to be compatible // with PostgreSQL. - (25, "text", "textin"), - (1301, "rw_int256", "rw_int256_in"), + (25, "text", "textin",0,1009), + (1301, "rw_int256", "rw_int256_in",0,0), + // Note: Here is only to avoid some components of psql from not being able to find relevant results, causing errors. We will not use it in the RW. + $( + ($oid_array, concat!("_", stringify!($name)), "array_in", $oid, 0), + )* ] } } -pub const RW_TYPE_DATA: &[(i32, &str, &str)] = for_all_base_types! { impl_pg_type_data }; +pub const RW_TYPE_DATA: &[(i32, &str, &str, i32, i32)] = for_all_base_types! { impl_pg_type_data }; /// `rw_types` stores all supported types in the database. pub static RW_TYPES: LazyLock = LazyLock::new(|| BuiltinTable { @@ -46,6 +50,8 @@ pub static RW_TYPES: LazyLock = LazyLock::new(|| BuiltinTable { (DataType::Int32, "id"), (DataType::Varchar, "name"), (DataType::Varchar, "input_oid"), + (DataType::Int32, "typelem"), + (DataType::Int32, "typarray"), ], pk: &[0], }); @@ -54,11 +60,13 @@ impl SysCatalogReaderImpl { pub fn read_rw_types(&self) -> Result> { Ok(RW_TYPE_DATA .iter() - .map(|(id, name, input)| { + .map(|(id, name, input, typelem, typarray)| { OwnedRow::new(vec![ Some(ScalarImpl::Int32(*id)), Some(ScalarImpl::Utf8(name.to_string().into())), Some(ScalarImpl::Utf8(input.to_string().into())), + Some(ScalarImpl::Int32(*typelem)), + Some(ScalarImpl::Int32(*typarray)), ]) }) .collect_vec()) From 71851d628602c4a7bab357781782198fefa511da Mon Sep 17 00:00:00 2001 From: Wallace Date: Thu, 26 Oct 2023 14:01:57 +0800 Subject: [PATCH 29/48] fix(compactor): collect metrics for fast compact runer (#13064) Signed-off-by: Little-Wallace --- .../hummock_test/src/compactor_tests.rs | 9 +- .../src/hummock/compactor/compactor_runner.rs | 4 +- .../compactor/fast_compactor_runner.rs | 83 +++++++++++++++++-- src/storage/src/hummock/utils.rs | 3 - 4 files changed, 79 insertions(+), 20 deletions(-) diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index 50d739c5d1eb9..34b23a9b79774 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -1454,13 +1454,8 @@ pub(crate) mod tests { .await .unwrap(); let ret = ret1.into_iter().map(|sst| sst.sst_info).collect_vec(); - let fast_ret = fast_compact_runner - .run() - .await - .unwrap() - .into_iter() - .map(|sst| sst.sst_info) - .collect_vec(); + let (ssts, _) = fast_compact_runner.run().await.unwrap(); + let fast_ret = ssts.into_iter().map(|sst| sst.sst_info).collect_vec(); println!("ssts: {} vs {}", fast_ret.len(), ret.len()); let mut fast_tables = Vec::with_capacity(fast_ret.len()); let mut normal_tables = Vec::with_capacity(ret.len()); diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index a21016014d247..1925acbce7534 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -472,8 +472,8 @@ pub async fn compact( task_progress_guard.progress.clone(), ); match runner.run().await { - Ok(ssts) => { - output_ssts.push((0, ssts, CompactionStatistics::default())); + Ok((ssts, statistics)) => { + output_ssts.push((0, ssts, statistics)); } Err(e) => { task_status = TaskStatus::ExecuteFailed; diff --git a/src/storage/src/hummock/compactor/fast_compactor_runner.rs b/src/storage/src/hummock/compactor/fast_compactor_runner.rs index 6dcfb0e2392cf..c3184fc3e5f76 100644 --- a/src/storage/src/hummock/compactor/fast_compactor_runner.rs +++ b/src/storage/src/hummock/compactor/fast_compactor_runner.rs @@ -24,12 +24,15 @@ use bytes::Bytes; use itertools::Itertools; use risingwave_hummock_sdk::key::FullKey; use risingwave_hummock_sdk::key_range::KeyRange; +use risingwave_hummock_sdk::table_stats::TableStats; use risingwave_hummock_sdk::{can_concat, HummockEpoch, LocalSstableInfo}; use risingwave_pb::hummock::{CompactTask, SstableInfo}; use crate::filter_key_extractor::FilterKeyExtractorImpl; use crate::hummock::compactor::task_progress::TaskProgress; -use crate::hummock::compactor::{Compactor, CompactorContext, RemoteBuilderFactory, TaskConfig}; +use crate::hummock::compactor::{ + CompactionStatistics, Compactor, CompactorContext, RemoteBuilderFactory, TaskConfig, +}; use crate::hummock::multi_builder::{CapacitySplitTableBuilder, TableBuilderFactory}; use crate::hummock::sstable_store::{BlockStream, SstableStoreRef}; use crate::hummock::value::HummockValue; @@ -280,7 +283,6 @@ pub struct CompactorRunner { >, compression_algorithm: CompressionAlgorithm, metrics: Arc, - task_progress: Arc, } impl CompactorRunner { @@ -343,17 +345,16 @@ impl CompactorRunner { )); Self { - executor: CompactTaskExecutor::new(sst_builder, task_config), + executor: CompactTaskExecutor::new(sst_builder, task_config, task_progress), left, right, task_id: task.task_id, metrics: context.compactor_metrics.clone(), compression_algorithm, - task_progress, } } - pub async fn run(mut self) -> HummockResult> { + pub async fn run(mut self) -> HummockResult<(Vec, CompactionStatistics)> { self.left.rewind().await?; self.right.rewind().await?; let mut skip_raw_block_count = 0; @@ -409,6 +410,7 @@ impl CompactorRunner { let largest_key = first.current_sstable().current_block_largest(); let block_len = block.len() as u64; + let block_key_count = meta.total_key_count; if self .executor @@ -419,6 +421,7 @@ impl CompactorRunner { skip_raw_block_size += block_len; skip_raw_block_count += 1; } + self.executor.may_report_process_key(block_key_count); self.executor.clear(); } if !first.current_sstable().is_valid() { @@ -462,6 +465,7 @@ impl CompactorRunner { sstable_iter.download_next_block().await?.unwrap(); let largest_key = sstable_iter.current_block_largest(); let block_len = block.len() as u64; + let block_key_count = block_meta.total_key_count; if self .executor .builder @@ -471,6 +475,7 @@ impl CompactorRunner { skip_raw_block_count += 1; skip_raw_block_size += block_len; } + self.executor.may_report_process_key(block_key_count); } rest_data.next_sstable().await?; } @@ -491,37 +496,61 @@ impl CompactorRunner { skip_raw_block_size * 100 / total_read_bytes, ); + let statistic = self.executor.take_statistics(); let outputs = self.executor.builder.finish().await?; let ssts = Compactor::report_progress( self.metrics.clone(), - Some(self.task_progress.clone()), + Some(self.executor.task_progress.clone()), outputs, false, ) .await?; let sst_infos = ssts.iter().map(|sst| sst.sst_info.clone()).collect_vec(); assert!(can_concat(&sst_infos)); - Ok(ssts) + Ok((ssts, statistic)) } } pub struct CompactTaskExecutor { last_key: FullKey>, + compaction_statistics: CompactionStatistics, + last_table_id: Option, + last_table_stats: TableStats, watermark_can_see_last_key: bool, builder: CapacitySplitTableBuilder, task_config: TaskConfig, + task_progress: Arc, last_key_is_delete: bool, + progress_key_num: u32, } impl CompactTaskExecutor { - pub fn new(builder: CapacitySplitTableBuilder, task_config: TaskConfig) -> Self { + pub fn new( + builder: CapacitySplitTableBuilder, + task_config: TaskConfig, + task_progress: Arc, + ) -> Self { Self { builder, task_config, last_key: FullKey::default(), watermark_can_see_last_key: false, last_key_is_delete: false, + compaction_statistics: CompactionStatistics::default(), + last_table_id: None, + last_table_stats: TableStats::default(), + progress_key_num: 0, + task_progress, + } + } + + fn take_statistics(&mut self) -> CompactionStatistics { + if let Some(last_table_id) = self.last_table_id.take() { + self.compaction_statistics + .delta_drop_stat + .insert(last_table_id, std::mem::take(&mut self.last_table_stats)); } + std::mem::take(&mut self.compaction_statistics) } fn clear(&mut self) { @@ -532,6 +561,17 @@ impl CompactTaskExecutor { self.last_key_is_delete = false; } + #[inline(always)] + fn may_report_process_key(&mut self, key_count: u32) { + const PROGRESS_KEY_INTERVAL: u32 = 100; + self.progress_key_num += key_count; + if self.progress_key_num > PROGRESS_KEY_INTERVAL { + self.task_progress + .inc_progress_key(self.progress_key_num as u64); + self.progress_key_num = 0; + } + } + pub async fn run( &mut self, iter: &mut BlockIterator, @@ -540,6 +580,9 @@ impl CompactTaskExecutor { while iter.is_valid() && iter.key().le(&target_key) { let is_new_user_key = !self.last_key.is_empty() && iter.key().user_key != self.last_key.user_key.as_ref(); + self.compaction_statistics.iter_total_key_counts += 1; + self.may_report_process_key(1); + let mut drop = false; let epoch = iter.key().epoch; let value = HummockValue::from_slice(iter.value()).unwrap(); @@ -562,7 +605,31 @@ impl CompactTaskExecutor { self.watermark_can_see_last_key = true; } + if self.last_table_id.map_or(true, |last_table_id| { + last_table_id != self.last_key.user_key.table_id.table_id + }) { + if let Some(last_table_id) = self.last_table_id.take() { + self.compaction_statistics + .delta_drop_stat + .insert(last_table_id, std::mem::take(&mut self.last_table_stats)); + } + self.last_table_id = Some(self.last_key.user_key.table_id.table_id); + } + if drop { + self.compaction_statistics.iter_drop_key_counts += 1; + + let should_count = match self.task_config.stats_target_table_ids.as_ref() { + Some(target_table_ids) => { + target_table_ids.contains(&self.last_key.user_key.table_id.table_id) + } + None => true, + }; + if should_count { + self.last_table_stats.total_key_count -= 1; + self.last_table_stats.total_key_size -= self.last_key.encoded_len() as i64; + self.last_table_stats.total_value_size -= value.encoded_len() as i64; + } iter.next(); continue; } diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index 7ccb3fbf04790..6404d80bb265f 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -604,9 +604,6 @@ mod tests { use crate::hummock::utils::MemoryLimiter; - // This is a clippy bug, see https://github.com/rust-lang/rust-clippy/issues/11380. - // TODO: remove `allow` here after the issued is closed. - #[expect(clippy::needless_pass_by_ref_mut)] async fn assert_pending(future: &mut (impl Future + Unpin)) { for _ in 0..10 { assert!(poll_fn(|cx| Poll::Ready(future.poll_unpin(cx))) From 0f9e783d2ab5c02811e3c8254e6979cf63b46b7f Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Thu, 26 Oct 2023 14:22:04 +0800 Subject: [PATCH 30/48] refactor: always apply flow control on source/chain (#13057) --- src/stream/src/executor/flow_control.rs | 31 +++++++++++-------- src/stream/src/from_proto/chain.rs | 7 ++--- src/stream/src/from_proto/source/fs_fetch.rs | 6 ++-- .../src/from_proto/source/trad_source.rs | 7 ++--- 4 files changed, 24 insertions(+), 27 deletions(-) diff --git a/src/stream/src/executor/flow_control.rs b/src/stream/src/executor/flow_control.rs index 45e04717e2a9d..230359109fca0 100644 --- a/src/stream/src/executor/flow_control.rs +++ b/src/stream/src/executor/flow_control.rs @@ -32,12 +32,12 @@ use super::*; /// It is used to throttle problematic MVs that are consuming too much resources. pub struct FlowControlExecutor { input: BoxedExecutor, - rate_limit: u32, + rate_limit: Option, } impl FlowControlExecutor { #[allow(clippy::too_many_arguments)] - pub fn new(input: Box, rate_limit: u32) -> Self { + pub fn new(input: Box, rate_limit: Option) -> Self { #[cfg(madsim)] println!("FlowControlExecutor rate limiter is disabled in madsim as it will spawn system threads"); Self { input, rate_limit } @@ -45,9 +45,12 @@ impl FlowControlExecutor { #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(self) { - let quota = Quota::per_second(NonZeroU32::new(self.rate_limit).unwrap()); - let clock = MonotonicClock; - let rate_limiter = RateLimiter::direct_with_clock(quota, &clock); + let get_rate_limiter = |rate_limit: u32| { + let quota = Quota::per_second(NonZeroU32::new(rate_limit).unwrap()); + let clock = MonotonicClock; + RateLimiter::direct_with_clock(quota, &clock) + }; + let rate_limiter = self.rate_limit.map(get_rate_limiter); #[for_await] for msg in self.input.execute() { let msg = msg?; @@ -55,14 +58,16 @@ impl FlowControlExecutor { Message::Chunk(chunk) => { #[cfg(not(madsim))] { - let result = rate_limiter - .until_n_ready(NonZeroU32::new(chunk.cardinality() as u32).unwrap()) - .await; - if let Err(InsufficientCapacity(n)) = result { - tracing::error!( - "Rate Limit {} smaller than chunk cardinality {n}", - self.rate_limit, - ); + if let Some(rate_limiter) = &rate_limiter { + let result = rate_limiter + .until_n_ready(NonZeroU32::new(chunk.cardinality() as u32).unwrap()) + .await; + if let Err(InsufficientCapacity(n)) = result { + tracing::error!( + "Rate Limit {:?} smaller than chunk cardinality {n}", + self.rate_limit, + ); + } } } yield Message::Chunk(chunk); diff --git a/src/stream/src/from_proto/chain.rs b/src/stream/src/from_proto/chain.rs index 81030526b82f3..ba48d2c5a25f1 100644 --- a/src/stream/src/from_proto/chain.rs +++ b/src/stream/src/from_proto/chain.rs @@ -178,10 +178,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { } ChainType::ChainUnspecified => unreachable!(), }; - if let Ok(rate_limit) = node.get_rate_limit() { - Ok(FlowControlExecutor::new(executor, *rate_limit).boxed()) - } else { - Ok(executor) - } + let rate_limit = node.get_rate_limit().cloned().ok(); + Ok(FlowControlExecutor::new(executor, rate_limit).boxed()) } } diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index b6df84c8560e4..65923d2dced3a 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -112,9 +112,7 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { ) .boxed(); - if let Ok(rate_limit) = source.get_rate_limit() { - return Ok(FlowControlExecutor::new(executor, *rate_limit).boxed()); - } - Ok(executor) + let rate_limit = source.get_rate_limit().cloned().ok(); + 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 3f0793595c7c5..b87ce5ff39dc7 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -207,11 +207,8 @@ impl ExecutorBuilder for SourceExecutorBuilder { } } }; - if let Ok(rate_limit) = source.get_rate_limit() { - Ok(FlowControlExecutor::new(executor, *rate_limit).boxed()) - } else { - Ok(executor) - } + let rate_limit = source.get_rate_limit().cloned().ok(); + Ok(FlowControlExecutor::new(executor, rate_limit).boxed()) } else { // If there is no external stream source, then no data should be persisted. We pass a // `PanicStateStore` type here for indication. From 9d08bb28adbcaa95cf1f5df94c7f96acb95c2072 Mon Sep 17 00:00:00 2001 From: "Zhanxiang (Patrick) Huang" Date: Thu, 26 Oct 2023 14:26:44 +0800 Subject: [PATCH 31/48] fix(sink): fix integer array type for remote sink (#12334) --- e2e_test/sink/remote/jdbc.check.pg.slt | 8 ++--- e2e_test/sink/remote/jdbc.load.slt | 14 +++++---- e2e_test/sink/remote/mysql_create_table.sql | 6 +++- .../sink/remote/mysql_expected_result_2.tsv | 6 ++-- e2e_test/sink/remote/pg_create_table.sql | 6 +++- .../connector/jdbc/PostgresDialect.java | 30 +++++++++++++++++-- src/jni_core/src/lib.rs | 6 ++-- 7 files changed, 56 insertions(+), 20 deletions(-) diff --git a/e2e_test/sink/remote/jdbc.check.pg.slt b/e2e_test/sink/remote/jdbc.check.pg.slt index 6293c44a5a444..7cf6faa7b0fb6 100644 --- a/e2e_test/sink/remote/jdbc.check.pg.slt +++ b/e2e_test/sink/remote/jdbc.check.pg.slt @@ -22,13 +22,13 @@ select * from t_remote_1 order by id; query III select * from biz.t_types order by id; ---- -1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} {"Value 1","Value 2"} {12.345,56.789} -2 Varcharvalue2 Textvalue2 234 567 890 23.45 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} {"Value 3","Value 4"} {43.21,65.432} -3 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} {"How're you?","\"hello\\ \\world\""} {12.345,56.789} +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} {"Value 1","Value 2"} {12.345,56.789} {1,2,3} {1,2,3} {1,2,3} {12.3,56.7} +2 Varcharvalue2 Textvalue2 234 567 890 23.45 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} {"Value 3","Value 4"} {43.21,65.432} {4,5,6} {4,5,6} {4,5,6} {43.2,65.4} +3 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} {"How're you?","\"hello\\ \\world\""} {12.345,56.789} {1,2,3} {1,2,3} {1,2,3} {43.2,65.4} query IT -select * from t_append_only order by v1, v2; +select * from t_append_only order by v1,v2; ---- 1 aaa 1 bbb diff --git a/e2e_test/sink/remote/jdbc.load.slt b/e2e_test/sink/remote/jdbc.load.slt index 70ad3f9a3a42b..a3bc63e48f7de 100644 --- a/e2e_test/sink/remote/jdbc.load.slt +++ b/e2e_test/sink/remote/jdbc.load.slt @@ -50,7 +50,11 @@ CREATE TABLE rw_typed_data ( interval_column INTERVAL, jsonb_column JSONB, array_column VARCHAR[], - array_column2 FLOAT[] + array_column2 FLOAT[], + array_column3 SMALLINT[], + array_column4 INTEGER[], + array_column5 BIGINT[], + array_column6 DOUBLE PRECISION[], ); statement ok @@ -196,10 +200,10 @@ INSERT INTO t_remote_1 VALUES (6, 'Varchar value 6', 'Text value 6', 789, 123, 456, 67.89, 34.56, 78.91, FALSE, '2023-05-27', '23:45:01', '2023-05-27 23:45:01', '2023-05-27 23:45:01', '2 years 3 months 4 days 5 hours 6 minutes 7 seconds', '{"key": "value6"}', E'\\xDEADBABE'); statement ok -INSERT INTO rw_typed_data (id, varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, interval_column, jsonb_column, array_column, array_column2) VALUES - (1, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}', ARRAY['Value 1', 'Value 2'], '{12.345,56.789}'), - (2, 'Varcharvalue2', 'Textvalue2', 234, 567, 890, 23.45, 67.89, 01.23, FALSE, '2023-05-23', '23:45:01', '2023-05-23 23:45:01', '2 days', '{"key": "value2"}', ARRAY['Value 3', 'Value 4'], '{43.21,65.432}'), - (3, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}', ARRAY['How''re you?', '"hello\ \world"'], ARRAY[12.345,56.789]); +INSERT INTO rw_typed_data (id, varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, interval_column, jsonb_column, array_column, array_column2, array_column3, array_column4, array_column5, array_column6) VALUES + (1, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}', ARRAY['Value 1', 'Value 2'], '{12.345,56.789}', '{1, 2, 3}', '{1, 2, 3}', '{1, 2, 3}', '{12.3,56.7}'), + (2, 'Varcharvalue2', 'Textvalue2', 234, 567, 890, 23.45, 67.89, 01.23, FALSE, '2023-05-23', '23:45:01', '2023-05-23 23:45:01', '2 days', '{"key": "value2"}', ARRAY['Value 3', 'Value 4'], '{43.21,65.432}', '{4, 5, 6}', '{4, 5, 6}', '{4, 5, 6}', '{43.2,65.4}'), + (3, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}', ARRAY['How''re you?', '"hello\ \world"'], ARRAY[12.345,56.789], ARRAY[1, 2, 3], ARRAY[1, 2, 3], ARRAY[1, 2, 3], ARRAY[43.2,65.4]); statement ok FLUSH; diff --git a/e2e_test/sink/remote/mysql_create_table.sql b/e2e_test/sink/remote/mysql_create_table.sql index 0cbe15f7dcb8e..9eab3d0f5e13a 100644 --- a/e2e_test/sink/remote/mysql_create_table.sql +++ b/e2e_test/sink/remote/mysql_create_table.sql @@ -47,5 +47,9 @@ CREATE TABLE t_types ( interval_column VARCHAR(100), jsonb_column JSON, array_column LONGTEXT, - array_column2 LONGTEXT + array_column2 LONGTEXT, + array_column3 LONGTEXT, + array_column4 LONGTEXT, + array_column5 LONGTEXT, + array_column6 LONGTEXT ); diff --git a/e2e_test/sink/remote/mysql_expected_result_2.tsv b/e2e_test/sink/remote/mysql_expected_result_2.tsv index 87ac3cb3bd123..061ee02d39d17 100644 --- a/e2e_test/sink/remote/mysql_expected_result_2.tsv +++ b/e2e_test/sink/remote/mysql_expected_result_2.tsv @@ -1,3 +1,3 @@ -1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 1 2023-05-22 12:34:56 2023-05-22 12:34:56 P0Y0M1DT0H0M0S {"key": "value"} Value 1,Value 2 12.345,56.789 -2 Varcharvalue2 Textvalue2 234 567 890 23.45 67.89 1.23 0 2023-05-23 23:45:01 2023-05-23 23:45:01 P0Y0M2DT0H0M0S {"key": "value2"} Value 3,Value 4 43.21,65.432 -3 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 1 2023-05-22 12:34:56 2023-05-22 12:34:56 P0Y0M1DT0H0M0S {"key": "value"} How're you?,"hello\ \world" 12.345,56.789 +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 1 2023-05-22 12:34:56 2023-05-22 12:34:56 P0Y0M1DT0H0M0S {"key": "value"} Value 1,Value 2 12.345,56.789 1,2,3 1,2,3 1,2,3 12.3,56.7 +2 Varcharvalue2 Textvalue2 234 567 890 23.45 67.89 1.23 0 2023-05-23 23:45:01 2023-05-23 23:45:01 P0Y0M2DT0H0M0S {"key": "value2"} Value 3,Value 4 43.21,65.432 4,5,6 4,5,6 4,5,6 43.2,65.4 +3 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 1 2023-05-22 12:34:56 2023-05-22 12:34:56 P0Y0M1DT0H0M0S {"key": "value"} How're you?,"hello\ \world" 12.345,56.789 1,2,3 1,2,3 1,2,3 43.2,65.4 diff --git a/e2e_test/sink/remote/pg_create_table.sql b/e2e_test/sink/remote/pg_create_table.sql index fd06aca93ce7b..3677dbb067131 100644 --- a/e2e_test/sink/remote/pg_create_table.sql +++ b/e2e_test/sink/remote/pg_create_table.sql @@ -73,7 +73,11 @@ CREATE TABLE biz.t_types ( interval_column INTERVAL, jsonb_column JSONB, array_column VARCHAR[], - array_column2 DECIMAL[] + array_column2 FLOAT[], + array_column3 SMALLINT[], + array_column4 INTEGER[], + array_column5 BIGINT[], + array_column6 DOUBLE PRECISION[] ); CREATE TABLE biz.t2 ( diff --git a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/jdbc/PostgresDialect.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/jdbc/PostgresDialect.java index 570e2beaf5a67..5bbdf1116f2c1 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/jdbc/PostgresDialect.java +++ b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/jdbc/PostgresDialect.java @@ -16,9 +16,11 @@ import com.risingwave.connector.api.TableSchema; import com.risingwave.connector.api.sink.SinkRow; +import com.risingwave.proto.Data.DataType.TypeName; import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.SQLException; +import java.util.HashMap; import java.util.List; import java.util.Optional; import java.util.stream.Collectors; @@ -33,6 +35,26 @@ public PostgresDialect(int[] columnSqlTypes) { this.columnSqlTypes = columnSqlTypes; } + private static final HashMap RW_TYPE_TO_JDBC_TYPE_NAME; + + static { + RW_TYPE_TO_JDBC_TYPE_NAME = new HashMap(); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.INT16, "int2"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.INT32, "int4"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.INT64, "int8"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.FLOAT, "float4"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.DOUBLE, "float8"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.BOOLEAN, "bool"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.VARCHAR, "varchar"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.DECIMAL, "numeric"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.TIME, "time"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.TIMESTAMP, "timestamp"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.INTERVAL, "varchar"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.DATE, "date"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.TIMESTAMPTZ, "timestamptz"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.JSONB, "varchar"); + } + @Override public SchemaTableName createSchemaTableName(String schemaName, String tableName) { if (schemaName == null || schemaName.isBlank()) { @@ -115,9 +137,11 @@ public void bindInsertIntoStatement( Object[] objArray = (Object[]) val; assert (column.getDataType().getFieldTypeCount() == 1); var fieldType = column.getDataType().getFieldType(0); - stmt.setArray( - placeholderIdx++, - conn.createArrayOf(fieldType.getTypeName().name(), objArray)); + var typeName = RW_TYPE_TO_JDBC_TYPE_NAME.get(fieldType.getTypeName()); + if (typeName == null) { + typeName = fieldType.getTypeName().name(); + } + stmt.setArray(placeholderIdx++, conn.createArrayOf(typeName, objArray)); break; case VARCHAR: // since VARCHAR column may sink to a UUID column, we get the target type diff --git a/src/jni_core/src/lib.rs b/src/jni_core/src/lib.rs index 4815cd7368370..c92ad2f146e6c 100644 --- a/src/jni_core/src/lib.rs +++ b/src/jni_core/src/lib.rs @@ -782,7 +782,7 @@ extern "system" fn Java_com_risingwave_java_binding_Binding_iteratorGetArrayValu let obj = env.call_static_method( &class, "valueOf", - "(S)Ljava.lang.Short;", + "(S)Ljava/lang/Short;", &[JValue::from(v as jshort)], )?; if let JValueOwned::Object(o) = obj { @@ -793,7 +793,7 @@ extern "system" fn Java_com_risingwave_java_binding_Binding_iteratorGetArrayValu let obj = env.call_static_method( &class, "valueOf", - "(I)Ljava.lang.Integer;", + "(I)Ljava/lang/Integer;", &[JValue::from(v as jint)], )?; if let JValueOwned::Object(o) = obj { @@ -804,7 +804,7 @@ extern "system" fn Java_com_risingwave_java_binding_Binding_iteratorGetArrayValu let obj = env.call_static_method( &class, "valueOf", - "(J)Ljava.lang.Long;", + "(J)Ljava/lang/Long;", &[JValue::from(v as jlong)], )?; if let JValueOwned::Object(o) = obj { From efdf3c9688a04232ebf5c7876220b965444e48db Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Thu, 26 Oct 2023 14:39:44 +0800 Subject: [PATCH 32/48] refactor(optimizer): type-safe plan base with compile-time convention check (#13000) Signed-off-by: Bugen Zhao --- Cargo.lock | 1 + src/frontend/Cargo.toml | 1 + src/frontend/src/handler/explain.rs | 1 + src/frontend/src/optimizer/mod.rs | 8 +- src/frontend/src/optimizer/plan_node/batch.rs | 8 + .../src/optimizer/plan_node/batch_delete.rs | 6 +- .../src/optimizer/plan_node/batch_exchange.rs | 5 +- .../src/optimizer/plan_node/batch_expand.rs | 3 +- .../src/optimizer/plan_node/batch_filter.rs | 3 +- .../optimizer/plan_node/batch_group_topn.rs | 3 +- .../src/optimizer/plan_node/batch_hash_agg.rs | 3 +- .../optimizer/plan_node/batch_hash_join.rs | 3 +- .../optimizer/plan_node/batch_hop_window.rs | 3 +- .../src/optimizer/plan_node/batch_insert.rs | 5 +- .../src/optimizer/plan_node/batch_limit.rs | 3 +- .../optimizer/plan_node/batch_lookup_join.rs | 3 +- .../plan_node/batch_nested_loop_join.rs | 3 +- .../optimizer/plan_node/batch_over_window.rs | 3 +- .../src/optimizer/plan_node/batch_project.rs | 4 +- .../optimizer/plan_node/batch_project_set.rs | 3 +- .../src/optimizer/plan_node/batch_seq_scan.rs | 5 +- .../optimizer/plan_node/batch_simple_agg.rs | 3 +- .../src/optimizer/plan_node/batch_sort.rs | 3 +- .../src/optimizer/plan_node/batch_sort_agg.rs | 3 +- .../src/optimizer/plan_node/batch_source.rs | 4 +- .../plan_node/batch_table_function.rs | 4 +- .../src/optimizer/plan_node/batch_topn.rs | 3 +- .../src/optimizer/plan_node/batch_union.rs | 3 +- .../src/optimizer/plan_node/batch_update.rs | 3 +- .../src/optimizer/plan_node/batch_values.rs | 4 +- .../src/optimizer/plan_node/generic/agg.rs | 14 +- .../plan_node/generic/dynamic_filter.rs | 4 +- .../src/optimizer/plan_node/generic/join.rs | 4 +- .../src/optimizer/plan_node/generic/mod.rs | 2 + .../src/optimizer/plan_node/logical_agg.rs | 8 +- .../src/optimizer/plan_node/logical_apply.rs | 6 +- .../src/optimizer/plan_node/logical_dedup.rs | 10 +- .../src/optimizer/plan_node/logical_delete.rs | 6 +- .../src/optimizer/plan_node/logical_except.rs | 6 +- .../src/optimizer/plan_node/logical_expand.rs | 6 +- .../src/optimizer/plan_node/logical_filter.rs | 6 +- .../optimizer/plan_node/logical_hop_window.rs | 7 +- .../src/optimizer/plan_node/logical_insert.rs | 6 +- .../optimizer/plan_node/logical_intersect.rs | 6 +- .../src/optimizer/plan_node/logical_join.rs | 12 +- .../src/optimizer/plan_node/logical_limit.rs | 6 +- .../optimizer/plan_node/logical_multi_join.rs | 4 +- .../src/optimizer/plan_node/logical_now.rs | 4 +- .../plan_node/logical_over_window.rs | 6 +- .../optimizer/plan_node/logical_project.rs | 6 +- .../plan_node/logical_project_set.rs | 7 +- .../src/optimizer/plan_node/logical_scan.rs | 4 +- .../src/optimizer/plan_node/logical_share.rs | 8 +- .../src/optimizer/plan_node/logical_source.rs | 8 +- .../plan_node/logical_table_function.rs | 4 +- .../src/optimizer/plan_node/logical_topn.rs | 10 +- .../src/optimizer/plan_node/logical_union.rs | 6 +- .../src/optimizer/plan_node/logical_update.rs | 6 +- .../src/optimizer/plan_node/logical_values.rs | 6 +- src/frontend/src/optimizer/plan_node/mod.rs | 184 ++++++---- .../src/optimizer/plan_node/plan_base.rs | 316 ++++++++++++------ .../src/optimizer/plan_node/stream.rs | 8 + .../src/optimizer/plan_node/stream_dedup.rs | 8 +- .../optimizer/plan_node/stream_delta_join.rs | 3 +- .../src/optimizer/plan_node/stream_dml.rs | 3 +- .../plan_node/stream_dynamic_filter.rs | 3 +- .../plan_node/stream_eowc_over_window.rs | 3 +- .../optimizer/plan_node/stream_exchange.rs | 5 +- .../src/optimizer/plan_node/stream_expand.rs | 3 +- .../src/optimizer/plan_node/stream_filter.rs | 3 +- .../optimizer/plan_node/stream_fs_fetch.rs | 4 +- .../optimizer/plan_node/stream_group_topn.rs | 3 +- .../optimizer/plan_node/stream_hash_agg.rs | 3 +- .../optimizer/plan_node/stream_hash_join.rs | 3 +- .../optimizer/plan_node/stream_hop_window.rs | 3 +- .../optimizer/plan_node/stream_materialize.rs | 3 +- .../src/optimizer/plan_node/stream_now.rs | 4 +- .../optimizer/plan_node/stream_over_window.rs | 3 +- .../src/optimizer/plan_node/stream_project.rs | 3 +- .../optimizer/plan_node/stream_project_set.rs | 3 +- .../optimizer/plan_node/stream_row_id_gen.rs | 3 +- .../src/optimizer/plan_node/stream_share.rs | 27 +- .../optimizer/plan_node/stream_simple_agg.rs | 5 +- .../src/optimizer/plan_node/stream_sink.rs | 9 +- .../src/optimizer/plan_node/stream_sort.rs | 5 +- .../src/optimizer/plan_node/stream_source.rs | 4 +- .../plan_node/stream_stateless_simple_agg.rs | 3 +- .../optimizer/plan_node/stream_table_scan.rs | 6 +- .../plan_node/stream_temporal_join.rs | 3 +- .../src/optimizer/plan_node/stream_topn.rs | 3 +- .../src/optimizer/plan_node/stream_union.rs | 3 +- .../src/optimizer/plan_node/stream_values.rs | 4 +- .../plan_node/stream_watermark_filter.rs | 9 +- .../src/optimizer/property/distribution.rs | 1 + src/frontend/src/optimizer/property/order.rs | 2 + src/frontend/src/scheduler/plan_fragmenter.rs | 2 +- 96 files changed, 622 insertions(+), 333 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3f7df00f7648d..1c741e3a5ae67 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7491,6 +7491,7 @@ dependencies = [ "async-recursion", "async-trait", "auto_enums", + "auto_impl", "bk-tree", "bytes", "clap", diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 37f9f6326faea..bae582dd06e24 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -21,6 +21,7 @@ arrow-schema = { workspace = true } async-recursion = "1.0.5" async-trait = "0.1" auto_enums = { version = "0.8", features = ["futures03"] } +auto_impl = "1" bk-tree = "0.5.0" bytes = "1" clap = { version = "4", features = ["derive"] } diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index 5de7ec95b38bd..fe798f3fa2857 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -30,6 +30,7 @@ use super::create_table::{ use super::query::gen_batch_plan_by_statement; use super::RwPgResponse; use crate::handler::HandlerArgs; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{Convention, Explain}; use crate::optimizer::OptimizerContext; use crate::scheduler::worker_node_manager::WorkerNodeSelector; diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 0df387b0a53d5..4004748a2f4f9 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -17,6 +17,7 @@ use std::ops::DerefMut; pub mod plan_node; pub use plan_node::{Explain, PlanRef}; + pub mod property; mod delta_join_solver; @@ -46,10 +47,11 @@ use risingwave_connector::sink::catalog::SinkFormatDesc; use risingwave_pb::catalog::WatermarkDesc; use self::heuristic_optimizer::ApplyOrder; +use self::plan_node::generic::{self, PhysicalPlanRef}; use self::plan_node::{ - generic, stream_enforce_eowc_requirement, BatchProject, Convention, LogicalProject, - LogicalSource, StreamDml, StreamMaterialize, StreamProject, StreamRowIdGen, StreamSink, - StreamWatermarkFilter, ToStreamContext, + stream_enforce_eowc_requirement, BatchProject, Convention, LogicalProject, LogicalSource, + StreamDml, StreamMaterialize, StreamProject, StreamRowIdGen, StreamSink, StreamWatermarkFilter, + ToStreamContext, }; #[cfg(debug_assertions)] use self::plan_visitor::InputRefValidator; diff --git a/src/frontend/src/optimizer/plan_node/batch.rs b/src/frontend/src/optimizer/plan_node/batch.rs index 2cb2360b3e51d..5eeafab64f1cd 100644 --- a/src/frontend/src/optimizer/plan_node/batch.rs +++ b/src/frontend/src/optimizer/plan_node/batch.rs @@ -23,6 +23,14 @@ use crate::optimizer::property::Order; /// [`PhysicalPlanRef`]. /// /// [`GenericPlanRef`]: super::generic::GenericPlanRef +#[auto_impl::auto_impl(&)] pub trait BatchPlanRef: PhysicalPlanRef { fn order(&self) -> &Order; } + +/// Prelude for batch plan nodes. +pub mod prelude { + pub use super::super::generic::{GenericPlanRef, PhysicalPlanRef}; + pub use super::super::Batch; + pub use super::BatchPlanRef; +} diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index 85d22a46b450e..96ca967a2f745 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -16,24 +16,26 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::DeleteNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; +use crate::optimizer::plan_node::generic::PhysicalPlanRef; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchDelete` implements [`super::LogicalDelete`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchDelete { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Delete, } impl BatchDelete { pub fn new(core: generic::Delete) -> Self { assert_eq!(core.input.distribution(), &Distribution::Single); - let base: PlanBase = + let base = PlanBase::new_batch_with_core(&core, core.input.distribution().clone(), Order::any()); Self { base, core } } diff --git a/src/frontend/src/optimizer/plan_node/batch_exchange.rs b/src/frontend/src/optimizer/plan_node/batch_exchange.rs index 6477c7ec213e2..ec3a195e050bb 100644 --- a/src/frontend/src/optimizer/plan_node/batch_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/batch_exchange.rs @@ -17,8 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{ExchangeNode, MergeSortExchangeNode}; -use super::batch::BatchPlanRef; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; use crate::optimizer::plan_node::ToLocalBatch; @@ -28,7 +27,7 @@ use crate::optimizer::property::{Distribution, DistributionDisplay, Order, Order /// without changing its content. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchExchange { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, } diff --git a/src/frontend/src/optimizer/plan_node/batch_expand.rs b/src/frontend/src/optimizer/plan_node/batch_expand.rs index af4413b9e5152..7b31d59fd13e5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_expand.rs +++ b/src/frontend/src/optimizer/plan_node/batch_expand.rs @@ -18,6 +18,7 @@ use risingwave_pb::batch_plan::expand_node::Subset; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ExpandNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable}; use crate::optimizer::plan_node::{ @@ -28,7 +29,7 @@ use crate::optimizer::PlanRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchExpand { - pub base: PlanBase, + pub base: PlanBase, core: generic::Expand, } diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index 4bff7cbfee3c0..c5c1430772750 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::FilterNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; use crate::expr::{Expr, ExprImpl, ExprRewriter}; @@ -25,7 +26,7 @@ use crate::utils::Condition; /// `BatchFilter` implements [`super::LogicalFilter`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::Filter, } diff --git a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs index 70ee8328623f5..1817c90442761 100644 --- a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::GroupTopNNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, @@ -26,7 +27,7 @@ use crate::optimizer::property::{Order, RequiredDist}; /// `BatchGroupTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchGroupTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs index b4ab3341ace29..58512a42f0240 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -17,6 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashAggNode; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef, PlanAggCall}; use super::utils::impl_distill_by_unit; use super::{ @@ -30,7 +31,7 @@ use crate::utils::{ColIndexMappingRewriteExt, IndexSet}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchHashAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, } diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs index bad586d4af1e4..d2135bb47b520 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -18,6 +18,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashJoinNode; use risingwave_pb::plan_common::JoinType; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::{ @@ -35,7 +36,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// get output rows. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchHashJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs index 2a4a27f9a0583..3cbfbb38372f4 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HopWindowNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, @@ -29,7 +30,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// input rows #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchHopWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::HopWindow, window_start_exprs: Vec, window_end_exprs: Vec, diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index aec05eee145b8..2d89b2a14b128 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -18,6 +18,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::InsertNode; use risingwave_pb::plan_common::{DefaultColumns, IndexAndExpr}; +use super::batch::prelude::*; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; @@ -28,14 +29,14 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchInsert` implements [`super::LogicalInsert`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchInsert { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Insert, } impl BatchInsert { pub fn new(core: generic::Insert) -> Self { assert_eq!(core.input.distribution(), &Distribution::Single); - let base: PlanBase = + let base: PlanBase = PlanBase::new_batch_with_core(&core, core.input.distribution().clone(), Order::any()); BatchInsert { base, core } diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index 93b14d0198979..f1b9739e8ac2b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::LimitNode; +use super::batch::prelude::*; use super::generic::PhysicalPlanRef; use super::utils::impl_distill_by_unit; use super::{ @@ -27,7 +28,7 @@ use crate::optimizer::property::{Order, RequiredDist}; /// `BatchLimit` implements [`super::LogicalLimit`] to fetch specified rows from input #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchLimit { - pub base: PlanBase, + pub base: PlanBase, core: generic::Limit, } diff --git a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs index 48f99668c3af7..b78bf314c1276 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -18,6 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{DistributedLookupJoinNode, LocalLookupJoinNode}; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::ExprRewritable; @@ -33,7 +34,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchLookupJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs index 8980ad2f23f6d..fe789b47937f5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs @@ -17,6 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::NestedLoopJoinNode; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchPb, ToDistributedBatch}; @@ -30,7 +31,7 @@ use crate::utils::ConditionDisplay; /// against all pairs of rows from inner & outer side within 2 layers of loops. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchNestedLoopJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, } diff --git a/src/frontend/src/optimizer/plan_node/batch_over_window.rs b/src/frontend/src/optimizer/plan_node/batch_over_window.rs index fb455758f331a..011de0dfb1459 100644 --- a/src/frontend/src/optimizer/plan_node/batch_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_over_window.rs @@ -17,6 +17,7 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortOverWindowNode; +use super::batch::prelude::*; use super::batch::BatchPlanRef; use super::generic::PlanWindowFunction; use super::utils::impl_distill_by_unit; @@ -28,7 +29,7 @@ use crate::optimizer::property::{Order, RequiredDist}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index 642683967c5c3..b32498d547dbf 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -18,7 +18,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ProjectNode; use risingwave_pb::expr::ExprNode; -use super::generic::GenericPlanRef; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, @@ -31,7 +31,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// rows #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchProject { - pub base: PlanBase, + pub base: PlanBase, core: generic::Project, } diff --git a/src/frontend/src/optimizer/plan_node/batch_project_set.rs b/src/frontend/src/optimizer/plan_node/batch_project_set.rs index 5888df9d15889..5a355c0db22db 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -17,6 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ProjectSetNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable}; use crate::expr::ExprRewriter; @@ -28,7 +29,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchProjectSet { - pub base: PlanBase, + pub base: PlanBase, core: generic::ProjectSet, } 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 6834ed29353b9..7a2d97c266b36 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -24,8 +24,7 @@ use risingwave_pb::batch_plan::row_seq_scan_node::ChunkSize; use risingwave_pb::batch_plan::{RowSeqScanNode, SysRowSeqScanNode}; use risingwave_pb::plan_common::PbColumnDesc; -use super::batch::BatchPlanRef; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch}; use crate::catalog::ColumnId; @@ -36,7 +35,7 @@ use crate::optimizer::property::{Distribution, DistributionDisplay, Order}; /// `BatchSeqScan` implements [`super::LogicalScan`] to scan from a row-oriented table #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSeqScan { - pub base: PlanBase, + pub base: PlanBase, core: generic::Scan, scan_ranges: Vec, } diff --git a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs index bae8d70c2eedf..cbc8f0a95c1f8 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef, PlanAggCall}; use super::utils::impl_distill_by_unit; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; @@ -25,7 +26,7 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSimpleAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, } diff --git a/src/frontend/src/optimizer/plan_node/batch_sort.rs b/src/frontend/src/optimizer/plan_node/batch_sort.rs index e7bff6d51d85b..c2e713c68138a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort.rs @@ -17,6 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortNode; +use super::batch::prelude::*; use super::batch::BatchPlanRef; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; @@ -27,7 +28,7 @@ use crate::optimizer::property::{Order, OrderDisplay}; /// collation required by user or parent plan node. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSort { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, } diff --git a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs index 2252d4c0c0ee0..cd8e3b43ca8ec 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -17,6 +17,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; use risingwave_pb::expr::ExprNode; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef, PlanAggCall}; use super::utils::impl_distill_by_unit; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; @@ -27,7 +28,7 @@ use crate::utils::{ColIndexMappingRewriteExt, IndexSet}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSortAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, input_order: Order, } diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index 9e2cd6006db0b..56f6399cd844e 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -19,7 +19,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SourceNode; -use super::generic::GenericPlanRef; +use super::batch::prelude::*; use super::utils::{childless_record, column_names_pretty, Distill}; use super::{ generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch, ToLocalBatch, @@ -30,7 +30,7 @@ use crate::optimizer::property::{Distribution, Order}; /// [`BatchSource`] represents a table/connector source at the very beginning of the graph. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSource { - pub base: PlanBase, + pub base: PlanBase, core: generic::Source, } diff --git a/src/frontend/src/optimizer/plan_node/batch_table_function.rs b/src/frontend/src/optimizer/plan_node/batch_table_function.rs index 0b9887cd4aaba..a249ac722cdcf 100644 --- a/src/frontend/src/optimizer/plan_node/batch_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/batch_table_function.rs @@ -17,7 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TableFunctionNode; -use super::generic::GenericPlanRef; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchPb, ToDistributedBatch}; use crate::expr::ExprRewriter; @@ -27,7 +27,7 @@ use crate::optimizer::property::{Distribution, Order}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchTableFunction { - pub base: PlanBase, + pub base: PlanBase, logical: LogicalTableFunction, } diff --git a/src/frontend/src/optimizer/plan_node/batch_topn.rs b/src/frontend/src/optimizer/plan_node/batch_topn.rs index b2eda24046d28..d508b0da53317 100644 --- a/src/frontend/src/optimizer/plan_node/batch_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_topn.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TopNNode; +use super::batch::prelude::*; use super::generic::TopNLimit; use super::utils::impl_distill_by_unit; use super::{ @@ -28,7 +29,7 @@ use crate::optimizer::property::{Order, RequiredDist}; /// `BatchTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/batch_union.rs b/src/frontend/src/optimizer/plan_node/batch_union.rs index c7c71111174c6..5f9e354970a65 100644 --- a/src/frontend/src/optimizer/plan_node/batch_union.rs +++ b/src/frontend/src/optimizer/plan_node/batch_union.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UnionNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanRef, ToBatchPb, ToDistributedBatch}; use crate::optimizer::plan_node::{PlanBase, PlanTreeNode, ToLocalBatch}; @@ -24,7 +25,7 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchUnion` implements [`super::LogicalUnion`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchUnion { - pub base: PlanBase, + pub base: PlanBase, core: generic::Union, } diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index 20e4b8b6b966c..fdcc546f873a8 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -18,6 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UpdateNode; +use super::batch::prelude::*; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; use super::{ @@ -30,7 +31,7 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchUpdate` implements [`super::LogicalUpdate`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchUpdate { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Update, } diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index 9348cddba7422..00483f37256f4 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -18,7 +18,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::values_node::ExprTuple; use risingwave_pb::batch_plan::ValuesNode; -use super::generic::GenericPlanRef; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ ExprRewritable, LogicalValues, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchPb, @@ -30,7 +30,7 @@ use crate::optimizer::property::{Distribution, Order}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchValues { - pub base: PlanBase, + pub base: PlanBase, logical: LogicalValues, } diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index e0c7e339ee6a6..98109e695110f 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -265,7 +265,7 @@ pub struct MaterializedInputState { impl Agg { pub fn infer_tables( &self, - me: &impl stream::StreamPlanRef, + me: impl stream::StreamPlanRef, vnode_col_idx: Option, window_col_idx: Option, ) -> ( @@ -274,9 +274,9 @@ impl Agg { HashMap, ) { ( - self.infer_intermediate_state_table(me, vnode_col_idx, window_col_idx), - self.infer_stream_agg_state(me, vnode_col_idx, window_col_idx), - self.infer_distinct_dedup_tables(me, vnode_col_idx, window_col_idx), + self.infer_intermediate_state_table(&me, vnode_col_idx, window_col_idx), + self.infer_stream_agg_state(&me, vnode_col_idx, window_col_idx), + self.infer_distinct_dedup_tables(&me, vnode_col_idx, window_col_idx), ) } @@ -338,7 +338,7 @@ impl Agg { /// Infer `AggCallState`s for streaming agg. pub fn infer_stream_agg_state( &self, - me: &impl stream::StreamPlanRef, + me: impl stream::StreamPlanRef, vnode_col_idx: Option, window_col_idx: Option, ) -> Vec { @@ -487,7 +487,7 @@ impl Agg { /// group key | state for AGG1 | state for AGG2 | ... pub fn infer_intermediate_state_table( &self, - me: &impl GenericPlanRef, + me: impl GenericPlanRef, vnode_col_idx: Option, window_col_idx: Option, ) -> TableCatalog { @@ -550,7 +550,7 @@ impl Agg { /// group key | distinct key | count for AGG1(distinct x) | count for AGG2(distinct x) | ... pub fn infer_distinct_dedup_tables( &self, - me: &impl GenericPlanRef, + me: impl GenericPlanRef, vnode_col_idx: Option, window_col_idx: Option, ) -> HashMap { diff --git a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs index 85ffd922c43e7..edadd8b07ae3f 100644 --- a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs @@ -140,7 +140,7 @@ impl DynamicFilter { } pub fn infer_left_internal_table_catalog( - me: &impl stream::StreamPlanRef, + me: impl stream::StreamPlanRef, left_key_index: usize, ) -> TableCatalog { let schema = me.schema(); @@ -171,7 +171,7 @@ pub fn infer_left_internal_table_catalog( internal_table_catalog_builder.build(dist_keys, read_prefix_len_hint) } -pub fn infer_right_internal_table_catalog(input: &impl stream::StreamPlanRef) -> TableCatalog { +pub fn infer_right_internal_table_catalog(input: impl stream::StreamPlanRef) -> TableCatalog { let schema = input.schema(); // We require that the right table has distribution `Single` diff --git a/src/frontend/src/optimizer/plan_node/generic/join.rs b/src/frontend/src/optimizer/plan_node/generic/join.rs index 87c03cc14c8c9..2536cee984558 100644 --- a/src/frontend/src/optimizer/plan_node/generic/join.rs +++ b/src/frontend/src/optimizer/plan_node/generic/join.rs @@ -70,10 +70,10 @@ impl Join { } } -impl Join { +impl Join { /// Return stream hash join internal table catalog and degree table catalog. pub fn infer_internal_and_degree_table_catalog( - input: &PlanRef, + input: I, join_key_indices: Vec, dk_indices_in_jk: Vec, ) -> (TableCatalog, TableCatalog, Vec) { diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index aec59c90bcc4e..fe85e0919804e 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -84,6 +84,7 @@ macro_rules! impl_distill_unit_from_fields { } pub(super) use impl_distill_unit_from_fields; +#[auto_impl::auto_impl(&)] pub trait GenericPlanRef: Eq + Hash { fn id(&self) -> PlanNodeId; fn schema(&self) -> &Schema; @@ -92,6 +93,7 @@ pub trait GenericPlanRef: Eq + Hash { fn ctx(&self) -> OptimizerContextRef; } +#[auto_impl::auto_impl(&)] pub trait PhysicalPlanRef: GenericPlanRef { fn distribution(&self) -> &Distribution; } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 0ad9b828ead4b..0aed32abec40e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -22,7 +22,7 @@ use risingwave_expr::aggregate::{agg_kinds, AggKind}; use super::generic::{self, Agg, GenericPlanRef, PlanAggCall, ProjectBuilder}; use super::utils::impl_distill_by_unit; use super::{ - BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, PlanBase, PlanRef, + BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHashAgg, StreamProject, StreamSimpleAgg, StreamStatelessSimpleAgg, ToBatch, ToStream, }; @@ -48,7 +48,7 @@ use crate::utils::{ /// The output schema will first include the group key and then the aggregation calls. #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub struct LogicalAgg { - pub base: PlanBase, + pub base: PlanBase, core: Agg, } @@ -182,6 +182,8 @@ impl LogicalAgg { /// Generates distributed stream plan. fn gen_dist_stream_agg_plan(&self, stream_input: PlanRef) -> Result { + use super::stream::prelude::*; + let input_dist = stream_input.distribution(); debug_assert!(*input_dist != Distribution::Broadcast); @@ -1137,6 +1139,8 @@ fn new_stream_hash_agg(core: Agg, vnode_col_idx: Option) -> Stre impl ToStream for LogicalAgg { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { + use super::stream::prelude::*; + for agg_call in self.agg_calls() { if matches!(agg_call.agg_kind, agg_kinds::unimplemented_in_stream!()) { return Err(ErrorCode::NotImplemented( diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index b398ce7494f61..fa5576c61710b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -11,8 +11,8 @@ // 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 pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result, RwError}; @@ -23,7 +23,7 @@ use super::generic::{ }; use super::utils::{childless_record, Distill}; use super::{ - ColPrunable, LogicalJoin, LogicalProject, PlanBase, PlanRef, PlanTreeNodeBinary, + ColPrunable, Logical, LogicalJoin, LogicalProject, PlanBase, PlanRef, PlanTreeNodeBinary, PredicatePushdown, ToBatch, ToStream, }; use crate::expr::{CorrelatedId, Expr, ExprImpl, ExprRewriter, InputRef}; @@ -38,7 +38,7 @@ use crate::utils::{ColIndexMapping, Condition, ConditionDisplay}; /// left side. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalApply { - pub base: PlanBase, + pub base: PlanBase, left: PlanRef, right: PlanRef, on: Condition, diff --git a/src/frontend/src/optimizer/plan_node/logical_dedup.rs b/src/frontend/src/optimizer/plan_node/logical_dedup.rs index dd46f9af9be1d..7be8b64ae01af 100644 --- a/src/frontend/src/optimizer/plan_node/logical_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/logical_dedup.rs @@ -21,9 +21,9 @@ use super::generic::TopNLimit; use super::utils::impl_distill_by_unit; use super::{ gen_filter_and_pushdown, generic, BatchGroupTopN, ColPrunable, ColumnPruningContext, - ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, - PredicatePushdownContext, RewriteStreamContext, StreamDedup, StreamGroupTopN, ToBatch, - ToStream, ToStreamContext, + ExprRewritable, Logical, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, + PredicatePushdown, PredicatePushdownContext, RewriteStreamContext, StreamDedup, + StreamGroupTopN, ToBatch, ToStream, ToStreamContext, }; use crate::optimizer::property::{Order, RequiredDist}; use crate::utils::Condition; @@ -32,7 +32,7 @@ use crate::utils::Condition; /// an `ORDER BY`. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalDedup { - pub base: PlanBase, + pub base: PlanBase, core: generic::Dedup, } @@ -99,6 +99,8 @@ impl ToStream for LogicalDedup { } fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { + use super::stream::prelude::*; + let input = self.input().to_stream(ctx)?; let input = RequiredDist::hash_shard(self.dedup_cols()) .enforce_if_not_satisfies(input, &Order::any())?; diff --git a/src/frontend/src/optimizer/plan_node/logical_delete.rs b/src/frontend/src/optimizer/plan_node/logical_delete.rs index 2cda2b782af7e..63f8c81b5991d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_delete.rs +++ b/src/frontend/src/optimizer/plan_node/logical_delete.rs @@ -17,8 +17,8 @@ use risingwave_common::error::Result; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchDelete, ColPrunable, ExprRewritable, LogicalProject, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchDelete, ColPrunable, ExprRewritable, Logical, + LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; use crate::optimizer::plan_node::{ @@ -31,7 +31,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// It corresponds to the `DELETE` statements in SQL. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalDelete { - pub base: PlanBase, + pub base: PlanBase, core: generic::Delete, } diff --git a/src/frontend/src/optimizer/plan_node/logical_except.rs b/src/frontend/src/optimizer/plan_node/logical_except.rs index 11ff8b0210ee6..353cfe6583d5d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_except.rs +++ b/src/frontend/src/optimizer/plan_node/logical_except.rs @@ -17,7 +17,9 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::Result; use super::utils::impl_distill_by_unit; -use super::{ColPrunable, ExprRewritable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream}; +use super::{ + ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, +}; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ generic, ColumnPruningContext, PlanTreeNode, PredicatePushdownContext, RewriteStreamContext, @@ -29,7 +31,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// matching rows from its other inputs. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalExcept { - pub base: PlanBase, + pub base: PlanBase, core: generic::Except, } diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index d1f3b666feef5..31209122e16a3 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -18,8 +18,8 @@ use risingwave_common::error::Result; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchExpand, ColPrunable, ExprRewritable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamExpand, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchExpand, ColPrunable, ExprRewritable, Logical, PlanBase, + PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamExpand, ToBatch, ToStream, }; use crate::optimizer::plan_node::{ ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, @@ -37,7 +37,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// is used to distinguish between different `subset`s in `column_subsets`. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalExpand { - pub base: PlanBase, + pub base: PlanBase, core: generic::Expand, } diff --git a/src/frontend/src/optimizer/plan_node/logical_filter.rs b/src/frontend/src/optimizer/plan_node/logical_filter.rs index a62b91aac5277..a9ebd26d38219 100644 --- a/src/frontend/src/optimizer/plan_node/logical_filter.rs +++ b/src/frontend/src/optimizer/plan_node/logical_filter.rs @@ -21,8 +21,8 @@ use risingwave_common::types::DataType; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; use super::{ - generic, ColPrunable, ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, + generic, ColPrunable, ExprRewritable, Logical, LogicalProject, PlanBase, PlanRef, + PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::expr::{assert_input_ref, ExprImpl, ExprRewriter, ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ @@ -37,7 +37,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// If the condition allows nulls, then a null value is treated the same as false. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::Filter, } diff --git a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs index da2ec2138c3d1..a592337f7e26d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -20,8 +20,9 @@ use risingwave_common::types::Interval; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchHopWindow, ColPrunable, ExprRewritable, LogicalFilter, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchHopWindow, ColPrunable, ExprRewritable, Logical, + LogicalFilter, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, + ToBatch, ToStream, }; use crate::expr::{ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ @@ -32,7 +33,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition}; /// `LogicalHopWindow` implements Hop Table Function. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalHopWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::HopWindow, } diff --git a/src/frontend/src/optimizer/plan_node/logical_insert.rs b/src/frontend/src/optimizer/plan_node/logical_insert.rs index e93b77d79c1f2..f801affcc93f9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_insert.rs +++ b/src/frontend/src/optimizer/plan_node/logical_insert.rs @@ -19,8 +19,8 @@ use risingwave_common::error::Result; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{ - gen_filter_and_pushdown, generic, BatchInsert, ColPrunable, ExprRewritable, LogicalProject, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchInsert, ColPrunable, ExprRewritable, Logical, + LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; use crate::expr::{ExprImpl, ExprRewriter}; @@ -35,7 +35,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// statements, the input relation would be [`super::LogicalValues`]. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalInsert { - pub base: PlanBase, + pub base: PlanBase, core: generic::Insert, } diff --git a/src/frontend/src/optimizer/plan_node/logical_intersect.rs b/src/frontend/src/optimizer/plan_node/logical_intersect.rs index 2da89e01d8319..6e20cf87c6927 100644 --- a/src/frontend/src/optimizer/plan_node/logical_intersect.rs +++ b/src/frontend/src/optimizer/plan_node/logical_intersect.rs @@ -17,7 +17,9 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::Result; use super::utils::impl_distill_by_unit; -use super::{ColPrunable, ExprRewritable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream}; +use super::{ + ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, +}; use crate::optimizer::plan_node::{ generic, ColumnPruningContext, PlanTreeNode, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, @@ -28,7 +30,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// If `all` is false, it needs to eliminate duplicates. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalIntersect { - pub base: PlanBase, + pub base: PlanBase, core: generic::Intersect, } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index a586af2f0bf42..a928481230d3c 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -26,8 +26,8 @@ use super::generic::{ }; use super::utils::{childless_record, Distill}; use super::{ - generic, ColPrunable, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, PredicatePushdown, - StreamHashJoin, StreamProject, ToBatch, ToStream, + generic, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeBinary, + PredicatePushdown, StreamHashJoin, StreamProject, ToBatch, ToStream, }; use crate::expr::{CollectInputRef, Expr, ExprImpl, ExprRewriter, ExprType, InputRef}; use crate::optimizer::plan_node::generic::DynamicFilter; @@ -49,7 +49,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition, Condit /// right columns, dependent on the output indices provided. A repeat output index is illegal. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, } @@ -866,6 +866,8 @@ impl LogicalJoin { predicate: EqJoinPredicate, ctx: &mut ToStreamContext, ) -> Result { + use super::stream::prelude::*; + assert!(predicate.has_eq()); let mut right = self.right().to_stream_with_dist_required( &RequiredDist::shard_by_key(self.right().schema().len(), &predicate.right_eq_indexes()), @@ -1009,6 +1011,8 @@ impl LogicalJoin { predicate: EqJoinPredicate, ctx: &mut ToStreamContext, ) -> Result { + use super::stream::prelude::*; + assert!(predicate.has_eq()); let right = self.right(); @@ -1179,6 +1183,8 @@ impl LogicalJoin { predicate: Condition, ctx: &mut ToStreamContext, ) -> Result> { + use super::stream::prelude::*; + // If there is exactly one predicate, it is a comparison (<, <=, >, >=), and the // join is a `Inner` or `LeftSemi` join, we can convert the scalar subquery into a // `StreamDynamicFilter` diff --git a/src/frontend/src/optimizer/plan_node/logical_limit.rs b/src/frontend/src/optimizer/plan_node/logical_limit.rs index ea53c4a2d546c..f6678faf396a1 100644 --- a/src/frontend/src/optimizer/plan_node/logical_limit.rs +++ b/src/frontend/src/optimizer/plan_node/logical_limit.rs @@ -17,8 +17,8 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchLimit, ColPrunable, ExprRewritable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchLimit, ColPrunable, ExprRewritable, Logical, PlanBase, + PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::optimizer::plan_node::{ ColumnPruningContext, LogicalTopN, PredicatePushdownContext, RewriteStreamContext, @@ -30,7 +30,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// `LogicalLimit` fetches up to `limit` rows from `offset` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalLimit { - pub base: PlanBase, + pub base: PlanBase, pub(super) core: generic::Limit, } diff --git a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs index 9b740abd7718e..819f84e963cfd 100644 --- a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs @@ -23,7 +23,7 @@ use risingwave_pb::plan_common::JoinType; use super::utils::{childless_record, Distill}; use super::{ - ColPrunable, ExprRewritable, LogicalFilter, LogicalJoin, LogicalProject, PlanBase, + ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalJoin, LogicalProject, PlanBase, PlanNodeType, PlanRef, PlanTreeNodeBinary, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; @@ -46,7 +46,7 @@ use crate::utils::{ /// expressed as 2-way `LogicalJoin`s. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalMultiJoin { - pub base: PlanBase, + pub base: PlanBase, inputs: Vec, on: Condition, output_indices: Vec, diff --git a/src/frontend/src/optimizer/plan_node/logical_now.rs b/src/frontend/src/optimizer/plan_node/logical_now.rs index 1d720db15b71a..c13a0c93a6e70 100644 --- a/src/frontend/src/optimizer/plan_node/logical_now.rs +++ b/src/frontend/src/optimizer/plan_node/logical_now.rs @@ -21,7 +21,7 @@ use risingwave_common::types::DataType; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{ - ColPrunable, ColumnPruningContext, ExprRewritable, LogicalFilter, PlanBase, PlanRef, + ColPrunable, ColumnPruningContext, ExprRewritable, Logical, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, RewriteStreamContext, StreamNow, ToBatch, ToStream, ToStreamContext, }; use crate::optimizer::plan_node::utils::column_names_pretty; @@ -31,7 +31,7 @@ use crate::OptimizerContextRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalNow { - pub base: PlanBase, + pub base: PlanBase, } impl LogicalNow { diff --git a/src/frontend/src/optimizer/plan_node/logical_over_window.rs b/src/frontend/src/optimizer/plan_node/logical_over_window.rs index b1796ddc62752..a78a145ab1997 100644 --- a/src/frontend/src/optimizer/plan_node/logical_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_over_window.rs @@ -23,7 +23,7 @@ use risingwave_expr::window_function::{Frame, FrameBound, WindowFuncKind}; use super::generic::{GenericPlanRef, OverWindow, PlanWindowFunction, ProjectBuilder}; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, BatchOverWindow, ColPrunable, ExprRewritable, LogicalProject, + gen_filter_and_pushdown, BatchOverWindow, ColPrunable, ExprRewritable, Logical, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamEowcOverWindow, StreamEowcSort, StreamOverWindow, ToBatch, ToStream, }; @@ -358,7 +358,7 @@ impl<'a> ExprVisitor for OverWindowProjectBuilder<'a> { /// The output schema is the input schema plus the window functions. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: OverWindow, } @@ -772,6 +772,8 @@ impl ToBatch for LogicalOverWindow { impl ToStream for LogicalOverWindow { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { + use super::stream::prelude::*; + if self .window_functions() .iter() diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index a96de7d91ecd5..59a5509ebcd70 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -19,8 +19,8 @@ use risingwave_common::error::Result; use super::utils::{childless_record, Distill}; use super::{ - gen_filter_and_pushdown, generic, BatchProject, ColPrunable, ExprRewritable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamProject, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchProject, ColPrunable, ExprRewritable, Logical, PlanBase, + PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProject, ToBatch, ToStream, }; use crate::expr::{collect_input_refs, ExprImpl, ExprRewriter, InputRef}; use crate::optimizer::plan_node::generic::GenericPlanRef; @@ -33,7 +33,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition, Substi /// `LogicalProject` computes a set of expressions from its input relation. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalProject { - pub base: PlanBase, + pub base: PlanBase, core: generic::Project, } diff --git a/src/frontend/src/optimizer/plan_node/logical_project_set.rs b/src/frontend/src/optimizer/plan_node/logical_project_set.rs index 4bf6b18cdabe3..fc894713c2f43 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project_set.rs @@ -19,8 +19,9 @@ use risingwave_common::types::DataType; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchProjectSet, ColPrunable, ExprRewritable, LogicalProject, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchProjectSet, ColPrunable, ExprRewritable, Logical, + LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, + ToBatch, ToStream, }; use crate::expr::{ collect_input_refs, Expr, ExprImpl, ExprRewriter, FunctionCall, InputRef, TableFunction, @@ -41,7 +42,7 @@ use crate::utils::{ColIndexMapping, Condition, Substitute}; /// column. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalProjectSet { - pub base: PlanBase, + pub base: PlanBase, core: generic::ProjectSet, } diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 07d2a6c7653e7..200302ae70f97 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -25,7 +25,7 @@ use risingwave_common::util::sort_util::ColumnOrder; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::{ - generic, BatchFilter, BatchProject, ColPrunable, ExprRewritable, PlanBase, PlanRef, + generic, BatchFilter, BatchProject, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, StreamTableScan, ToBatch, ToStream, }; use crate::catalog::{ColumnId, IndexCatalog}; @@ -42,7 +42,7 @@ use crate::utils::{ColIndexMapping, Condition, ConditionDisplay}; /// `LogicalScan` returns contents of a table or other equivalent object #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalScan { - pub base: PlanBase, + pub base: PlanBase, core: generic::Scan, } diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs index d6b5711740a98..3e7193342fb67 100644 --- a/src/frontend/src/optimizer/plan_node/logical_share.rs +++ b/src/frontend/src/optimizer/plan_node/logical_share.rs @@ -20,8 +20,8 @@ use risingwave_common::error::Result; use super::utils::{childless_record, Distill}; use super::{ - generic, ColPrunable, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, - ToBatch, ToStream, + generic, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeUnary, + PredicatePushdown, ToBatch, ToStream, }; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ @@ -49,7 +49,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// ``` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalShare { - pub base: PlanBase, + pub base: PlanBase, core: generic::Share, } @@ -68,7 +68,7 @@ impl LogicalShare { LogicalShare::new(input).into() } - pub(super) fn pretty_fields<'a>(base: &PlanBase, name: &'a str) -> XmlNode<'a> { + pub(super) fn pretty_fields(base: impl GenericPlanRef, name: &str) -> XmlNode<'_> { childless_record(name, vec![("id", Pretty::debug(&base.id().0))]) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 45a5fbcb2240f..01166e74f1359 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -32,9 +32,9 @@ use super::generic::GenericPlanRef; use super::stream_watermark_filter::StreamWatermarkFilter; use super::utils::{childless_record, Distill}; use super::{ - generic, BatchProject, BatchSource, ColPrunable, ExprRewritable, LogicalFilter, LogicalProject, - PlanBase, PlanRef, PredicatePushdown, StreamProject, StreamRowIdGen, StreamSource, ToBatch, - ToStream, + generic, BatchProject, BatchSource, ColPrunable, ExprRewritable, Logical, LogicalFilter, + LogicalProject, PlanBase, PlanRef, PredicatePushdown, StreamProject, StreamRowIdGen, + StreamSource, ToBatch, ToStream, }; use crate::catalog::source_catalog::SourceCatalog; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, InputRef}; @@ -51,7 +51,7 @@ use crate::utils::{ColIndexMapping, Condition, IndexRewriter}; /// `LogicalSource` returns contents of a table or other equivalent object #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalSource { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Source, /// Expressions to output. This field presents and will be turned to a `Project` when diff --git a/src/frontend/src/optimizer/plan_node/logical_table_function.rs b/src/frontend/src/optimizer/plan_node/logical_table_function.rs index 15d510cc1c6fd..4553722dca328 100644 --- a/src/frontend/src/optimizer/plan_node/logical_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/logical_table_function.rs @@ -19,7 +19,7 @@ use risingwave_common::types::DataType; use super::utils::{childless_record, Distill}; use super::{ - ColPrunable, ExprRewritable, LogicalFilter, LogicalProject, PlanBase, PlanRef, + ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalProject, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, }; use crate::expr::{Expr, ExprRewriter, TableFunction}; @@ -35,7 +35,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// If the function returns a struct, it will be flattened into multiple columns. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalTableFunction { - pub base: PlanBase, + pub base: PlanBase, pub table_function: TableFunction, pub with_ordinality: bool, } diff --git a/src/frontend/src/optimizer/plan_node/logical_topn.rs b/src/frontend/src/optimizer/plan_node/logical_topn.rs index 39d97a56fe3a6..940714d7d4abb 100644 --- a/src/frontend/src/optimizer/plan_node/logical_topn.rs +++ b/src/frontend/src/optimizer/plan_node/logical_topn.rs @@ -20,9 +20,9 @@ use risingwave_common::util::sort_util::ColumnOrder; use super::generic::TopNLimit; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchGroupTopN, ColPrunable, ExprRewritable, PlanBase, - PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamGroupTopN, StreamProject, ToBatch, - ToStream, + gen_filter_and_pushdown, generic, BatchGroupTopN, ColPrunable, ExprRewritable, Logical, + PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamGroupTopN, StreamProject, + ToBatch, ToStream, }; use crate::expr::{ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ @@ -36,7 +36,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition}; /// `LogicalTopN` sorts the input data and fetches up to `limit` rows from `offset` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } @@ -107,6 +107,8 @@ impl LogicalTopN { } fn gen_dist_stream_top_n_plan(&self, stream_input: PlanRef) -> Result { + use super::stream::prelude::*; + let input_dist = stream_input.distribution().clone(); // if it is append only, for now we don't generate 2-phase rules diff --git a/src/frontend/src/optimizer/plan_node/logical_union.rs b/src/frontend/src/optimizer/plan_node/logical_union.rs index 1f02b026c0020..e108707e0b13a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_union.rs +++ b/src/frontend/src/optimizer/plan_node/logical_union.rs @@ -21,7 +21,9 @@ use risingwave_common::error::Result; use risingwave_common::types::{DataType, Scalar}; use super::utils::impl_distill_by_unit; -use super::{ColPrunable, ExprRewritable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream}; +use super::{ + ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, +}; use crate::expr::{ExprImpl, InputRef, Literal}; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::stream_union::StreamUnion; @@ -37,7 +39,7 @@ use crate::Explain; /// If `all` is false, it needs to eliminate duplicates. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalUnion { - pub base: PlanBase, + pub base: PlanBase, core: generic::Union, } diff --git a/src/frontend/src/optimizer/plan_node/logical_update.rs b/src/frontend/src/optimizer/plan_node/logical_update.rs index 1dbe1d3d3c5c9..34c0939f13bc9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_update.rs +++ b/src/frontend/src/optimizer/plan_node/logical_update.rs @@ -18,8 +18,8 @@ use risingwave_common::error::Result; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchUpdate, ColPrunable, ExprRewritable, LogicalProject, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchUpdate, ColPrunable, ExprRewritable, Logical, + LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; use crate::expr::{ExprImpl, ExprRewriter}; @@ -34,7 +34,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// It corresponds to the `UPDATE` statements in SQL. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalUpdate { - pub base: PlanBase, + pub base: PlanBase, core: generic::Update, } diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index e62c6400f2015..38867b3d9c223 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -24,8 +24,8 @@ use risingwave_common::types::{DataType, ScalarImpl}; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{ - BatchValues, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, - StreamValues, ToBatch, ToStream, + BatchValues, ColPrunable, ExprRewritable, Logical, LogicalFilter, PlanBase, PlanRef, + PredicatePushdown, StreamValues, ToBatch, ToStream, }; use crate::expr::{Expr, ExprImpl, ExprRewriter, Literal}; use crate::optimizer::optimizer_context::OptimizerContextRef; @@ -38,7 +38,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// `LogicalValues` builds rows according to a list of expressions #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalValues { - pub base: PlanBase, + pub base: PlanBase, rows: Arc<[Vec]>, } diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index f16ebfb0c792c..ec7777e42e737 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -51,11 +51,95 @@ use self::stream::StreamPlanRef; use self::utils::Distill; use super::property::{Distribution, FunctionalDependencySet, Order}; +/// A marker trait for different conventions, used for enforcing type safety. +/// +/// Implementors are [`Logical`], [`Batch`], and [`Stream`]. +pub trait ConventionMarker: 'static + Sized { + /// The extra fields in the [`PlanBase`] of this convention. + type Extra: 'static + Eq + Hash + Clone + Debug; + + /// Get the [`Convention`] enum value. + fn value() -> Convention; +} + +/// The marker for logical convention. +pub struct Logical; +impl ConventionMarker for Logical { + type Extra = plan_base::NoExtra; + + fn value() -> Convention { + Convention::Logical + } +} + +/// The marker for batch convention. +pub struct Batch; +impl ConventionMarker for Batch { + type Extra = plan_base::BatchExtra; + + fn value() -> Convention { + Convention::Batch + } +} + +/// The marker for stream convention. +pub struct Stream; +impl ConventionMarker for Stream { + type Extra = plan_base::StreamExtra; + + fn value() -> Convention { + Convention::Stream + } +} + +/// The trait for accessing the meta data and [`PlanBase`] for plan nodes. pub trait PlanNodeMeta { - fn node_type(&self) -> PlanNodeType; - fn plan_base(&self) -> &PlanBase; - fn convention(&self) -> Convention; + type Convention: ConventionMarker; + + const NODE_TYPE: PlanNodeType; + + /// Get the reference to the [`PlanBase`] with corresponding convention. + fn plan_base(&self) -> &PlanBase; + + /// Get the reference to the [`PlanBase`] with erased convention. + /// + /// This is mainly used for implementing [`AnyPlanNodeMeta`]. Callers should prefer + /// [`PlanNodeMeta::plan_base`] instead as it is more type-safe. + fn plan_base_ref(&self) -> PlanBaseRef<'_>; +} + +// Intentionally made private. +mod plan_node_meta { + use super::*; + + /// The object-safe version of [`PlanNodeMeta`], used as a super trait of [`PlanNode`]. + /// + /// Check [`PlanNodeMeta`] for more details. + pub trait AnyPlanNodeMeta { + fn node_type(&self) -> PlanNodeType; + fn plan_base(&self) -> PlanBaseRef<'_>; + fn convention(&self) -> Convention; + } + + /// Implement [`AnyPlanNodeMeta`] for all [`PlanNodeMeta`]. + impl

AnyPlanNodeMeta for P + where + P: PlanNodeMeta, + { + fn node_type(&self) -> PlanNodeType { + P::NODE_TYPE + } + + fn plan_base(&self) -> PlanBaseRef<'_> { + PlanNodeMeta::plan_base_ref(self) + } + + fn convention(&self) -> Convention { + P::Convention::value() + } + } } +use plan_node_meta::AnyPlanNodeMeta; /// The common trait over all plan nodes. Used by optimizer framework which will treat all node as /// `dyn PlanNode` @@ -77,7 +161,7 @@ pub trait PlanNode: + ToPb + ToLocalBatch + PredicatePushdown - + PlanNodeMeta + + AnyPlanNodeMeta { } @@ -194,7 +278,7 @@ pub trait VisitPlan: Visit { } } -#[derive(Debug, PartialEq)] +#[derive(Clone, Copy, Debug, PartialEq)] pub enum Convention { Logical, Batch, @@ -419,12 +503,13 @@ impl PlanTreeNode for PlanRef { } } -impl PlanNodeMeta for PlanRef { +/// Implement again for the `dyn` newtype wrapper. +impl AnyPlanNodeMeta for PlanRef { fn node_type(&self) -> PlanNodeType { self.0.node_type() } - fn plan_base(&self) -> &PlanBase { + fn plan_base(&self) -> PlanBaseRef<'_> { self.0.plan_base() } @@ -433,11 +518,9 @@ impl PlanNodeMeta for PlanRef { } } -/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. -impl

GenericPlanRef for P -where - P: PlanNodeMeta + Eq + Hash, -{ +/// Allow access to all fields defined in [`GenericPlanRef`] for the type-erased plan node. +// TODO: may also implement on `dyn PlanNode` directly. +impl GenericPlanRef for PlanRef { fn id(&self) -> PlanNodeId { self.plan_base().id() } @@ -459,23 +542,17 @@ where } } -/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. -// TODO: further constrain the convention to be `Stream` or `Batch`. -impl

PhysicalPlanRef for P -where - P: PlanNodeMeta + Eq + Hash, -{ +/// Allow access to all fields defined in [`PhysicalPlanRef`] for the type-erased plan node. +// TODO: may also implement on `dyn PlanNode` directly. +impl PhysicalPlanRef for PlanRef { fn distribution(&self) -> &Distribution { self.plan_base().distribution() } } -/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. -// TODO: further constrain the convention to be `Stream`. -impl

StreamPlanRef for P -where - P: PlanNodeMeta + Eq + Hash, -{ +/// Allow access to all fields defined in [`StreamPlanRef`] for the type-erased plan node. +// TODO: may also implement on `dyn PlanNode` directly. +impl StreamPlanRef for PlanRef { fn append_only(&self) -> bool { self.plan_base().append_only() } @@ -489,12 +566,9 @@ where } } -/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. -// TODO: further constrain the convention to be `Batch`. -impl

BatchPlanRef for P -where - P: PlanNodeMeta + Eq + Hash, -{ +/// Allow access to all fields defined in [`BatchPlanRef`] for the type-erased plan node. +// TODO: may also implement on `dyn PlanNode` directly. +impl BatchPlanRef for PlanRef { fn order(&self) -> &Order { self.plan_base().order() } @@ -553,6 +627,8 @@ pub(crate) fn pretty_config() -> PrettyConfig { } } +/// Directly implement methods for [`PlanNode`] to access the fields defined in [`GenericPlanRef`]. +// TODO: always require `GenericPlanRef` to make it more consistent. impl dyn PlanNode { pub fn id(&self) -> PlanNodeId { self.plan_base().id() @@ -570,36 +646,19 @@ impl dyn PlanNode { self.plan_base().stream_key() } - pub fn order(&self) -> &Order { - self.plan_base().order() - } - - // TODO: avoid no manual delegation - pub fn distribution(&self) -> &Distribution { - self.plan_base().distribution() - } - - pub fn append_only(&self) -> bool { - self.plan_base().append_only() - } - - pub fn emit_on_window_close(&self) -> bool { - self.plan_base().emit_on_window_close() - } - pub fn functional_dependency(&self) -> &FunctionalDependencySet { self.plan_base().functional_dependency() } +} - pub fn watermark_columns(&self) -> &FixedBitSet { - self.plan_base().watermark_columns() - } - +impl dyn PlanNode { /// Serialize the plan node and its children to a stream plan proto. /// /// Note that [`StreamTableScan`] has its own implementation of `to_stream_prost`. We have a /// hook inside to do some ad-hoc thing for [`StreamTableScan`]. pub fn to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> StreamPlanPb { + use stream::prelude::*; + if let Some(stream_table_scan) = self.as_stream_table_scan() { return stream_table_scan.adhoc_to_stream_prost(state); } @@ -626,7 +685,7 @@ impl dyn PlanNode { .map(|x| *x as u32) .collect(), fields: self.schema().to_prost(), - append_only: self.append_only(), + append_only: self.plan_base().append_only(), } } @@ -1079,14 +1138,23 @@ macro_rules! impl_plan_node_meta { } $(impl PlanNodeMeta for [<$convention $name>] { - fn node_type(&self) -> PlanNodeType{ - PlanNodeType::[<$convention $name>] - } - fn plan_base(&self) -> &PlanBase { + type Convention = $convention; + const NODE_TYPE: PlanNodeType = PlanNodeType::[<$convention $name>]; + + fn plan_base(&self) -> &PlanBase<$convention> { &self.base } - fn convention(&self) -> Convention { - Convention::$convention + + fn plan_base_ref(&self) -> PlanBaseRef<'_> { + PlanBaseRef::$convention(&self.base) + } + } + + impl Deref for [<$convention $name>] { + type Target = PlanBase<$convention>; + + fn deref(&self) -> &Self::Target { + &self.base } })* } diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index 51b1aa5f41141..0d2e649379112 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -21,17 +21,36 @@ use super::*; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::property::{Distribution, FunctionalDependencySet, Order}; -/// Common extra fields for physical plan nodes. +/// No extra fields for logical plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] -struct PhysicalCommonExtra { - /// The distribution property of the PlanNode's output, store an `Distribution::any()` here - /// will not affect correctness, but insert unnecessary exchange in plan - dist: Distribution, +pub struct NoExtra; + +// Make them public types in a private module to allow using them as public trait bounds, +// while still keeping them private to the super module. +mod physical_common { + use super::*; + + /// Common extra fields for physical plan nodes. + #[derive(Clone, Debug, PartialEq, Eq, Hash)] + pub struct PhysicalCommonExtra { + /// The distribution property of the PlanNode's output, store an `Distribution::any()` here + /// will not affect correctness, but insert unnecessary exchange in plan + pub dist: Distribution, + } + + /// A helper trait to reuse code for accessing the common physical fields of batch and stream + /// plan bases. + pub trait GetPhysicalCommon { + fn physical(&self) -> &PhysicalCommonExtra; + fn physical_mut(&mut self) -> &mut PhysicalCommonExtra; + } } +use physical_common::*; + /// Extra fields for stream plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] -struct StreamExtra { +pub struct StreamExtra { /// Common fields for physical plan nodes. physical: PhysicalCommonExtra, @@ -45,9 +64,19 @@ struct StreamExtra { watermark_columns: FixedBitSet, } +impl GetPhysicalCommon for StreamExtra { + fn physical(&self) -> &PhysicalCommonExtra { + &self.physical + } + + fn physical_mut(&mut self) -> &mut PhysicalCommonExtra { + &mut self.physical + } +} + /// Extra fields for batch plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] -struct BatchExtra { +pub struct BatchExtra { /// Common fields for physical plan nodes. physical: PhysicalCommonExtra, @@ -56,55 +85,31 @@ struct BatchExtra { order: Order, } -/// Extra fields for physical plan nodes. -#[derive(Clone, Debug, PartialEq, Eq, Hash)] -enum PhysicalExtra { - Stream(StreamExtra), - Batch(BatchExtra), -} - -impl PhysicalExtra { - fn common(&self) -> &PhysicalCommonExtra { - match self { - PhysicalExtra::Stream(stream) => &stream.physical, - PhysicalExtra::Batch(batch) => &batch.physical, - } - } - - fn common_mut(&mut self) -> &mut PhysicalCommonExtra { - match self { - PhysicalExtra::Stream(stream) => &mut stream.physical, - PhysicalExtra::Batch(batch) => &mut batch.physical, - } - } - - fn stream(&self) -> &StreamExtra { - match self { - PhysicalExtra::Stream(extra) => extra, - _ => panic!("access stream properties from batch plan node"), - } +impl GetPhysicalCommon for BatchExtra { + fn physical(&self) -> &PhysicalCommonExtra { + &self.physical } - fn batch(&self) -> &BatchExtra { - match self { - PhysicalExtra::Batch(extra) => extra, - _ => panic!("access batch properties from stream plan node"), - } + fn physical_mut(&mut self) -> &mut PhysicalCommonExtra { + &mut self.physical } } -/// the common fields of all nodes, please make a field named `base` in -/// every planNode and correctly value it when construct the planNode. +/// The common fields of all plan nodes with different conventions. +/// +/// Please make a field named `base` in every planNode and correctly value +/// it when construct the planNode. /// /// All fields are intentionally made private and immutable, as they should /// normally be the same as the given [`GenericPlanNode`] when constructing. /// /// - To access them, use traits including [`GenericPlanRef`], -/// [`PhysicalPlanRef`], [`StreamPlanRef`] and [`BatchPlanRef`]. +/// [`PhysicalPlanRef`], [`StreamPlanRef`] and [`BatchPlanRef`] with +/// compile-time checks. /// - To mutate them, use methods like `new_*` or `clone_with_*`. -#[derive(Clone, Debug, Educe)] -#[educe(PartialEq, Eq, Hash)] -pub struct PlanBase { +#[derive(Educe)] +#[educe(PartialEq, Eq, Hash, Clone, Debug)] +pub struct PlanBase { // -- common fields -- #[educe(PartialEq(ignore), Hash(ignore))] id: PlanNodeId, @@ -113,29 +118,17 @@ pub struct PlanBase { schema: Schema, /// the pk indices of the PlanNode's output, a empty stream key vec means there is no stream key - // TODO: this is actually a logical and stream only property + // TODO: this is actually a logical and stream only property. + // - For logical nodes, this is `None` in most time expect for the phase after `logical_rewrite_for_stream`. + // - For stream nodes, this is always `Some`. stream_key: Option>, functional_dependency: FunctionalDependencySet, - /// Extra fields if the plan node is physical. - physical_extra: Option, + /// Extra fields for different conventions. + extra: C::Extra, } -impl PlanBase { - fn physical_extra(&self) -> &PhysicalExtra { - self.physical_extra - .as_ref() - .expect("access physical properties from logical plan node") - } - - fn physical_extra_mut(&mut self) -> &mut PhysicalExtra { - self.physical_extra - .as_mut() - .expect("access physical properties from logical plan node") - } -} - -impl generic::GenericPlanRef for PlanBase { +impl generic::GenericPlanRef for PlanBase { fn id(&self) -> PlanNodeId { self.id } @@ -157,33 +150,44 @@ impl generic::GenericPlanRef for PlanBase { } } -impl generic::PhysicalPlanRef for PlanBase { +impl generic::PhysicalPlanRef for PlanBase +where + C::Extra: GetPhysicalCommon, +{ fn distribution(&self) -> &Distribution { - &self.physical_extra().common().dist + &self.extra.physical().dist } } -impl stream::StreamPlanRef for PlanBase { +impl stream::StreamPlanRef for PlanBase { fn append_only(&self) -> bool { - self.physical_extra().stream().append_only + self.extra.append_only } fn emit_on_window_close(&self) -> bool { - self.physical_extra().stream().emit_on_window_close + self.extra.emit_on_window_close } fn watermark_columns(&self) -> &FixedBitSet { - &self.physical_extra().stream().watermark_columns + &self.extra.watermark_columns } } -impl batch::BatchPlanRef for PlanBase { +impl batch::BatchPlanRef for PlanBase { fn order(&self) -> &Order { - &self.physical_extra().batch().order + &self.extra.order + } +} + +impl PlanBase { + pub fn clone_with_new_plan_id(&self) -> Self { + let mut new = self.clone(); + new.id = self.ctx().next_plan_node_id(); + new } } -impl PlanBase { +impl PlanBase { pub fn new_logical( ctx: OptimizerContextRef, schema: Schema, @@ -197,7 +201,7 @@ impl PlanBase { schema, stream_key, functional_dependency, - physical_extra: None, + extra: NoExtra, } } @@ -209,7 +213,9 @@ impl PlanBase { core.functional_dependency(), ) } +} +impl PlanBase { pub fn new_stream( ctx: OptimizerContextRef, schema: Schema, @@ -228,14 +234,12 @@ impl PlanBase { schema, stream_key, functional_dependency, - physical_extra: Some(PhysicalExtra::Stream({ - StreamExtra { - physical: PhysicalCommonExtra { dist }, - append_only, - emit_on_window_close, - watermark_columns, - } - })), + extra: StreamExtra { + physical: PhysicalCommonExtra { dist }, + append_only, + emit_on_window_close, + watermark_columns, + }, } } @@ -257,7 +261,9 @@ impl PlanBase { watermark_columns, ) } +} +impl PlanBase { pub fn new_batch( ctx: OptimizerContextRef, schema: Schema, @@ -272,12 +278,10 @@ impl PlanBase { schema, stream_key: None, functional_dependency, - physical_extra: Some(PhysicalExtra::Batch({ - BatchExtra { - physical: PhysicalCommonExtra { dist }, - order, - } - })), + extra: BatchExtra { + physical: PhysicalCommonExtra { dist }, + order, + }, } } @@ -288,27 +292,149 @@ impl PlanBase { ) -> Self { Self::new_batch(core.ctx(), core.schema(), dist, order) } +} - pub fn clone_with_new_plan_id(&self) -> Self { - let mut new = self.clone(); - new.id = self.ctx().next_plan_node_id(); - new - } - +impl PlanBase +where + C::Extra: GetPhysicalCommon, +{ /// Clone the plan node with a new distribution. /// /// Panics if the plan node is not physical. pub fn clone_with_new_distribution(&self, dist: Distribution) -> Self { let mut new = self.clone(); - new.physical_extra_mut().common_mut().dist = dist; + new.extra.physical_mut().dist = dist; new } } // Mutators for testing only. #[cfg(test)] -impl PlanBase { +impl PlanBase { pub fn functional_dependency_mut(&mut self) -> &mut FunctionalDependencySet { &mut self.functional_dependency } } + +/// Reference to [`PlanBase`] with erased conventions. +/// +/// Used for accessing fields on a type-erased plan node. All traits of [`GenericPlanRef`], +/// [`PhysicalPlanRef`], [`StreamPlanRef`] and [`BatchPlanRef`] are implemented for this type, +/// so runtime checks are required when calling methods on it. +#[derive(PartialEq, Eq, Hash, Clone, Copy, Debug, enum_as_inner::EnumAsInner)] +pub enum PlanBaseRef<'a> { + Logical(&'a PlanBase), + Stream(&'a PlanBase), + Batch(&'a PlanBase), +} + +impl PlanBaseRef<'_> { + /// Get the convention of this plan base. + pub fn convention(self) -> Convention { + match self { + PlanBaseRef::Logical(_) => Convention::Logical, + PlanBaseRef::Stream(_) => Convention::Stream, + PlanBaseRef::Batch(_) => Convention::Batch, + } + } +} + +/// Dispatch a method call to the corresponding plan base type. +macro_rules! dispatch_plan_base { + ($self:ident, [$($convention:ident),+ $(,)?], $method:expr) => { + match $self { + $( + PlanBaseRef::$convention(plan) => $method(plan), + )+ + + #[allow(unreachable_patterns)] + _ => unreachable!("calling `{}` on a plan node of `{:?}`", stringify!($method), $self.convention()), + } + } +} + +/// Workaround for getters returning references. +/// +/// For example, callers writing `GenericPlanRef::schema(&foo.plan_base())` will lead to a +/// borrow checker error, as it borrows [`PlanBaseRef`] again, which is already a reference. +/// +/// As a workaround, we directly let the getters below take the ownership of [`PlanBaseRef`], +/// which is `Copy`. When callers write `foo.plan_base().schema()`, the compiler will prefer +/// these ones over the ones defined in traits like [`GenericPlanRef`]. +impl<'a> PlanBaseRef<'a> { + pub(super) fn schema(self) -> &'a Schema { + dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::schema) + } + + pub(super) fn stream_key(self) -> Option<&'a [usize]> { + dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::stream_key) + } + + pub(super) fn functional_dependency(self) -> &'a FunctionalDependencySet { + dispatch_plan_base!( + self, + [Logical, Stream, Batch], + GenericPlanRef::functional_dependency + ) + } + + pub(super) fn distribution(self) -> &'a Distribution { + dispatch_plan_base!(self, [Stream, Batch], PhysicalPlanRef::distribution) + } + + pub(super) fn watermark_columns(self) -> &'a FixedBitSet { + dispatch_plan_base!(self, [Stream], StreamPlanRef::watermark_columns) + } + + pub(super) fn order(self) -> &'a Order { + dispatch_plan_base!(self, [Batch], BatchPlanRef::order) + } +} + +impl GenericPlanRef for PlanBaseRef<'_> { + fn id(&self) -> PlanNodeId { + dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::id) + } + + fn schema(&self) -> &Schema { + (*self).schema() + } + + fn stream_key(&self) -> Option<&[usize]> { + (*self).stream_key() + } + + fn functional_dependency(&self) -> &FunctionalDependencySet { + (*self).functional_dependency() + } + + fn ctx(&self) -> OptimizerContextRef { + dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::ctx) + } +} + +impl PhysicalPlanRef for PlanBaseRef<'_> { + fn distribution(&self) -> &Distribution { + (*self).distribution() + } +} + +impl StreamPlanRef for PlanBaseRef<'_> { + fn append_only(&self) -> bool { + dispatch_plan_base!(self, [Stream], StreamPlanRef::append_only) + } + + fn emit_on_window_close(&self) -> bool { + dispatch_plan_base!(self, [Stream], StreamPlanRef::emit_on_window_close) + } + + fn watermark_columns(&self) -> &FixedBitSet { + (*self).watermark_columns() + } +} + +impl BatchPlanRef for PlanBaseRef<'_> { + fn order(&self) -> &Order { + (*self).order() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 866c62c2413a5..394a64b656ad3 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -24,8 +24,16 @@ use super::generic::PhysicalPlanRef; /// [`PhysicalPlanRef`]. /// /// [`GenericPlanRef`]: super::generic::GenericPlanRef +#[auto_impl::auto_impl(&)] pub trait StreamPlanRef: PhysicalPlanRef { fn append_only(&self) -> bool; fn emit_on_window_close(&self) -> bool; fn watermark_columns(&self) -> &FixedBitSet; } + +/// Prelude for stream plan nodes. +pub mod prelude { + pub use super::super::generic::{GenericPlanRef, PhysicalPlanRef}; + pub use super::super::Stream; + pub use super::StreamPlanRef; +} diff --git a/src/frontend/src/optimizer/plan_node/stream_dedup.rs b/src/frontend/src/optimizer/plan_node/stream_dedup.rs index 51b5e589e886e..af1e0cec77e41 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dedup.rs @@ -17,17 +17,17 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::DedupNode; -use super::generic::{self, GenericPlanNode, GenericPlanRef, PhysicalPlanRef}; -use super::stream::StreamPlanRef; +use super::generic::GenericPlanNode; +use super::stream::prelude::*; use super::utils::{impl_distill_by_unit, TableCatalogBuilder}; -use super::{ExprRewritable, PlanBase, PlanTreeNodeUnary, StreamNode}; +use super::{generic, ExprRewritable, PlanBase, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::plan_node::PlanRef; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDedup { - pub base: PlanBase, + pub base: PlanBase, core: generic::Dedup, } diff --git a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs index bb18f9cffdf0f..49a2b97a97454 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -21,6 +21,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ArrangementInfo, DeltaIndexJoinNode}; use super::generic::{self, GenericPlanRef}; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamNode}; use crate::expr::{Expr, ExprRewriter}; @@ -34,7 +35,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// inputs to be indexes. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDeltaJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/stream_dml.rs b/src/frontend/src/optimizer/plan_node/stream_dml.rs index 9b000974786e4..07154463392f2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dml.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dml.rs @@ -17,6 +17,7 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{ColumnDesc, INITIAL_TABLE_VERSION_ID}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -24,7 +25,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDml { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, column_descs: Vec, } diff --git a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs index a4b74f37208e7..0ac631413ca3e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -18,6 +18,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::DynamicFilterNode; use super::generic::{DynamicFilter, GenericPlanRef}; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, column_names_pretty, watermark_pretty, Distill}; use super::{generic, ExprRewritable}; @@ -28,7 +29,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDynamicFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::DynamicFilter, cleaned_by_watermark: bool, } diff --git a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs index d8c5a9635ce59..065fcfebf74b7 100644 --- a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs @@ -19,6 +19,7 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{self, GenericPlanRef, PlanWindowFunction}; +use super::stream::prelude::*; use super::utils::{impl_distill_by_unit, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -26,7 +27,7 @@ use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamEowcOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/stream_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs index 99e6c3c5161a1..d9d28fce298ad 100644 --- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs @@ -16,8 +16,7 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{DispatchStrategy, DispatcherType, ExchangeNode}; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; -use super::stream::StreamPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, plan_node_name, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::{Distribution, DistributionDisplay}; @@ -27,7 +26,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// without changing its content. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamExchange { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, no_shuffle: bool, } diff --git a/src/frontend/src/optimizer/plan_node/stream_expand.rs b/src/frontend/src/optimizer/plan_node/stream_expand.rs index 5959b8d6be4d2..1085d38c06a18 100644 --- a/src/frontend/src/optimizer/plan_node/stream_expand.rs +++ b/src/frontend/src/optimizer/plan_node/stream_expand.rs @@ -17,6 +17,7 @@ use risingwave_pb::stream_plan::expand_node::Subset; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ExpandNode; +use super::stream::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::Distribution; @@ -24,7 +25,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamExpand { - pub base: PlanBase, + pub base: PlanBase, core: generic::Expand, } diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index 0f000e6b8c0db..8bba484a90e29 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -15,6 +15,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::FilterNode; +use super::stream::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::{Expr, ExprImpl, ExprRewriter}; @@ -25,7 +26,7 @@ use crate::utils::Condition; /// `StreamFilter` implements [`super::LogicalFilter`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::Filter, } 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 95fd72e9f6aa0..2fc1aba8127fd 100644 --- a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs +++ b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs @@ -20,9 +20,9 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{PbStreamFsFetch, StreamFsFetchNode}; +use super::stream::prelude::*; use super::{PlanBase, PlanRef, PlanTreeNodeUnary}; use crate::catalog::source_catalog::SourceCatalog; -use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::utils::{childless_record, Distill}; use crate::optimizer::plan_node::{generic, ExprRewritable, StreamNode}; use crate::optimizer::property::Distribution; @@ -30,7 +30,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamFsFetch { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, core: generic::Source, } diff --git a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs index 3e8f3c00206c4..d0c3077f83286 100644 --- a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs @@ -17,6 +17,7 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{DistillUnit, GenericPlanRef, TopNLimit}; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{plan_node_name, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanTreeNodeUnary, StreamNode}; @@ -27,7 +28,7 @@ use crate::PlanRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamGroupTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, /// an optional column index which is the vnode of each row computed by the input's consistent /// hash distribution diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 55ab6b5906e59..6ef8849b1e142 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -19,6 +19,7 @@ use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{self, GenericPlanRef, PlanAggCall}; +use super::stream::prelude::*; use super::utils::{childless_record, plan_node_name, watermark_pretty, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; @@ -28,7 +29,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, IndexSet}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamHashAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, /// An optional column index which is the vnode of each row computed by the input's consistent diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs index 9d9c41425c4b1..36aff15d96055 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -21,6 +21,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{DeltaExpression, HashJoinNode, PbInequalityPair}; use super::generic::{GenericPlanRef, Join}; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, plan_node_name, watermark_pretty, Distill}; use super::{ @@ -38,7 +39,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// get output rows. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamHashJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs index e177be6942360..e2a55ff121949 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -18,6 +18,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::HopWindowNode; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -28,7 +29,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// [`StreamHopWindow`] represents a hop window table function. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamHopWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::HopWindow, window_start_exprs: Vec, window_end_exprs: Vec, diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index d5435e9beb397..fd34c47f7c19a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -24,6 +24,7 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::derive::derive_columns; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, Distill}; use super::{reorganize_elements_id, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -38,7 +39,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// Materializes a stream. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamMaterialize { - pub base: PlanBase, + pub base: PlanBase, /// Child of Materialize plan input: PlanRef, table: TableCatalog, diff --git a/src/frontend/src/optimizer/plan_node/stream_now.rs b/src/frontend/src/optimizer/plan_node/stream_now.rs index 91ebc344fa51d..36f9c3811516c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_now.rs +++ b/src/frontend/src/optimizer/plan_node/stream_now.rs @@ -19,7 +19,7 @@ use risingwave_common::types::DataType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::NowNode; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; +use super::stream::prelude::*; use super::utils::{childless_record, Distill, TableCatalogBuilder}; use super::{ExprRewritable, LogicalNow, PlanBase, StreamNode}; use crate::optimizer::plan_node::utils::column_names_pretty; @@ -29,7 +29,7 @@ use crate::OptimizerContextRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamNow { - pub base: PlanBase, + pub base: PlanBase, } impl StreamNow { diff --git a/src/frontend/src/optimizer/plan_node/stream_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_over_window.rs index 5a2f9d98f1340..89469f63e5740 100644 --- a/src/frontend/src/optimizer/plan_node/stream_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_over_window.rs @@ -19,6 +19,7 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{GenericPlanNode, PlanWindowFunction}; +use super::stream::prelude::*; use super::utils::{impl_distill_by_unit, TableCatalogBuilder}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::plan_node::generic::GenericPlanRef; @@ -27,7 +28,7 @@ use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index c0ff0d1cf2f43..da68260e8930b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -18,6 +18,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ProjectNode; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -29,7 +30,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// rows. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamProject { - pub base: PlanBase, + pub base: PlanBase, core: generic::Project, /// All the watermark derivations, (input_column_index, output_column_index). And the /// derivation expression is the project's expression itself. diff --git a/src/frontend/src/optimizer/plan_node/stream_project_set.rs b/src/frontend/src/optimizer/plan_node/stream_project_set.rs index ba09d79c96c60..5fc922b215975 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project_set.rs @@ -17,6 +17,7 @@ use itertools::Itertools; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ProjectSetNode; +use super::stream::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::{try_derive_watermark, ExprRewriter, WatermarkDerivation}; @@ -25,7 +26,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamProjectSet { - pub base: PlanBase, + pub base: PlanBase, core: generic::ProjectSet, /// All the watermark derivations, (input_column_idx, expr_idx). And the /// derivation expression is the project_set's expression itself. diff --git a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs index 083cb877cd4d6..203bf8951493f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs +++ b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs @@ -15,6 +15,7 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::Distribution; @@ -22,7 +23,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamRowIdGen { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, row_id_index: usize, } diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index 3acf0b132805e..15690c047aa60 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -17,7 +17,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::PbStreamNode; use super::generic::GenericPlanRef; -use super::stream::StreamPlanRef; +use super::stream::prelude::*; use super::utils::Distill; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamExchange, StreamNode}; use crate::optimizer::plan_node::{LogicalShare, PlanBase, PlanTreeNode}; @@ -27,22 +27,25 @@ use crate::Explain; /// `StreamShare` will be translated into an `ExchangeNode` based on its distribution finally. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamShare { - pub base: PlanBase, + pub base: PlanBase, core: generic::Share, } impl StreamShare { pub fn new(core: generic::Share) -> Self { - let input = core.input.borrow().0.clone(); - let dist = input.distribution().clone(); - // Filter executor won't change the append-only behavior of the stream. - let base = PlanBase::new_stream_with_core( - &core, - dist, - input.append_only(), - input.emit_on_window_close(), - input.watermark_columns().clone(), - ); + let base = { + let input = core.input.borrow(); + let dist = input.distribution().clone(); + // Filter executor won't change the append-only behavior of the stream. + PlanBase::new_stream_with_core( + &core, + dist, + input.append_only(), + input.emit_on_window_close(), + input.watermark_columns().clone(), + ) + }; + StreamShare { base, core } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs index 92d96fdf21b08..06aa227d7e316 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -18,17 +18,16 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{self, PlanAggCall}; +use super::stream::prelude::*; use super::utils::{childless_record, plan_node_name, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; -use crate::optimizer::plan_node::generic::PhysicalPlanRef; -use crate::optimizer::plan_node::stream::StreamPlanRef; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamSimpleAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, /// The index of `count(*)` in `agg_calls`. diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 32e9fb487910c..cd48fda60e06b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -38,6 +38,7 @@ use tracing::info; use super::derive::{derive_columns, derive_pk}; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, Distill, IndicesDisplay, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, StreamNode}; use crate::optimizer::plan_node::PlanTreeNodeUnary; @@ -50,7 +51,7 @@ const DOWNSTREAM_PK_KEY: &str = "primary_key"; /// [`StreamSink`] represents a table/connector sink at the very end of the graph. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamSink { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, sink_desc: SinkDesc, } @@ -58,7 +59,11 @@ pub struct StreamSink { impl StreamSink { #[must_use] pub fn new(input: PlanRef, sink_desc: SinkDesc) -> Self { - let base = input.plan_base().clone_with_new_plan_id(); + let base = input + .plan_base() + .into_stream() + .expect("input should be stream plan") + .clone_with_new_plan_id(); Self { base, input, diff --git a/src/frontend/src/optimizer/plan_node/stream_sort.rs b/src/frontend/src/optimizer/plan_node/stream_sort.rs index 41a56a0fd5df2..2468fe52858e4 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sort.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sort.rs @@ -20,8 +20,7 @@ use risingwave_common::catalog::FieldDisplay; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; -use super::stream::StreamPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, Distill, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -29,7 +28,7 @@ use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamEowcSort { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, sort_column_index: usize, diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index ae66cf568118b..51a764b53fab2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -20,10 +20,10 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::{PbStreamSource, SourceNode}; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, StreamNode}; use crate::catalog::source_catalog::SourceCatalog; -use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::utils::column_names_pretty; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -31,7 +31,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// [`StreamSource`] represents a table/connector source at the very beginning of the graph. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamSource { - pub base: PlanBase, + pub base: PlanBase, pub(crate) core: generic::Source, } diff --git a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs index 474582ec877c7..ca076132f3a55 100644 --- a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs @@ -17,6 +17,7 @@ use itertools::Itertools; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{self, PlanAggCall}; +use super::stream::prelude::*; use super::utils::impl_distill_by_unit; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; @@ -32,7 +33,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// by `StreamSimpleAgg`. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamStatelessSimpleAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, } 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 965ca217a3369..882410e5c9010 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -24,13 +24,11 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::{ChainType, PbStreamNode}; -use super::generic::PhysicalPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanNodeId, PlanRef, StreamNode}; use crate::catalog::ColumnId; use crate::expr::{ExprRewriter, FunctionCall}; -use crate::optimizer::plan_node::generic::GenericPlanRef; -use crate::optimizer::plan_node::stream::StreamPlanRef; use crate::optimizer::plan_node::utils::{IndicesDisplay, TableCatalogBuilder}; use crate::optimizer::property::{Distribution, DistributionDisplay}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -41,7 +39,7 @@ use crate::{Explain, TableCatalog}; /// creation request. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTableScan { - pub base: PlanBase, + pub base: PlanBase, core: generic::Scan, batch_plan_id: PlanNodeId, chain_type: ChainType, diff --git a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs index 675dbeb9ab381..cb524f35bd329 100644 --- a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs @@ -19,6 +19,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::TemporalJoinNode; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamNode}; @@ -33,7 +34,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTemporalJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, eq_join_predicate: EqJoinPredicate, } diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs index 87890625f6be7..1f3618369d15e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs @@ -17,6 +17,7 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{DistillUnit, TopNLimit}; +use super::stream::prelude::*; use super::utils::{plan_node_name, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::{Distribution, Order}; @@ -25,7 +26,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index 6d6dca2d8dd02..425cdc6914564 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -20,6 +20,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::UnionNode; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanRef}; @@ -30,7 +31,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamUnion` implements [`super::LogicalUnion`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamUnion { - pub base: PlanBase, + pub base: PlanBase, core: generic::Union, } diff --git a/src/frontend/src/optimizer/plan_node/stream_values.rs b/src/frontend/src/optimizer/plan_node/stream_values.rs index f8cc5db851159..3ae19a3112c5e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_values.rs +++ b/src/frontend/src/optimizer/plan_node/stream_values.rs @@ -18,7 +18,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::values_node::ExprTuple; use risingwave_pb::stream_plan::ValuesNode; -use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, LogicalValues, PlanBase, StreamNode}; use crate::expr::{Expr, ExprImpl}; @@ -28,7 +28,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamValues` implements `LogicalValues.to_stream()` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamValues { - pub base: PlanBase, + pub base: PlanBase, logical: LogicalValues, } diff --git a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs index 066bc9a234ca5..45997be797c0f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs @@ -21,6 +21,7 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::WatermarkDesc; use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -30,7 +31,7 @@ use crate::{TableCatalog, WithOptions}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamWatermarkFilter { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, watermark_descs: Vec, } @@ -54,7 +55,11 @@ impl StreamWatermarkFilter { Self::with_base(base, input, watermark_descs) } - fn with_base(base: PlanBase, input: PlanRef, watermark_descs: Vec) -> Self { + fn with_base( + base: PlanBase, + input: PlanRef, + watermark_descs: Vec, + ) -> Self { Self { base, input, diff --git a/src/frontend/src/optimizer/property/distribution.rs b/src/frontend/src/optimizer/property/distribution.rs index 2df1d7ae00bc3..8ab78c0cd4279 100644 --- a/src/frontend/src/optimizer/property/distribution.rs +++ b/src/frontend/src/optimizer/property/distribution.rs @@ -47,6 +47,7 @@ use std::fmt; use std::fmt::Debug; use fixedbitset::FixedBitSet; +use generic::PhysicalPlanRef; use itertools::Itertools; use risingwave_common::catalog::{FieldDisplay, Schema, TableId}; use risingwave_common::error::Result; diff --git a/src/frontend/src/optimizer/property/order.rs b/src/frontend/src/optimizer/property/order.rs index 19ad7586e1c11..bf373e8629d7b 100644 --- a/src/frontend/src/optimizer/property/order.rs +++ b/src/frontend/src/optimizer/property/order.rs @@ -85,6 +85,8 @@ const ANY_ORDER: Order = Order { impl Order { pub fn enforce_if_not_satisfies(&self, plan: PlanRef) -> Result { + use crate::optimizer::plan_node::batch::prelude::*; + if !plan.order().satisfies(self) { Ok(self.enforce(plan)) } else { diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index cb20103b3e76f..cc23256f43687 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -42,7 +42,7 @@ use uuid::Uuid; use super::SchedulerError; use crate::catalog::catalog_service::CatalogReader; use crate::catalog::TableId; -use crate::optimizer::plan_node::generic::GenericPlanRef; +use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; use crate::optimizer::plan_node::{PlanNodeId, PlanNodeType}; use crate::optimizer::property::Distribution; use crate::optimizer::PlanRef; From e74b32dab8229bd61af22a6250c32588510dea44 Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Thu, 26 Oct 2023 16:11:58 +0800 Subject: [PATCH 33/48] feat: Add pulsar iceberg table reader (#12735) --- Cargo.lock | 3 +- Cargo.toml | 2 +- src/connector/src/aws_utils.rs | 10 +- src/connector/src/error.rs | 3 + src/connector/src/source/pulsar/mod.rs | 11 +- .../src/source/pulsar/source/reader.rs | 372 +++++++++++++++++- src/connector/src/source/pulsar/topic.rs | 9 + 7 files changed, 400 insertions(+), 10 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 1c741e3a5ae67..2f2ee2991b10f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3852,7 +3852,7 @@ dependencies = [ [[package]] name = "icelake" version = "0.0.10" -source = "git+https://github.com/icelake-io/icelake?rev=16dab0e36ab337e58ee8002d828def2d212fa116#16dab0e36ab337e58ee8002d828def2d212fa116" +source = "git+https://github.com/icelake-io/icelake?rev=186fde7663545d1d6a5856ce9fbbc541224eadfb#186fde7663545d1d6a5856ce9fbbc541224eadfb" dependencies = [ "anyhow", "apache-avro 0.15.0 (registry+https://github.com/rust-lang/crates.io-index)", @@ -3869,6 +3869,7 @@ dependencies = [ "bytes", "chrono", "csv", + "derive_builder", "enum-display", "faster-hex", "futures", diff --git a/Cargo.toml b/Cargo.toml index ac533e733f7a8..f0fbb8485d318 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -112,7 +112,7 @@ criterion = { version = "0.5", features = ["async_futures"] } tonic = { package = "madsim-tonic", version = "0.4.0" } tonic-build = { package = "madsim-tonic-build", version = "0.4.2" } prost = { version = "0.12" } -icelake = { git = "https://github.com/icelake-io/icelake", rev = "16dab0e36ab337e58ee8002d828def2d212fa116" } +icelake = { git = "https://github.com/icelake-io/icelake", rev = "186fde7663545d1d6a5856ce9fbbc541224eadfb" } arrow-array = "48" arrow-cast = "48" arrow-schema = "48" diff --git a/src/connector/src/aws_utils.rs b/src/connector/src/aws_utils.rs index 785a4396bacba..c62d9b9d6d14d 100644 --- a/src/connector/src/aws_utils.rs +++ b/src/connector/src/aws_utils.rs @@ -24,12 +24,16 @@ use url::Url; use crate::aws_auth::AwsAuthProps; +pub const REGION: &str = "region"; +pub const ACCESS_KEY: &str = "access_key"; +pub const SECRET_ACCESS: &str = "secret_access"; + pub const AWS_DEFAULT_CONFIG: [&str; 7] = [ - "region", + REGION, "arn", "profile", - "access_key", - "secret_access", + ACCESS_KEY, + SECRET_ACCESS, "session_token", "endpoint_url", ]; diff --git a/src/connector/src/error.rs b/src/connector/src/error.rs index 73cd27d5801e5..2cdfdd99ecfe0 100644 --- a/src/connector/src/error.rs +++ b/src/connector/src/error.rs @@ -35,6 +35,9 @@ pub enum ConnectorError { #[error("MySQL error: {0}")] MySql(#[from] mysql_async::Error), + #[error("Pulsar error: {0}")] + Pulsar(anyhow::Error), + #[error(transparent)] Internal(#[from] anyhow::Error), } diff --git a/src/connector/src/source/pulsar/mod.rs b/src/connector/src/source/pulsar/mod.rs index 544d1b7fb3ed3..c9db87fd3bb94 100644 --- a/src/connector/src/source/pulsar/mod.rs +++ b/src/connector/src/source/pulsar/mod.rs @@ -19,10 +19,11 @@ pub mod topic; pub use enumerator::*; use serde::Deserialize; +use serde_with::serde_as; pub use split::*; +use self::source::reader::PulsarSplitReader; use crate::common::PulsarCommon; -use crate::source::pulsar::source::reader::PulsarSplitReader; use crate::source::SourceProperties; pub const PULSAR_CONNECTOR: &str = "pulsar"; @@ -36,6 +37,7 @@ impl SourceProperties for PulsarProperties { } #[derive(Clone, Debug, Deserialize)] +#[serde_as] pub struct PulsarProperties { #[serde(rename = "scan.startup.mode", alias = "pulsar.scan.startup.mode")] pub scan_startup_mode: Option, @@ -45,4 +47,11 @@ pub struct PulsarProperties { #[serde(flatten)] pub common: PulsarCommon, + + #[serde(rename = "iceberg.enabled")] + #[serde_as(as = "Option")] + pub iceberg_loader_enabled: bool, + + #[serde(rename = "iceberg.bucket", default)] + pub iceberg_bucket: Option, } diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index 85d85a8d18714..0568a8935932e 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -12,26 +12,92 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::time::{SystemTime, UNIX_EPOCH}; use anyhow::{anyhow, ensure, Result}; +use arrow_array::{Int32Array, Int64Array, RecordBatch}; use async_trait::async_trait; use futures::StreamExt; use futures_async_stream::try_stream; +use icelake::catalog::{load_catalog, CATALOG_NAME, CATALOG_TYPE}; +use icelake::io::FileScanStream; +use icelake::types::{Any, AnyValue, StructValueBuilder}; +use icelake::{Table, TableIdentifier}; use itertools::Itertools; use pulsar::consumer::InitialPosition; use pulsar::message::proto::MessageIdData; use pulsar::{Consumer, ConsumerBuilder, ConsumerOptions, Pulsar, SubType, TokioExecutor}; +use risingwave_common::array::{DataChunk, StreamChunk}; +use risingwave_common::catalog::ROWID_PREFIX; +use risingwave_common::error::RwError; +use crate::aws_utils::{ACCESS_KEY, REGION, SECRET_ACCESS}; +use crate::error::ConnectorError; use crate::parser::ParserConfig; use crate::source::pulsar::split::PulsarSplit; use crate::source::pulsar::{PulsarEnumeratorOffset, PulsarProperties}; use crate::source::{ into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, - SourceMessage, SplitId, SplitMetaData, SplitReader, + SourceMessage, SplitId, SplitMetaData, SplitReader, StreamChunkWithState, }; -pub struct PulsarSplitReader { +pub enum PulsarSplitReader { + Broker(PulsarBrokerReader), + Iceberg(PulsarIcebergReader), +} + +#[async_trait] +impl SplitReader for PulsarSplitReader { + type Properties = PulsarProperties; + type Split = PulsarSplit; + + async fn new( + props: PulsarProperties, + splits: Vec, + parser_config: ParserConfig, + source_ctx: SourceContextRef, + _columns: Option>, + ) -> Result { + ensure!(splits.len() == 1, "only support single split"); + let split = splits.into_iter().next().unwrap(); + let topic = split.topic.to_string(); + + tracing::debug!("creating consumer for pulsar split topic {}", topic,); + + if props.iceberg_loader_enabled + && matches!(split.start_offset, PulsarEnumeratorOffset::Earliest) + && !topic.starts_with("non-persistent://") + { + tracing::debug!("Creating iceberg reader for pulsar split topic {}", topic); + Ok(Self::Iceberg(PulsarIcebergReader::new( + props, + split, + source_ctx, + parser_config, + ))) + } else { + Ok(Self::Broker( + PulsarBrokerReader::new(props, vec![split], parser_config, source_ctx, None) + .await?, + )) + } + } + + fn into_stream(self) -> BoxSourceWithStateStream { + match self { + Self::Broker(reader) => { + let (parser_config, source_context) = + (reader.parser_config.clone(), reader.source_ctx.clone()); + Box::pin(into_chunk_stream(reader, parser_config, source_context)) + } + Self::Iceberg(reader) => Box::pin(reader.into_stream()), + } + } +} + +/// This reader reads from pulsar broker +pub struct PulsarBrokerReader { pulsar: Pulsar, consumer: Consumer, TokioExecutor>, split: PulsarSplit, @@ -84,7 +150,7 @@ fn parse_message_id(id: &str) -> Result { } #[async_trait] -impl SplitReader for PulsarSplitReader { +impl SplitReader for PulsarBrokerReader { type Properties = PulsarProperties; type Split = PulsarSplit; @@ -173,7 +239,7 @@ impl SplitReader for PulsarSplitReader { } } -impl CommonSplitReader for PulsarSplitReader { +impl CommonSplitReader for PulsarBrokerReader { #[try_stream(ok = Vec, error = anyhow::Error)] async fn into_data_stream(self) { let max_chunk_size = self.source_ctx.source_ctrl_opts.chunk_size; @@ -188,3 +254,301 @@ impl CommonSplitReader for PulsarSplitReader { } } } + +const META_COLUMN_TOPIC: &str = "__topic"; +const META_COLUMN_KEY: &str = "__key"; +const META_COLUMN_LEDGER_ID: &str = "__ledgerId"; +const META_COLUMN_ENTRY_ID: &str = "__entryId"; +const META_COLUMN_BATCH_INDEX: &str = "__batchIndex"; +const META_COLUMN_PARTITION: &str = "__partition"; + +/// Read history data from iceberg table +pub struct PulsarIcebergReader { + props: PulsarProperties, + split: PulsarSplit, + source_ctx: SourceContextRef, + parser_config: ParserConfig, +} + +impl PulsarIcebergReader { + fn new( + props: PulsarProperties, + split: PulsarSplit, + source_ctx: SourceContextRef, + parser_config: ParserConfig, + ) -> Self { + Self { + props, + split, + source_ctx, + parser_config, + } + } + + async fn scan(&self) -> Result { + let table = self.create_iceberg_table().await?; + let schema = table.current_table_metadata().current_schema()?; + tracing::debug!("Created iceberg pulsar table, schema is: {:?}", schema,); + + let max_chunk_size = self.source_ctx.source_ctrl_opts.chunk_size; + + let partition_value = match &self.split.topic.partition_index { + Some(partition_id) => { + let (partition_type, partition_field) = match table.current_partition_type()? { + Any::Struct(s) => { + let field = s + .fields() + .iter() + .find(|f| f.name == META_COLUMN_PARTITION) + .ok_or_else(|| { + ConnectorError::Pulsar(anyhow!( + "Partition field not found in partition spec" + )) + })?; + (s.clone(), field.clone()) + } + _ => { + return Err(ConnectorError::Pulsar(anyhow!( + "Partition type is not struct in iceberg table: {}", + table.table_name() + )))?; + } + }; + + let mut partition_value_builder = StructValueBuilder::new(partition_type); + partition_value_builder.add_field( + partition_field.id, + Some(AnyValue::Primitive(icelake::types::PrimitiveValue::Int( + *partition_id, + ))), + )?; + Some(partition_value_builder.build()?) + } + None => None, + }; + + // TODO: Add partition + Ok(table + .new_scan_builder() + .with_partition_value(partition_value) + .with_batch_size(max_chunk_size) + .build()? + .scan(&table) + .await?) + } + + async fn create_iceberg_table(&self) -> Result

{ + let catalog = load_catalog(&self.build_iceberg_configs()?) + .await + .map_err(|e| ConnectorError::Pulsar(anyhow!("Unable to load iceberg catalog: {e}")))?; + + let table_id = + TableIdentifier::new(vec![self.split.topic.topic_str_without_partition()?]) + .map_err(|e| ConnectorError::Pulsar(anyhow!("Unable to parse table name: {e}")))?; + + let table = catalog + .load_table(&table_id) + .await + .map_err(|err| ConnectorError::Pulsar(anyhow!(err)))?; + + Ok(table) + } + + #[try_stream(ok = StreamChunkWithState, error = anyhow::Error)] + async fn as_stream_chunk_stream(&self) { + #[for_await] + for file_scan in self.scan().await? { + let file_scan = file_scan?; + + #[for_await] + for record_batch in file_scan.scan().await? { + let batch = record_batch?; + let msgs = self.convert_record_batch_to_source_with_state(&batch)?; + yield msgs; + } + } + } + + #[try_stream(ok = StreamChunkWithState, error = RwError)] + async fn into_stream(self) { + let (props, mut split, parser_config, source_ctx) = ( + self.props.clone(), + self.split.clone(), + self.parser_config.clone(), + self.source_ctx.clone(), + ); + tracing::info!("Starting to read pulsar message from iceberg"); + let mut last_msg_id = None; + + #[for_await] + for msg in self.as_stream_chunk_stream() { + let msg = + msg.inspect_err(|e| tracing::error!("Failed to read message from iceberg: {}", e))?; + last_msg_id = msg + .split_offset_mapping + .as_ref() + .and_then(|m| m.get(self.split.topic.to_string().as_str())) + .cloned(); + } + + tracing::info!("Finished reading pulsar message from iceberg"); + // We finished reading all the data from iceberg table, now we need to start from broker. + if let Some(msg_id) = last_msg_id { + tracing::info!("Last iceberg message id is {}", msg_id); + split.start_offset = PulsarEnumeratorOffset::MessageId(msg_id); + } + + tracing::info!( + "Switching from pulsar iceberg reader to broker reader with offset: {:?}", + split.start_offset + ); + let broker_reader = PulsarSplitReader::Broker( + PulsarBrokerReader::new(props, vec![split], parser_config, source_ctx, None).await?, + ); + + #[for_await] + for msg in broker_reader.into_stream() { + yield msg?; + } + } + + fn build_iceberg_configs(&self) -> Result> { + let mut iceberg_configs = HashMap::new(); + + let bucket = + self.props.iceberg_bucket.as_ref().ok_or_else(|| { + ConnectorError::Pulsar(anyhow!("Iceberg bucket is not configured")) + })?; + + iceberg_configs.insert(CATALOG_TYPE.to_string(), "storage".to_string()); + iceberg_configs.insert(CATALOG_NAME.to_string(), "pulsar".to_string()); + iceberg_configs.insert( + "iceberg.catalog.pulsar.warehouse".to_string(), + format!( + "s3://{}/{}/{}", + bucket, self.split.topic.tenant, self.split.topic.namespace, + ), + ); + + if let Some(s3_configs) = self.props.common.oauth.as_ref().map(|s| &s.s3_credentials) { + if let Some(region) = s3_configs.get(REGION) { + iceberg_configs.insert("iceberg.table.io.region".to_string(), region.to_string()); + } + + if let Some(access_key) = s3_configs.get(ACCESS_KEY) { + iceberg_configs.insert( + "iceberg.table.io.access_key_id".to_string(), + access_key.to_string(), + ); + } + + if let Some(secret_key) = s3_configs.get(SECRET_ACCESS) { + iceberg_configs.insert( + "iceberg.table.io.secret_access_key".to_string(), + secret_key.to_string(), + ); + } + } + + iceberg_configs.insert("iceberg.table.io.bucket".to_string(), bucket.to_string()); + iceberg_configs.insert( + "iceberg.table.io.root".to_string(), + format!( + "/{}/{}", + self.split.topic.tenant, self.split.topic.namespace + ), + ); + // #TODO + // Support load config file + iceberg_configs.insert( + "iceberg.table.io.disable_config_load".to_string(), + "true".to_string(), + ); + + Ok(iceberg_configs) + } + + // Converts arrow record batch to stream chunk. + fn convert_record_batch_to_source_with_state( + &self, + record_batch: &RecordBatch, + ) -> Result { + let mut offsets = Vec::with_capacity(record_batch.num_rows()); + + let ledger_id_array = record_batch + .column_by_name(META_COLUMN_LEDGER_ID) + .ok_or_else(|| ConnectorError::Pulsar(anyhow!("Ledger id not found in iceberg table")))? + .as_any() + .downcast_ref::() + .ok_or_else(|| { + ConnectorError::Pulsar(anyhow!("Ledger id is not i64 in iceberg table")) + })?; + + let entry_id_array = record_batch + .column_by_name(META_COLUMN_ENTRY_ID) + .ok_or_else(|| ConnectorError::Pulsar(anyhow!("Entry id not found in iceberg table")))? + .as_any() + .downcast_ref::() + .ok_or_else(|| { + ConnectorError::Pulsar(anyhow!("Entry id is not i64 in iceberg table")) + })?; + + let partition_array = record_batch + .column_by_name(META_COLUMN_PARTITION) + .map(|arr| { + arr.as_any().downcast_ref::().ok_or_else(|| { + ConnectorError::Pulsar(anyhow!("Partition is not i32 in iceberg table")) + }) + }) + .transpose()?; + + let batch_index_array = record_batch + .column_by_name(META_COLUMN_BATCH_INDEX) + .map(|arr| { + arr.as_any().downcast_ref::().ok_or_else(|| { + ConnectorError::Pulsar(anyhow!("Batch index is not i64 in iceberg table")) + }) + }) + .transpose()?; + + let field_indices = self + .parser_config + .common + .rw_columns + .iter() + .filter(|col| col.name != ROWID_PREFIX) + .map(|col| { + record_batch + .schema() + .index_of(col.name.as_str()) + .map_err(|e| anyhow!(e)) + }) + .collect::>>()?; + + for row in 0..record_batch.num_rows() { + let offset = format!( + "{}:{}:{}:{}", + ledger_id_array.value(row), + entry_id_array.value(row), + partition_array.map(|arr| arr.value(row)).unwrap_or(-1), + batch_index_array.map(|arr| arr.value(row)).unwrap_or(-1) + ); + + offsets.push(offset); + } + + let data_chunk = DataChunk::try_from(&record_batch.project(&field_indices)?)?; + + let stream_chunk = StreamChunk::from(data_chunk); + + let state = Some(HashMap::from([( + self.split.topic.to_string().into(), + offsets.last().unwrap().clone(), + )])); + + Ok(StreamChunkWithState { + chunk: stream_chunk, + split_offset_mapping: state, + }) + } +} diff --git a/src/connector/src/source/pulsar/topic.rs b/src/connector/src/source/pulsar/topic.rs index ab4b1f84750f3..e6fbc6dbe15de 100644 --- a/src/connector/src/source/pulsar/topic.rs +++ b/src/connector/src/source/pulsar/topic.rs @@ -73,6 +73,15 @@ impl Topic { partition_index: Some(partition), }) } + + pub fn topic_str_without_partition(&self) -> Result { + if self.topic.contains(PARTITIONED_TOPIC_SUFFIX) { + let parts: Vec<&str> = self.topic.split(PARTITIONED_TOPIC_SUFFIX).collect(); + Ok(parts[0].to_string()) + } else { + Ok(self.topic.clone()) + } + } } /// `get_partition_index` returns the partition index of the topic. From 41d2bbd3988b33350240a41742f82392e538d9ff Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 26 Oct 2023 16:27:21 +0800 Subject: [PATCH 34/48] refactor(sqlparser): rename `SourceSchemaV2`/`SinkSchema` to `ConnectorSchema` (#13065) --- .../src/handler/alter_table_column.rs | 10 +- src/frontend/src/handler/create_sink.rs | 8 +- src/frontend/src/handler/create_source.rs | 15 ++- src/frontend/src/handler/create_table.rs | 10 +- src/sqlparser/src/ast/statement.rs | 94 ++++--------------- src/sqlparser/tests/testdata/create.yaml | 6 +- 6 files changed, 44 insertions(+), 99 deletions(-) diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index 1a6d02b963e9d..18313e0458a04 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -22,7 +22,7 @@ use risingwave_pb::catalog::Table; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_sqlparser::ast::{ - AlterTableOperation, ColumnOption, Encode, ObjectName, SourceSchemaV2, Statement, + AlterTableOperation, ColumnOption, ConnectorSchema, Encode, ObjectName, Statement, }; use risingwave_sqlparser::parser::Parser; @@ -32,7 +32,7 @@ use super::{HandlerArgs, RwPgResponse}; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::table_catalog::TableType; use crate::handler::create_table::gen_create_table_plan_with_source; -use crate::{build_graph, Binder, OptimizerContext, TableCatalog}; +use crate::{build_graph, Binder, OptimizerContext, TableCatalog, WithOptions}; /// Handle `ALTER TABLE [ADD|DROP] COLUMN` statements. The `operation` must be either `AddColumn` or /// `DropColumn`. @@ -262,12 +262,12 @@ pub async fn handle_alter_table_column( Ok(PgResponse::empty_result(StatementType::ALTER_TABLE)) } -fn schema_has_schema_registry(schema: &SourceSchemaV2) -> bool { +fn schema_has_schema_registry(schema: &ConnectorSchema) -> bool { match schema.row_encode { Encode::Avro | Encode::Protobuf => true, Encode::Json => { - let mut options = schema.gen_options().unwrap(); - matches!(get_json_schema_location(&mut options), Ok(Some(_))) + let mut options = WithOptions::try_from(schema.row_options()).unwrap(); + matches!(get_json_schema_location(options.inner_mut()), Ok(Some(_))) } _ => false, } diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index ddb1d697b856d..2ab987308a4f5 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -27,8 +27,8 @@ use risingwave_connector::sink::{ }; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ - CreateSink, CreateSinkStatement, EmitMode, Encode, Format, ObjectName, Query, Select, - SelectItem, SetExpr, SinkSchema, TableFactor, TableWithJoins, + ConnectorSchema, CreateSink, CreateSinkStatement, EmitMode, Encode, Format, ObjectName, Query, + Select, SelectItem, SetExpr, TableFactor, TableWithJoins, }; use super::create_mv::get_column_names; @@ -228,7 +228,7 @@ pub async fn handle_create_sink( /// Transforms the (format, encode, options) from sqlparser AST into an internal struct `SinkFormatDesc`. /// This is an analogy to (part of) [`crate::handler::create_source::try_bind_columns_from_source`] /// which transforms sqlparser AST `SourceSchemaV2` into `StreamSourceInfo`. -fn bind_sink_format_desc(value: SinkSchema) -> Result { +fn bind_sink_format_desc(value: ConnectorSchema) -> Result { use risingwave_connector::sink::catalog::{SinkEncode, SinkFormat}; use risingwave_sqlparser::ast::{Encode as E, Format as F}; @@ -288,7 +288,7 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock Result<()> { +pub fn validate_compatibility(connector: &str, format_desc: &ConnectorSchema) -> Result<()> { let compatible_formats = CONNECTORS_COMPATIBLE_FORMATS .get(connector) .ok_or_else(|| { diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 0c2398a608eb8..0ce3e32ed584e 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -15,7 +15,6 @@ use std::collections::{BTreeMap, HashMap}; use std::sync::LazyLock; -use anyhow::anyhow; use itertools::Itertools; use maplit::{convert_args, hashmap}; use pgwire::pg_response::{PgResponse, StatementType}; @@ -46,8 +45,8 @@ use risingwave_pb::catalog::{ }; use risingwave_pb::plan_common::{EncodeType, FormatType}; use risingwave_sqlparser::ast::{ - self, get_delimiter, AstString, AvroSchema, ColumnDef, ColumnOption, CreateSourceStatement, - DebeziumAvroSchema, Encode, Format, ProtobufSchema, SourceSchemaV2, SourceWatermark, + self, get_delimiter, AstString, AvroSchema, ColumnDef, ColumnOption, ConnectorSchema, + CreateSourceStatement, DebeziumAvroSchema, Encode, Format, ProtobufSchema, SourceWatermark, }; use super::RwPgResponse; @@ -278,7 +277,7 @@ fn get_name_strategy_or_default(name_strategy: Option) -> Result for more information. /// return `(columns, pk_names, source info)` pub(crate) async fn try_bind_columns_from_source( - source_schema: &SourceSchemaV2, + source_schema: &ConnectorSchema, sql_defined_pk_names: Vec, sql_defined_columns: &[ColumnDef], with_properties: &HashMap, @@ -290,7 +289,7 @@ pub(crate) async fn try_bind_columns_from_source( let sql_defined_pk = !sql_defined_pk_names.is_empty(); let sql_defined_schema = !sql_defined_columns.is_empty(); let is_kafka: bool = is_kafka_connector(with_properties); - let mut options = source_schema.gen_options().map_err(|e| anyhow!(e))?; + let mut options = WithOptions::try_from(source_schema.row_options())?.into_inner(); let get_key_message_name = |options: &mut BTreeMap| -> Option { consume_string_from_options(options, KEY_MESSAGE_NAME_KEY) @@ -904,7 +903,7 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock, ) -> Result<()> { let connector = get_connector(props) @@ -922,8 +921,8 @@ pub fn validate_compatibility( if connector != KAFKA_CONNECTOR { let res = match (&source_schema.format, &source_schema.row_encode) { (Format::Plain, Encode::Protobuf) | (Format::Plain, Encode::Avro) => { - let mut options = source_schema.gen_options().map_err(|e| anyhow!(e))?; - let (_, use_schema_registry) = get_schema_location(&mut options)?; + let mut options = WithOptions::try_from(source_schema.row_options())?; + let (_, use_schema_registry) = get_schema_location(options.inner_mut())?; use_schema_registry } (Format::Debezium, Encode::Avro) => true, diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index bb02797c21395..e412658cb712f 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -33,7 +33,7 @@ use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; use risingwave_pb::plan_common::{DefaultColumnDesc, GeneratedColumnDesc}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ - ColumnDef, ColumnOption, DataType as AstDataType, Format, ObjectName, SourceSchemaV2, + ColumnDef, ColumnOption, ConnectorSchema, DataType as AstDataType, Format, ObjectName, SourceWatermark, TableConstraint, }; @@ -436,7 +436,7 @@ pub(crate) async fn gen_create_table_plan_with_source( table_name: ObjectName, column_defs: Vec, constraints: Vec, - source_schema: SourceSchemaV2, + source_schema: ConnectorSchema, source_watermarks: Vec, mut col_id_gen: ColumnIdGenerator, append_only: bool, @@ -749,7 +749,7 @@ pub async fn handle_create_table( columns: Vec, constraints: Vec, if_not_exists: bool, - source_schema: Option, + source_schema: Option, source_watermarks: Vec, append_only: bool, notice: Option, @@ -826,8 +826,8 @@ pub async fn handle_create_table( pub fn check_create_table_with_source( with_options: &WithOptions, - source_schema: Option, -) -> Result> { + source_schema: Option, +) -> Result> { if with_options.inner().contains_key(UPSTREAM_SOURCE_KEY) { source_schema.as_ref().ok_or_else(|| { ErrorCode::InvalidInputSyntax("Please specify a source schema using FORMAT".to_owned()) diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index 58fb2d50c6287..3ff012c81b766 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -13,7 +13,6 @@ // limitations under the License. use core::fmt; -use std::collections::BTreeMap; use std::fmt::Write; use itertools::Itertools; @@ -105,7 +104,7 @@ pub enum SourceSchema { } impl SourceSchema { - pub fn into_source_schema_v2(self) -> SourceSchemaV2 { + pub fn into_source_schema_v2(self) -> ConnectorSchema { let (format, row_encode) = match self { SourceSchema::Protobuf(_) => (Format::Plain, Encode::Protobuf), SourceSchema::Json => (Format::Plain, Encode::Json), @@ -205,7 +204,7 @@ impl SourceSchema { _ => vec![], }; - SourceSchemaV2 { + ConnectorSchema { format, row_encode, row_options, @@ -336,7 +335,7 @@ impl Encode { #[derive(Debug, Clone, PartialEq, Eq, Hash)] #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -pub struct SourceSchemaV2 { +pub struct ConnectorSchema { pub format: Format, pub row_encode: Encode, pub row_options: Vec, @@ -346,7 +345,7 @@ pub struct SourceSchemaV2 { #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] pub enum CompatibleSourceSchema { RowFormat(SourceSchema), - V2(SourceSchemaV2), + V2(ConnectorSchema), } impl fmt::Display for CompatibleSourceSchema { @@ -363,7 +362,7 @@ impl fmt::Display for CompatibleSourceSchema { } impl CompatibleSourceSchema { - pub fn into_source_schema_v2(self) -> (SourceSchemaV2, Option) { + pub fn into_source_schema_v2(self) -> (ConnectorSchema, Option) { match self { CompatibleSourceSchema::RowFormat(inner) => ( inner.into_source_schema_v2(), @@ -373,29 +372,15 @@ impl CompatibleSourceSchema { } } -impl From for CompatibleSourceSchema { - fn from(value: SourceSchemaV2) -> Self { +impl From for CompatibleSourceSchema { + fn from(value: ConnectorSchema) -> Self { Self::V2(value) } } fn parse_source_schema(p: &mut Parser) -> Result { - if p.peek_nth_any_of_keywords(0, &[Keyword::FORMAT]) { - p.expect_keyword(Keyword::FORMAT)?; - let id = p.parse_identifier()?; - let s = id.value.to_ascii_uppercase(); - let format = Format::from_keyword(&s)?; - p.expect_keyword(Keyword::ENCODE)?; - let id = p.parse_identifier()?; - let s = id.value.to_ascii_uppercase(); - let row_encode = Encode::from_keyword(&s)?; - let row_options = p.parse_options()?; - - Ok(CompatibleSourceSchema::V2(SourceSchemaV2 { - format, - row_encode, - row_options, - })) + if let Some(schema_v2) = p.parse_schema()? { + Ok(CompatibleSourceSchema::V2(schema_v2)) } else if p.peek_nth_any_of_keywords(0, &[Keyword::ROW]) && p.peek_nth_any_of_keywords(1, &[Keyword::FORMAT]) { @@ -465,7 +450,7 @@ impl Parser { // row format for nexmark source must be native // default row format for datagen source is native if connector.contains("-cdc") { - let expected = SourceSchemaV2::debezium_json(); + let expected = ConnectorSchema::debezium_json(); if self.peek_source_schema_format() { let schema = parse_source_schema(self)?.into_source_schema_v2().0; if schema != expected { @@ -477,7 +462,7 @@ impl Parser { } Ok(expected.into()) } else if connector.contains("nexmark") { - let expected = SourceSchemaV2::native(); + let expected = ConnectorSchema::native(); if self.peek_source_schema_format() { let schema = parse_source_schema(self)?.into_source_schema_v2().0; if schema != expected { @@ -492,17 +477,15 @@ impl Parser { Ok(if self.peek_source_schema_format() { parse_source_schema(self)? } else { - SourceSchemaV2::native().into() + ConnectorSchema::native().into() }) } else { Ok(parse_source_schema(self)?) } } - /// Parse `FORMAT ... ENCODE ... (...)` in `CREATE SINK`. - /// - /// TODO: After [`SourceSchemaV2`] and [`SinkSchema`] merge, call this in [`parse_source_schema`]. - pub fn parse_schema(&mut self) -> Result, ParserError> { + /// Parse `FORMAT ... ENCODE ... (...)` in `CREATE SOURCE` and `CREATE SINK`. + pub fn parse_schema(&mut self) -> Result, ParserError> { if !self.parse_keyword(Keyword::FORMAT) { return Ok(None); } @@ -516,7 +499,7 @@ impl Parser { let row_encode = Encode::from_keyword(&s)?; let row_options = self.parse_options()?; - Ok(Some(SinkSchema { + Ok(Some(ConnectorSchema { format, row_encode, row_options, @@ -524,10 +507,10 @@ impl Parser { } } -impl SourceSchemaV2 { +impl ConnectorSchema { /// Create a new source schema with `Debezium` format and `Json` encoding. pub const fn debezium_json() -> Self { - SourceSchemaV2 { + ConnectorSchema { format: Format::Debezium, row_encode: Encode::Json, row_options: Vec::new(), @@ -536,35 +519,19 @@ impl SourceSchemaV2 { /// Create a new source schema with `Native` format and encoding. pub const fn native() -> Self { - SourceSchemaV2 { + ConnectorSchema { format: Format::Native, row_encode: Encode::Native, row_options: Vec::new(), } } - pub fn gen_options(&self) -> Result, ParserError> { - self.row_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(ParserError::ParserError( - "`row format options` only support single quoted string value and C style escaped string".to_owned(), - )), - }) - .try_collect() - } - pub fn row_options(&self) -> &[SqlOption] { self.row_options.as_ref() } } -impl fmt::Display for SourceSchemaV2 { +impl fmt::Display for ConnectorSchema { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { write!(f, "FORMAT {} ENCODE {}", self.format, self.row_encode)?; @@ -823,27 +790,6 @@ impl fmt::Display for CreateSink { } } -/// Same as [`SourceSchemaV2`]. Will be merged in a dedicated rename PR. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -pub struct SinkSchema { - pub format: Format, - pub row_encode: Encode, - pub row_options: Vec, -} - -impl fmt::Display for SinkSchema { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "FORMAT {} ENCODE {}", self.format, self.row_encode)?; - - if !self.row_options.is_empty() { - write!(f, " ({})", display_comma_separated(&self.row_options)) - } else { - Ok(()) - } - } -} - // sql_grammar!(CreateSinkStatement { // if_not_exists => [Keyword::IF, Keyword::NOT, Keyword::EXISTS], // sink_name: Ident, @@ -860,7 +806,7 @@ pub struct CreateSinkStatement { pub sink_from: CreateSink, pub columns: Vec, pub emit_mode: Option, - pub sink_schema: Option, + pub sink_schema: Option, } impl ParseTo for CreateSinkStatement { diff --git a/src/sqlparser/tests/testdata/create.yaml b/src/sqlparser/tests/testdata/create.yaml index 5509ccad53a04..cde40d8a75da1 100644 --- a/src/sqlparser/tests/testdata/create.yaml +++ b/src/sqlparser/tests/testdata/create.yaml @@ -21,13 +21,13 @@ formatted_sql: CREATE SOURCE src FORMAT PLAIN ENCODE JSON - input: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.location = 'file://') formatted_sql: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.location = 'file://') - formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(SourceSchemaV2 { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "location", quote_style: None }]), value: SingleQuotedString("file://") }] }), source_watermarks: [] } }' + formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(ConnectorSchema { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "location", quote_style: None }]), value: SingleQuotedString("file://") }] }), source_watermarks: [] } }' - input: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.registry = 'http://') formatted_sql: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.registry = 'http://') - formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(SourceSchemaV2 { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "registry", quote_style: None }]), value: SingleQuotedString("http://") }] }), source_watermarks: [] } }' + formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(ConnectorSchema { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "registry", quote_style: None }]), value: SingleQuotedString("http://") }] }), source_watermarks: [] } }' - input: CREATE SOURCE bid (auction INTEGER, bidder INTEGER, price INTEGER, WATERMARK FOR auction AS auction - 1, "date_time" TIMESTAMP) with (connector = 'nexmark', nexmark.table.type = 'Bid', nexmark.split.num = '12', nexmark.min.event.gap.in.ns = '0') formatted_sql: CREATE SOURCE bid (auction INT, bidder INT, price INT, "date_time" TIMESTAMP, WATERMARK FOR auction AS auction - 1) WITH (connector = 'nexmark', nexmark.table.type = 'Bid', nexmark.split.num = '12', nexmark.min.event.gap.in.ns = '0') FORMAT NATIVE ENCODE NATIVE - formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: false, columns: [ColumnDef { name: Ident { value: "auction", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "bidder", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "price", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "date_time", quote_style: Some(''"'') }, data_type: Some(Timestamp(false)), collation: None, options: [] }], constraints: [], source_name: ObjectName([Ident { value: "bid", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "connector", quote_style: None }]), value: SingleQuotedString("nexmark") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "table", quote_style: None }, Ident { value: "type", quote_style: None }]), value: SingleQuotedString("Bid") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "split", quote_style: None }, Ident { value: "num", quote_style: None }]), value: SingleQuotedString("12") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "min", quote_style: None }, Ident { value: "event", quote_style: None }, Ident { value: "gap", quote_style: None }, Ident { value: "in", quote_style: None }, Ident { value: "ns", quote_style: None }]), value: SingleQuotedString("0") }]), source_schema: V2(SourceSchemaV2 { format: Native, row_encode: Native, row_options: [] }), source_watermarks: [SourceWatermark { column: Ident { value: "auction", quote_style: None }, expr: BinaryOp { left: Identifier(Ident { value: "auction", quote_style: None }), op: Minus, right: Value(Number("1")) } }] } }' + formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: false, columns: [ColumnDef { name: Ident { value: "auction", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "bidder", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "price", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "date_time", quote_style: Some(''"'') }, data_type: Some(Timestamp(false)), collation: None, options: [] }], constraints: [], source_name: ObjectName([Ident { value: "bid", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "connector", quote_style: None }]), value: SingleQuotedString("nexmark") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "table", quote_style: None }, Ident { value: "type", quote_style: None }]), value: SingleQuotedString("Bid") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "split", quote_style: None }, Ident { value: "num", quote_style: None }]), value: SingleQuotedString("12") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "min", quote_style: None }, Ident { value: "event", quote_style: None }, Ident { value: "gap", quote_style: None }, Ident { value: "in", quote_style: None }, Ident { value: "ns", quote_style: None }]), value: SingleQuotedString("0") }]), source_schema: V2(ConnectorSchema { format: Native, row_encode: Native, row_options: [] }), source_watermarks: [SourceWatermark { column: Ident { value: "auction", quote_style: None }, expr: BinaryOp { left: Identifier(Ident { value: "auction", quote_style: None }), op: Minus, right: Value(Number("1")) } }] } }' - input: CREATE TABLE T (v1 INT, v2 STRUCT) formatted_sql: CREATE TABLE T (v1 INT, v2 STRUCT) - input: CREATE TABLE T (v1 INT, v2 STRUCT>) From d6e5bec7192e851700bd1ff753c699f9ffa9d99a Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 26 Oct 2023 16:38:46 +0800 Subject: [PATCH 35/48] feat(sink): upsert avro with schema registry (#13007) --- ci/scripts/e2e-kafka-sink-test.sh | 8 ++ e2e_test/sink/kafka/avro.slt | 110 ++++++++++++++++++ e2e_test/sink/kafka/protobuf.slt | 9 ++ e2e_test/sink/kafka/register_schema.py | 48 ++++++++ src/connector/src/schema/avro.rs | 105 +++++++++++++++++ src/connector/src/schema/mod.rs | 4 + .../src/schema/schema_registry/client.rs | 18 ++- src/connector/src/sink/encoder/avro.rs | 90 +++++++++++--- src/connector/src/sink/encoder/mod.rs | 2 +- src/connector/src/sink/formatter/mod.rs | 44 ++++++- src/connector/src/sink/kafka.rs | 2 + src/connector/src/sink/kinesis.rs | 17 ++- src/connector/src/sink/pulsar.rs | 17 ++- src/connector/src/sink/redis.rs | 6 +- src/connector/src/test_data/all-types.avsc | 69 +++++++++++ src/frontend/src/handler/create_sink.rs | 2 +- 16 files changed, 515 insertions(+), 36 deletions(-) create mode 100644 e2e_test/sink/kafka/avro.slt create mode 100644 e2e_test/sink/kafka/register_schema.py create mode 100644 src/connector/src/schema/avro.rs create mode 100644 src/connector/src/test_data/all-types.avsc diff --git a/ci/scripts/e2e-kafka-sink-test.sh b/ci/scripts/e2e-kafka-sink-test.sh index 71a91f2d8fba9..d51482a912235 100755 --- a/ci/scripts/e2e-kafka-sink-test.sh +++ b/ci/scripts/e2e-kafka-sink-test.sh @@ -138,3 +138,11 @@ cp src/connector/src/test_data/proto_recursive/recursive.pb ./proto-recursive ./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-append-only-protobuf --create > /dev/null 2>&1 sqllogictest -p 4566 -d dev 'e2e_test/sink/kafka/protobuf.slt' ./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-append-only-protobuf --delete > /dev/null 2>&1 + +echo "testing avro" +python3 -m pip install requests confluent-kafka +python3 e2e_test/sink/kafka/register_schema.py 'http://message_queue:8081' 'test-rw-sink-upsert-avro-value' src/connector/src/test_data/all-types.avsc +python3 e2e_test/sink/kafka/register_schema.py 'http://message_queue:8081' 'test-rw-sink-upsert-avro-key' src/connector/src/test_data/all-types.avsc 'string_field,int32_field' +./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-upsert-avro --create > /dev/null 2>&1 +sqllogictest -p 4566 -d dev 'e2e_test/sink/kafka/avro.slt' +./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-upsert-avro --delete > /dev/null 2>&1 diff --git a/e2e_test/sink/kafka/avro.slt b/e2e_test/sink/kafka/avro.slt new file mode 100644 index 0000000000000..e1b09e3608e37 --- /dev/null +++ b/e2e_test/sink/kafka/avro.slt @@ -0,0 +1,110 @@ +statement ok +create table from_kafka with ( + connector = 'kafka', + topic = 'test-rw-sink-upsert-avro', + properties.bootstrap.server = 'message_queue:29092') +format upsert encode avro ( + schema.registry = 'http://message_queue:8081'); + +statement ok +create table into_kafka ( + bool_field bool, + string_field varchar, + bytes_field bytea, + float_field real, + double_field double precision, + int32_field int, + int64_field bigint, + record_field struct, + array_field int[][], + timestamp_micros_field timestamptz, + timestamp_millis_field timestamptz, + date_field date, + time_micros_field time, + time_millis_field time); + +statement ok +insert into into_kafka values + (true, 'Rising', 'a0', 3.5, 4.25, 22, 23, null, array[array[null, 3], null, array[7, null, 2]], '2006-01-02 15:04:05-07:00', null, null, '12:34:56.123456', null), + (false, 'Wave', 'ZDF', 1.5, null, 11, 12, row(null::int, 'foo'), null, null, '2006-01-02 15:04:05-07:00', '2021-04-01', null, '23:45:16.654321'); + +statement ok +flush; + +statement ok +create sink sink0 from into_kafka with ( + connector = 'kafka', + topic = 'test-rw-sink-upsert-avro', + properties.bootstrap.server = 'message_queue:29092', + primary_key = 'int32_field,string_field') +format upsert encode avro ( + schema.registry = 'http://message_queue:8081'); + +sleep 2s + +query TTTRRIITTTTTTTT +select + bool_field, + string_field, + bytes_field, + float_field, + double_field, + int32_field, + int64_field, + record_field, + array_field, + timestamp_micros_field, + timestamp_millis_field, + date_field, + time_micros_field, + time_millis_field from from_kafka; +---- +t Rising \x6130 3.5 4.25 22 23 NULL {{NULL,3},NULL,{7,NULL,2}} 2006-01-02 22:04:05+00:00 NULL NULL 12:34:56.123456 NULL +f Wave \x5a4446 1.5 NULL 11 12 (NULL,foo) NULL NULL 2006-01-02 22:04:05+00:00 2021-04-01 NULL 23:45:16.654 + +statement error SchemaFetchError +create sink sink_err from into_kafka with ( + connector = 'kafka', + topic = 'test-rw-sink-upsert-avro-err', + properties.bootstrap.server = 'message_queue:29092', + primary_key = 'int32_field,string_field') +format upsert encode avro ( + schema.registry = 'http://message_queue:8081'); + +statement error encode extra_column error: field not in avro +create sink sink_err as select 1 as extra_column, * from into_kafka with ( + connector = 'kafka', + topic = 'test-rw-sink-upsert-avro', + properties.bootstrap.server = 'message_queue:29092', + primary_key = 'int32_field,string_field') +format upsert encode avro ( + schema.registry = 'http://message_queue:8081'); + +statement error unrecognized +create sink sink_err from into_kafka with ( + connector = 'kafka', + topic = 'test-rw-sink-upsert-avro', + properties.bootstrap.server = 'message_queue:29092', + primary_key = 'int32_field,string_field') +format upsert encode avro ( + schema.registry = 'http://message_queue:8081', + schema.registry.name.strategy = 'typo'); + +statement error empty field key.message +create sink sink_err from into_kafka with ( + connector = 'kafka', + topic = 'test-rw-sink-upsert-avro', + properties.bootstrap.server = 'message_queue:29092', + primary_key = 'int32_field,string_field') +format upsert encode avro ( + schema.registry = 'http://message_queue:8081', + schema.registry.name.strategy = 'record_name_strategy'); + +statement ok +drop sink sink0; + +statement ok +drop table into_kafka; + +statement ok +drop table from_kafka; diff --git a/e2e_test/sink/kafka/protobuf.slt b/e2e_test/sink/kafka/protobuf.slt index 87ab884eddbde..2f827aeda9fc0 100644 --- a/e2e_test/sink/kafka/protobuf.slt +++ b/e2e_test/sink/kafka/protobuf.slt @@ -95,3 +95,12 @@ format plain encode protobuf ( force_append_only = true, schema.location = 's3:///risingwave/proto-recursive', message = 'recursive.AllTypes'); + +statement ok +drop sink sink0; + +statement ok +drop table into_kafka; + +statement ok +drop table from_kafka; diff --git a/e2e_test/sink/kafka/register_schema.py b/e2e_test/sink/kafka/register_schema.py new file mode 100644 index 0000000000000..2606e07bcb89b --- /dev/null +++ b/e2e_test/sink/kafka/register_schema.py @@ -0,0 +1,48 @@ +import sys +from confluent_kafka.schema_registry import SchemaRegistryClient, Schema + + +def main(): + url = sys.argv[1] + subject = sys.argv[2] + with open(sys.argv[3]) as f: + schema_str = f.read() + if 4 < len(sys.argv): + keys = sys.argv[4].split(',') + else: + keys = [] + + client = SchemaRegistryClient({"url": url}) + + if keys: + schema_str = select_keys(schema_str, keys) + else: + schema_str = remove_unsupported(schema_str) + schema = Schema(schema_str, 'AVRO') + client.register_schema(subject, schema) + + +def select_fields(schema_str, f): + import json + root = json.loads(schema_str) + if not isinstance(root, dict): + return schema_str + if root['type'] != 'record': + return schema_str + root['fields'] = f(root['fields']) + return json.dumps(root) + + +def remove_unsupported(schema_str): + return select_fields(schema_str, lambda fields: [f for f in fields if f['name'] not in {'unsupported', 'mon_day_sec_field'}]) + + +def select_keys(schema_str, keys): + def process(fields): + by_name = {f['name']: f for f in fields} + return [by_name[k] for k in keys] + return select_fields(schema_str, process) + + +if __name__ == '__main__': + main() diff --git a/src/connector/src/schema/avro.rs b/src/connector/src/schema/avro.rs new file mode 100644 index 0000000000000..fc12ba90f25aa --- /dev/null +++ b/src/connector/src/schema/avro.rs @@ -0,0 +1,105 @@ +// 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::BTreeMap; +use std::sync::Arc; + +use apache_avro::Schema as AvroSchema; +use risingwave_pb::catalog::PbSchemaRegistryNameStrategy; + +use super::schema_registry::{ + get_subject_by_strategy, handle_sr_list, name_strategy_from_str, Client, ConfluentSchema, + SchemaRegistryAuth, +}; +use super::{ + SchemaFetchError, KEY_MESSAGE_NAME_KEY, MESSAGE_NAME_KEY, NAME_STRATEGY_KEY, + SCHEMA_REGISTRY_KEY, +}; + +pub struct SchemaWithId { + pub schema: Arc, + pub id: i32, +} + +impl TryFrom for SchemaWithId { + type Error = SchemaFetchError; + + fn try_from(fetched: ConfluentSchema) -> Result { + let parsed = + AvroSchema::parse_str(&fetched.content).map_err(|e| SchemaFetchError(e.to_string()))?; + Ok(Self { + schema: Arc::new(parsed), + id: fetched.id, + }) + } +} + +/// Schema registry only +pub async fn fetch_schema( + format_options: &BTreeMap, + topic: &str, +) -> Result<(SchemaWithId, SchemaWithId), SchemaFetchError> { + let schema_location = format_options + .get(SCHEMA_REGISTRY_KEY) + .ok_or_else(|| SchemaFetchError(format!("{SCHEMA_REGISTRY_KEY} required")))? + .clone(); + let client_config = format_options.into(); + let name_strategy = format_options + .get(NAME_STRATEGY_KEY) + .map(|s| { + name_strategy_from_str(s) + .ok_or_else(|| SchemaFetchError(format!("unrecognized strategy {s}"))) + }) + .transpose()? + .unwrap_or_default(); + let key_record_name = format_options + .get(KEY_MESSAGE_NAME_KEY) + .map(std::ops::Deref::deref); + let val_record_name = format_options + .get(MESSAGE_NAME_KEY) + .map(std::ops::Deref::deref); + + let (key_schema, val_schema) = fetch_schema_inner( + &schema_location, + &client_config, + &name_strategy, + topic, + key_record_name, + val_record_name, + ) + .await + .map_err(|e| SchemaFetchError(e.to_string()))?; + + Ok((key_schema.try_into()?, val_schema.try_into()?)) +} + +async fn fetch_schema_inner( + schema_location: &str, + client_config: &SchemaRegistryAuth, + name_strategy: &PbSchemaRegistryNameStrategy, + topic: &str, + key_record_name: Option<&str>, + val_record_name: Option<&str>, +) -> Result<(ConfluentSchema, ConfluentSchema), risingwave_common::error::RwError> { + let urls = handle_sr_list(schema_location)?; + let client = Client::new(urls, client_config)?; + + let key_subject = get_subject_by_strategy(name_strategy, topic, key_record_name, true)?; + let key_schema = client.get_schema_by_subject(&key_subject).await?; + + let val_subject = get_subject_by_strategy(name_strategy, topic, val_record_name, false)?; + let val_schema = client.get_schema_by_subject(&val_subject).await?; + + Ok((key_schema, val_schema)) +} diff --git a/src/connector/src/schema/mod.rs b/src/connector/src/schema/mod.rs index 3c8694fc8f359..75a521a50ec55 100644 --- a/src/connector/src/schema/mod.rs +++ b/src/connector/src/schema/mod.rs @@ -12,11 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod avro; pub mod protobuf; pub mod schema_registry; const MESSAGE_NAME_KEY: &str = "message"; +const KEY_MESSAGE_NAME_KEY: &str = "key.message"; const SCHEMA_LOCATION_KEY: &str = "schema.location"; +const SCHEMA_REGISTRY_KEY: &str = "schema.registry"; +const NAME_STRATEGY_KEY: &str = "schema.registry.name.strategy"; #[derive(Debug)] pub struct SchemaFetchError(pub String); diff --git a/src/connector/src/schema/schema_registry/client.rs b/src/connector/src/schema/schema_registry/client.rs index 591478752d926..6d5778cd367c4 100644 --- a/src/connector/src/schema/schema_registry/client.rs +++ b/src/connector/src/schema/schema_registry/client.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{HashMap, HashSet}; +use std::collections::{BTreeMap, HashMap, HashSet}; use std::fmt::Debug; use std::sync::Arc; @@ -43,6 +43,18 @@ impl From<&HashMap> for SchemaRegistryAuth { } } +impl From<&BTreeMap> for SchemaRegistryAuth { + fn from(props: &BTreeMap) -> Self { + const SCHEMA_REGISTRY_USERNAME: &str = "schema.registry.username"; + const SCHEMA_REGISTRY_PASSWORD: &str = "schema.registry.password"; + + SchemaRegistryAuth { + username: props.get(SCHEMA_REGISTRY_USERNAME).cloned(), + password: props.get(SCHEMA_REGISTRY_PASSWORD).cloned(), + } + } +} + /// An client for communication with schema registry #[derive(Debug)] pub struct Client { @@ -123,7 +135,9 @@ impl Client { Err(RwError::from(ProtocolError(format!( "all request confluent registry all timeout, req path {:?}, urls {:?}, err: {:?}", - path, self.url, errs + path, + self.url, + errs.iter().map(|e| e.to_string()).collect_vec() )))) } diff --git a/src/connector/src/sink/encoder/avro.rs b/src/connector/src/sink/encoder/avro.rs index fc2db75eb4c38..7aff29dbb43b7 100644 --- a/src/connector/src/sink/encoder/avro.rs +++ b/src/connector/src/sink/encoder/avro.rs @@ -16,7 +16,6 @@ use std::sync::Arc; use apache_avro::schema::Schema as AvroSchema; use apache_avro::types::{Record, Value}; -use apache_avro::Writer; use risingwave_common::catalog::Schema; use risingwave_common::row::Row; use risingwave_common::types::{DataType, DatumRef, ScalarRefImpl, StructType}; @@ -30,6 +29,28 @@ pub struct AvroEncoder { schema: Schema, col_indices: Option>, avro_schema: Arc, + header: AvroHeader, +} + +#[derive(Debug, Clone, Copy)] +pub enum AvroHeader { + None, + /// + /// + /// * C3 01 + /// * 8-byte little-endian CRC-64-AVRO fingerprint + SingleObject, + /// + /// + /// * 4F 62 6A 01 + /// * schema + /// * 16-byte random sync marker + ContainerFile, + /// + /// + /// * 00 + /// * 4-byte big-endian schema ID + ConfluentSchemaRegistry(i32), } impl AvroEncoder { @@ -37,6 +58,7 @@ impl AvroEncoder { schema: Schema, col_indices: Option>, avro_schema: Arc, + header: AvroHeader, ) -> SinkResult { match &col_indices { Some(col_indices) => validate_fields( @@ -59,12 +81,19 @@ impl AvroEncoder { schema, col_indices, avro_schema, + header, }) } } +pub struct AvroEncoded { + value: Value, + schema: Arc, + header: AvroHeader, +} + impl RowEncoder for AvroEncoder { - type Output = (Value, Arc); + type Output = AvroEncoded; fn schema(&self) -> &Schema { &self.schema @@ -86,16 +115,32 @@ impl RowEncoder for AvroEncoder { }), &self.avro_schema, )?; - Ok((record.into(), self.avro_schema.clone())) + Ok(AvroEncoded { + value: record.into(), + schema: self.avro_schema.clone(), + header: self.header, + }) } } -impl SerTo> for (Value, Arc) { +impl SerTo> for AvroEncoded { fn ser_to(self) -> SinkResult> { - let mut w = Writer::new(&self.1, Vec::new()); - w.append(self.0) - .and_then(|_| w.into_inner()) - .map_err(|e| crate::sink::SinkError::Encode(e.to_string())) + use bytes::BufMut as _; + + let AvroHeader::ConfluentSchemaRegistry(schema_id) = self.header else { + return Err(crate::sink::SinkError::Encode(format!( + "{:?} unsupported yet", + self.header + ))); + }; + let raw = apache_avro::to_avro_datum(&self.schema, self.value) + .map_err(|e| crate::sink::SinkError::Encode(e.to_string()))?; + let mut buf = Vec::with_capacity(1 + 4 + raw.len()); + buf.put_u8(0); + buf.put_i32(schema_id); + buf.put_slice(&raw); + + Ok(buf) } } @@ -616,8 +661,16 @@ mod tests { .unwrap(); let mut record = Record::new(&avro_schema).unwrap(); record.put("f0", Value::String("2".into())); - let res: SinkResult> = (Value::from(record), Arc::new(avro_schema)).ser_to(); - assert_eq!(res.unwrap_err().to_string(), "Encode error: Value does not match schema: Reason: Unsupported value-schema combination"); + let res: SinkResult> = AvroEncoded { + value: Value::from(record), + schema: Arc::new(avro_schema), + header: AvroHeader::ConfluentSchemaRegistry(42), + } + .ser_to(); + assert_eq!( + res.unwrap_err().to_string(), + "Encode error: Value does not match schema" + ); } #[test] @@ -634,6 +687,7 @@ mod tests { ) .unwrap(); let avro_schema = Arc::new(avro_schema); + let header = AvroHeader::None; let schema = Schema::new(vec![ Field::with_name(DataType::Int64, "opt"), @@ -643,10 +697,10 @@ mod tests { Some(ScalarImpl::Int64(31)), Some(ScalarImpl::Int32(15)), ]); - let encoder = AvroEncoder::new(schema, None, avro_schema.clone()).unwrap(); + let encoder = AvroEncoder::new(schema, None, avro_schema.clone(), header).unwrap(); let actual = encoder.encode(row).unwrap(); assert_eq!( - actual.0, + actual.value, Value::Record(vec![ ("req".into(), Value::Int(15)), ("opt".into(), Value::Union(1, Value::Long(31).into())), @@ -655,10 +709,10 @@ mod tests { let schema = Schema::new(vec![Field::with_name(DataType::Int32, "req")]); let row = OwnedRow::new(vec![Some(ScalarImpl::Int32(15))]); - let encoder = AvroEncoder::new(schema, None, avro_schema.clone()).unwrap(); + let encoder = AvroEncoder::new(schema, None, avro_schema.clone(), header).unwrap(); let actual = encoder.encode(row).unwrap(); assert_eq!( - actual.0, + actual.value, Value::Record(vec![ ("req".into(), Value::Int(15)), ("opt".into(), Value::Union(0, Value::Null.into())), @@ -666,7 +720,7 @@ mod tests { ); let schema = Schema::new(vec![Field::with_name(DataType::Int64, "opt")]); - let Err(err) = AvroEncoder::new(schema, None, avro_schema.clone()) else { + let Err(err) = AvroEncoder::new(schema, None, avro_schema.clone(), header) else { panic!() }; assert_eq!( @@ -679,7 +733,7 @@ mod tests { Field::with_name(DataType::Int32, "req"), Field::with_name(DataType::Varchar, "extra"), ]); - let Err(err) = AvroEncoder::new(schema, None, avro_schema.clone()) else { + let Err(err) = AvroEncoder::new(schema, None, avro_schema.clone(), header) else { panic!() }; assert_eq!( @@ -689,7 +743,7 @@ mod tests { let avro_schema = AvroSchema::parse_str(r#"["null", "long"]"#).unwrap(); let schema = Schema::new(vec![Field::with_name(DataType::Int64, "opt")]); - let Err(err) = AvroEncoder::new(schema, None, avro_schema.into()) else { + let Err(err) = AvroEncoder::new(schema, None, avro_schema.into(), header) else { panic!() }; assert_eq!( @@ -843,7 +897,7 @@ mod tests { /// The encoder is not using these buggy calls and is already tested above. #[test] fn test_encode_avro_lib_bug() { - use apache_avro::Reader; + use apache_avro::{Reader, Writer}; // a record with 2 optional int fields let avro_schema = AvroSchema::parse_str( diff --git a/src/connector/src/sink/encoder/mod.rs b/src/connector/src/sink/encoder/mod.rs index 83b2ab4f09df0..b55fd534d5eb3 100644 --- a/src/connector/src/sink/encoder/mod.rs +++ b/src/connector/src/sink/encoder/mod.rs @@ -25,7 +25,7 @@ mod json; mod proto; pub mod template; -pub use avro::AvroEncoder; +pub use avro::{AvroEncoder, AvroHeader}; pub use json::JsonEncoder; pub use proto::ProtoEncoder; diff --git a/src/connector/src/sink/formatter/mod.rs b/src/connector/src/sink/formatter/mod.rs index 17cb708292890..1e165268300fa 100644 --- a/src/connector/src/sink/formatter/mod.rs +++ b/src/connector/src/sink/formatter/mod.rs @@ -30,7 +30,9 @@ use super::catalog::{SinkEncode, SinkFormat, SinkFormatDesc}; use super::encoder::template::TemplateEncoder; use super::encoder::KafkaConnectParams; use super::redis::{KEY_FORMAT, VALUE_FORMAT}; -use crate::sink::encoder::{JsonEncoder, ProtoEncoder, TimestampHandlingMode}; +use crate::sink::encoder::{ + AvroEncoder, AvroHeader, JsonEncoder, ProtoEncoder, TimestampHandlingMode, +}; /// Transforms a `StreamChunk` into a sequence of key-value pairs according a specific format, /// for example append-only, upsert or debezium. @@ -67,6 +69,7 @@ pub enum SinkFormatterImpl { AppendOnlyJson(AppendOnlyFormatter), AppendOnlyProto(AppendOnlyFormatter), UpsertJson(UpsertFormatter), + UpsertAvro(UpsertFormatter), DebeziumJson(DebeziumJsonFormatter), AppendOnlyTemplate(AppendOnlyFormatter), UpsertTemplate(UpsertFormatter), @@ -79,6 +82,7 @@ impl SinkFormatterImpl { pk_indices: Vec, db_name: String, sink_from_name: String, + topic: &str, ) -> Result { let err_unsupported = || { Err(SinkError::Config(anyhow!( @@ -211,7 +215,27 @@ impl SinkFormatterImpl { val_encoder, ))) } - _ => err_unsupported(), + SinkEncode::Avro => { + let (key_schema, val_schema) = + crate::schema::avro::fetch_schema(&format_desc.options, topic) + .await + .map_err(|e| SinkError::Config(anyhow!("{e:?}")))?; + let key_encoder = AvroEncoder::new( + schema.clone(), + Some(pk_indices), + key_schema.schema, + AvroHeader::ConfluentSchemaRegistry(key_schema.id), + )?; + let val_encoder = AvroEncoder::new( + schema.clone(), + None, + val_schema.schema, + AvroHeader::ConfluentSchemaRegistry(val_schema.id), + )?; + let formatter = UpsertFormatter::new(key_encoder, val_encoder); + Ok(SinkFormatterImpl::UpsertAvro(formatter)) + } + SinkEncode::Protobuf => err_unsupported(), } } } @@ -225,6 +249,22 @@ macro_rules! dispatch_sink_formatter_impl { SinkFormatterImpl::AppendOnlyJson($name) => $body, SinkFormatterImpl::AppendOnlyProto($name) => $body, SinkFormatterImpl::UpsertJson($name) => $body, + SinkFormatterImpl::UpsertAvro($name) => $body, + SinkFormatterImpl::DebeziumJson($name) => $body, + SinkFormatterImpl::AppendOnlyTemplate($name) => $body, + SinkFormatterImpl::UpsertTemplate($name) => $body, + } + }; +} + +#[macro_export] +macro_rules! dispatch_sink_formatter_str_key_impl { + ($impl:expr, $name:ident, $body:expr) => { + match $impl { + SinkFormatterImpl::AppendOnlyJson($name) => $body, + SinkFormatterImpl::AppendOnlyProto($name) => $body, + SinkFormatterImpl::UpsertJson($name) => $body, + SinkFormatterImpl::UpsertAvro(_) => unreachable!(), SinkFormatterImpl::DebeziumJson($name) => $body, SinkFormatterImpl::AppendOnlyTemplate($name) => $body, SinkFormatterImpl::UpsertTemplate($name) => $body, diff --git a/src/connector/src/sink/kafka.rs b/src/connector/src/sink/kafka.rs index f77b2b0a88c36..07709f182dc47 100644 --- a/src/connector/src/sink/kafka.rs +++ b/src/connector/src/sink/kafka.rs @@ -312,6 +312,7 @@ impl Sink for KafkaSink { self.pk_indices.clone(), self.db_name.clone(), self.sink_from_name.clone(), + &self.config.common.topic, ) .await?; let max_delivery_buffer_size = (self @@ -343,6 +344,7 @@ impl Sink for KafkaSink { self.pk_indices.clone(), self.db_name.clone(), self.sink_from_name.clone(), + &self.config.common.topic, ) .await?; diff --git a/src/connector/src/sink/kinesis.rs b/src/connector/src/sink/kinesis.rs index 605edde3b1eb0..03e044ad37b91 100644 --- a/src/connector/src/sink/kinesis.rs +++ b/src/connector/src/sink/kinesis.rs @@ -29,7 +29,7 @@ use tokio_retry::Retry; use super::catalog::SinkFormatDesc; use super::SinkParam; use crate::common::KinesisCommon; -use crate::dispatch_sink_formatter_impl; +use crate::dispatch_sink_formatter_str_key_impl; use crate::sink::catalog::desc::SinkDesc; use crate::sink::formatter::SinkFormatterImpl; use crate::sink::log_store::DeliveryFutureManagerAddFuture; @@ -94,6 +94,7 @@ impl Sink for KinesisSink { self.pk_indices.clone(), self.db_name.clone(), self.sink_from_name.clone(), + &self.config.common.stream_name, ) .await?; @@ -161,9 +162,15 @@ impl KinesisSinkWriter { db_name: String, sink_from_name: String, ) -> Result { - let formatter = - SinkFormatterImpl::new(format_desc, schema, pk_indices, db_name, sink_from_name) - .await?; + let formatter = SinkFormatterImpl::new( + format_desc, + schema, + pk_indices, + db_name, + sink_from_name, + &config.common.stream_name, + ) + .await?; let client = config .common .build_client() @@ -228,7 +235,7 @@ impl AsyncTruncateSinkWriter for KinesisSinkWriter { chunk: StreamChunk, _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, ) -> Result<()> { - dispatch_sink_formatter_impl!( + dispatch_sink_formatter_str_key_impl!( &self.formatter, formatter, self.payload_writer.write_chunk(chunk, formatter).await diff --git a/src/connector/src/sink/pulsar.rs b/src/connector/src/sink/pulsar.rs index 9eb57c1ae0771..04da204ef79e7 100644 --- a/src/connector/src/sink/pulsar.rs +++ b/src/connector/src/sink/pulsar.rs @@ -36,7 +36,7 @@ use crate::sink::writer::{ AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, FormattedSink, }; use crate::sink::{DummySinkCommitCoordinator, Result}; -use crate::{deserialize_duration_from_string, dispatch_sink_formatter_impl}; +use crate::{deserialize_duration_from_string, dispatch_sink_formatter_str_key_impl}; pub const PULSAR_SINK: &str = "pulsar"; @@ -194,6 +194,7 @@ impl Sink for PulsarSink { self.downstream_pk.clone(), self.db_name.clone(), self.sink_from_name.clone(), + &self.config.common.topic, ) .await?; @@ -237,9 +238,15 @@ impl PulsarSinkWriter { db_name: String, sink_from_name: String, ) -> Result { - let formatter = - SinkFormatterImpl::new(format_desc, schema, downstream_pk, db_name, sink_from_name) - .await?; + let formatter = SinkFormatterImpl::new( + format_desc, + schema, + downstream_pk, + db_name, + sink_from_name, + &config.common.topic, + ) + .await?; let pulsar = config.common.build_client().await?; let producer = build_pulsar_producer(&pulsar, &config).await?; Ok(Self { @@ -322,7 +329,7 @@ impl AsyncTruncateSinkWriter for PulsarSinkWriter { chunk: StreamChunk, add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, ) -> Result<()> { - dispatch_sink_formatter_impl!(&self.formatter, formatter, { + dispatch_sink_formatter_str_key_impl!(&self.formatter, formatter, { let mut payload_writer = PulsarPayloadWriter { producer: &mut self.producer, add_future, diff --git a/src/connector/src/sink/redis.rs b/src/connector/src/sink/redis.rs index af3ec3b981620..910582b9662b7 100644 --- a/src/connector/src/sink/redis.rs +++ b/src/connector/src/sink/redis.rs @@ -28,7 +28,7 @@ use super::encoder::template::TemplateEncoder; use super::formatter::SinkFormatterImpl; use super::writer::FormattedSink; use super::{SinkError, SinkParam}; -use crate::dispatch_sink_formatter_impl; +use crate::dispatch_sink_formatter_str_key_impl; use crate::sink::log_store::DeliveryFutureManagerAddFuture; use crate::sink::writer::{ AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, @@ -224,6 +224,7 @@ impl RedisSinkWriter { pk_indices.clone(), db_name, sink_from_name, + "NO_TOPIC", ) .await?; @@ -248,6 +249,7 @@ impl RedisSinkWriter { pk_indices.clone(), "d1".to_string(), "t1".to_string(), + "NO_TOPIC", ) .await?; Ok(Self { @@ -266,7 +268,7 @@ impl AsyncTruncateSinkWriter for RedisSinkWriter { chunk: StreamChunk, _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, ) -> Result<()> { - dispatch_sink_formatter_impl!(&self.formatter, formatter, { + dispatch_sink_formatter_str_key_impl!(&self.formatter, formatter, { self.payload_writer.write_chunk(chunk, formatter).await }) } diff --git a/src/connector/src/test_data/all-types.avsc b/src/connector/src/test_data/all-types.avsc new file mode 100644 index 0000000000000..3fea69bbef4ca --- /dev/null +++ b/src/connector/src/test_data/all-types.avsc @@ -0,0 +1,69 @@ +{ + "type": "record", + "name": "AllTypes", + "fields": [ + {"name": "bool_field", "type": ["null", "boolean"]}, + {"name": "string_field", "type": ["null", "string"]}, + {"name": "bytes_field", "type": ["null", "bytes"]}, + {"name": "float_field", "type": ["null", "float"]}, + {"name": "double_field", "type": ["null", "double"]}, + {"name": "int32_field", "type": ["null", "int"]}, + {"name": "int64_field", "type": ["null", "long"]}, + {"name": "record_field", "type": ["null", { + "type": "record", + "name": "Nested", + "fields": [ + {"name": "id", "type": ["null", "int"]}, + {"name": "name", "type": ["null", "string"]} + ] + }]}, + {"name": "array_field", "type": ["null", { + "type": "array", + "items": ["null", { + "type": "array", + "items": ["null", "int"] + }] + }]}, + {"name": "timestamp_micros_field", "type": ["null", {"type": "long", "logicalType": "timestamp-micros"}]}, + {"name": "timestamp_millis_field", "type": ["null", {"type": "long", "logicalType": "timestamp-millis"}]}, + {"name": "date_field", "type": ["null", {"type": "int", "logicalType": "date"}]}, + {"name": "time_micros_field", "type": ["null", {"type": "long", "logicalType": "time-micros"}]}, + {"name": "time_millis_field", "type": ["null", {"type": "int", "logicalType": "time-millis"}]}, + {"name": "mon_day_sec_field", "type": ["null", { + "type": "fixed", + "name": "Duration", + "size": 12, + "logicalType": "duration" + }]}, + {"name": "unsupported", "type": ["null", { + "type": "record", + "name": "Unsupported", + "fields": [ + {"name": "enum_field", "type": ["null", { + "type": "enum", + "name": "Suit", + "symbols": ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"] + }]}, + {"name": "map_field", "type": ["null", { + "type": "map", + "values": ["null", "string"] + }]}, + {"name": "union_field", "type": ["null", "string", "double", "boolean"]}, + {"name": "fixed_field", "type": ["null", { + "type": "fixed", + "name": "Int256", + "size": 32 + }]}, + {"name": "decimal_field", "type": ["null", { + "type": "bytes", + "logicalType": "decimal", + "precision": 38, + "scale": 10 + }]}, + {"name": "uuid_field", "type": ["null", {"type": "string", "logicalType": "uuid"}]}, + {"name": "local_micros_field", "type": ["null", {"type": "long", "logicalType": "local-timestamp-micros"}]}, + {"name": "local_millis_field", "type": ["null", {"type": "long", "logicalType": "local-timestamp-millis"}]} + ] + }]} + ] +} diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 2ab987308a4f5..85d867d538e7f 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -269,7 +269,7 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock hashmap!( Format::Plain => vec![Encode::Json, Encode::Protobuf], - Format::Upsert => vec![Encode::Json], + Format::Upsert => vec![Encode::Json, Encode::Avro], Format::Debezium => vec![Encode::Json], ), KinesisSink::SINK_NAME => hashmap!( From b0f266b0a90c2a7a1c7dcd566d5fbc94f38751a2 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 26 Oct 2023 17:02:56 +0800 Subject: [PATCH 36/48] feat(mysql-cdc): deprecate `server.id` WITH option and generate it internally (#13031) --- Cargo.lock | 1 + ci/scripts/e2e-source-test.sh | 1 + .../cdc_inline/mysql/mysql_create_drop.slt | 133 ++++++++++++++++++ src/connector/src/macros.rs | 5 +- src/connector/src/source/base.rs | 2 +- src/connector/src/source/cdc/mod.rs | 11 ++ src/connector/src/source/cdc/source/reader.rs | 3 + src/connector/src/source/mod.rs | 2 +- src/meta/service/Cargo.toml | 1 + src/meta/service/src/ddl_service.rs | 29 ++++ src/source/src/connector_source.rs | 2 +- 11 files changed, 186 insertions(+), 4 deletions(-) create mode 100644 e2e_test/source/cdc_inline/mysql/mysql_create_drop.slt diff --git a/Cargo.lock b/Cargo.lock index 2f2ee2991b10f..dbce3dfae9f3b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7778,6 +7778,7 @@ dependencies = [ "itertools 0.11.0", "madsim-tokio", "madsim-tonic", + "rand", "regex", "risingwave_common", "risingwave_connector", diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 59618d24641aa..2edd099e31187 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -81,6 +81,7 @@ echo "waiting for connector node to start" wait_for_connector_node_start echo "--- inline cdc test" +export MYSQL_HOST=mysql MYSQL_TCP_PORT=3306 MYSQL_PWD=123456 sqllogictest -p 4566 -d dev './e2e_test/source/cdc_inline/**/*.slt' echo "--- mysql & postgres cdc validate test" diff --git a/e2e_test/source/cdc_inline/mysql/mysql_create_drop.slt b/e2e_test/source/cdc_inline/mysql/mysql_create_drop.slt new file mode 100644 index 0000000000000..071fe0ef2da83 --- /dev/null +++ b/e2e_test/source/cdc_inline/mysql/mysql_create_drop.slt @@ -0,0 +1,133 @@ +# create and drop CDC mysql tables concurrently + +control substitution on + +statement ok +ALTER SYSTEM SET max_concurrent_creating_streaming_jobs TO 4; + +system ok +mysql --protocol=tcp -u root -e " + DROP DATABASE IF EXISTS testdb1; CREATE DATABASE testdb1; + USE testdb1; + CREATE TABLE tt1 (v1 int primary key, v2 timestamp); + INSERT INTO tt1 VALUES (1, '2023-10-23 10:00:00'); + CREATE TABLE tt2 (v1 int primary key, v2 timestamp); + INSERT INTO tt2 VALUES (2, '2023-10-23 11:00:00'); + CREATE TABLE tt3 (v1 int primary key, v2 timestamp); + INSERT INTO tt3 VALUES (3, '2023-10-23 12:00:00'); + CREATE TABLE tt4 (v1 int primary key, v2 timestamp); + INSERT INTO tt4 VALUES (4, '2023-10-23 13:00:00'); + CREATE TABLE tt5 (v1 int primary key, v2 timestamp); + INSERT INTO tt5 VALUES (5, '2023-10-23 14:00:00');" + +statement ok +create table tt1 (v1 int, + v2 timestamptz, + PRIMARY KEY (v1) +) with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'dbz', + password = '${MYSQL_PWD:}', + database.name = 'testdb1', + table.name = 'tt1', +); + +statement ok +create table tt2 (v1 int, + v2 timestamptz, + PRIMARY KEY (v1) +) with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'dbz', + password = '${MYSQL_PWD:}', + database.name = 'testdb1', + table.name = 'tt2', +); + +statement ok +create table tt3 (v1 int, + v2 timestamptz, + PRIMARY KEY (v1) +) with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'dbz', + password = '${MYSQL_PWD:}', + database.name = 'testdb1', + table.name = 'tt3', +); + +statement ok +create table tt4 (v1 int, + v2 timestamptz, + PRIMARY KEY (v1) +) with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'dbz', + password = '${MYSQL_PWD:}', + database.name = 'testdb1', + table.name = 'tt4', +); + +statement ok +create table tt5 (v1 int, + v2 timestamptz, + PRIMARY KEY (v1) +) with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'dbz', + password = '${MYSQL_PWD:}', + database.name = 'testdb1', + table.name = 'tt5', +); + +sleep 3s + +query IT +select * from tt1; +---- +1 2023-10-23 10:00:00+00:00 + +query IT +select * from tt2; +---- +2 2023-10-23 11:00:00+00:00 + +query IT +select * from tt3; +---- +3 2023-10-23 12:00:00+00:00 + +query IT +select * from tt4; +---- +4 2023-10-23 13:00:00+00:00 + +query IT +select * from tt5; +---- +5 2023-10-23 14:00:00+00:00 + +statement ok +drop table tt1; + +statement ok +drop table tt2; + +statement ok +drop table tt3; + +statement ok +drop table tt4; + +statement ok +drop table tt5; diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 62a3cfdcd9682..fdc3ed8867297 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -234,12 +234,13 @@ macro_rules! impl_cdc_source_type { $( $cdc_source_type, )* + Unspecified, } impl From for CdcSourceType { fn from(value: PbSourceType) -> Self { match value { - PbSourceType::Unspecified => unreachable!(), + PbSourceType::Unspecified => CdcSourceType::Unspecified, $( PbSourceType::$cdc_source_type => CdcSourceType::$cdc_source_type, )* @@ -253,8 +254,10 @@ macro_rules! impl_cdc_source_type { $( CdcSourceType::$cdc_source_type => PbSourceType::$cdc_source_type, )* + CdcSourceType::Unspecified => PbSourceType::Unspecified, } } } + } } diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 6a8cd12ce9fac..49dc3b5d87119 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -51,7 +51,7 @@ use crate::{ const SPLIT_TYPE_FIELD: &str = "split_type"; const SPLIT_INFO_FIELD: &str = "split_info"; -const UPSTREAM_SOURCE_KEY: &str = "connector"; +pub const UPSTREAM_SOURCE_KEY: &str = "connector"; pub trait TryFromHashmap: Sized { fn try_from_hashmap(props: HashMap) -> Result; diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index 1d795a7141e84..d55273bf725db 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -42,6 +42,17 @@ pub trait CdcSourceTypeTrait: Send + Sync + Clone + 'static { for_all_classified_sources!(impl_cdc_source_type); +impl<'a> From<&'a str> for CdcSourceType { + fn from(name: &'a str) -> Self { + match name { + MYSQL_CDC_CONNECTOR => CdcSourceType::Mysql, + POSTGRES_CDC_CONNECTOR => CdcSourceType::Postgres, + CITUS_CDC_CONNECTOR => CdcSourceType::Citus, + _ => CdcSourceType::Unspecified, + } + } +} + #[derive(Clone, Debug, Default)] pub struct CdcProperties { /// Properties specified in the WITH clause by user diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index 7410834ce1daa..4d25d82c106c3 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -87,6 +87,9 @@ impl SplitReader for CdcSplitReader { parser_config, source_ctx, }), + CdcSourceType::Unspecified => { + unreachable!(); + } } } diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index 869b7089ac271..c866ed6c3c223 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -24,7 +24,7 @@ pub mod monitor; pub mod nats; pub mod nexmark; pub mod pulsar; -pub use base::*; +pub use base::{UPSTREAM_SOURCE_KEY, *}; pub(crate) use common::*; pub use google_pubsub::GOOGLE_PUBSUB_CONNECTOR; pub use kafka::KAFKA_CONNECTOR; diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml index 87b293f64a5e6..d1c08a642c8ca 100644 --- a/src/meta/service/Cargo.toml +++ b/src/meta/service/Cargo.toml @@ -19,6 +19,7 @@ async-trait = "0.1" either = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } itertools = "0.11" +rand = "0.8" regex = "1" risingwave_common = { workspace = true } risingwave_connector = { workspace = true } diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 6f08ebfb18d17..5f73ffb815117 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -16,9 +16,12 @@ use std::collections::HashMap; use std::sync::Arc; use anyhow::anyhow; +use rand::Rng; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::stream_graph_visitor::visit_fragment; use risingwave_connector::sink::catalog::SinkId; +use risingwave_connector::source::cdc::CdcSourceType; +use risingwave_connector::source::UPSTREAM_SOURCE_KEY; use risingwave_pb::catalog::connection::private_link_service::{ PbPrivateLinkProvider, PrivateLinkProvider, }; @@ -428,6 +431,16 @@ impl DdlService for DdlServiceImpl { // Generate source id. let source_id = self.gen_unique_id::<{ IdCategory::Table }>().await?; // TODO: Use source category fill_table_source(source, source_id, &mut mview, table_id, &mut fragment_graph); + + // Modify properties for cdc sources if needed + if let Some(connector) = source.properties.get(UPSTREAM_SOURCE_KEY) { + if matches!( + CdcSourceType::from(connector.as_str()), + CdcSourceType::Mysql + ) { + fill_cdc_mysql_server_id(&mut fragment_graph); + } + } } let mut stream_job = StreamingJob::Table(source, mview); @@ -827,3 +840,19 @@ fn fill_table_source( table.optional_associated_source_id = Some(OptionalAssociatedSourceId::AssociatedSourceId(source_id)); } + +// `server.id` (in the range from 1 to 2^32 - 1). This value MUST be unique across whole replication +// group (that is, different from any other server id being used by any master or slave) +fn fill_cdc_mysql_server_id(fragment_graph: &mut PbStreamFragmentGraph) { + for fragment in fragment_graph.fragments.values_mut() { + visit_fragment(fragment, |node_body| { + if let NodeBody::Source(source_node) = node_body { + let props = &mut source_node.source_inner.as_mut().unwrap().properties; + let rand_server_id = rand::thread_rng().gen_range(1..u32::MAX); + props + .entry("server.id".to_string()) + .or_insert(rand_server_id.to_string()); + } + }); + } +} diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 733ba6a8c4a83..31ee763d2a0b9 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -155,7 +155,7 @@ impl ConnectorSource { let to_reader_splits = splits.into_iter().map(|split| vec![split]); try_join_all(to_reader_splits.into_iter().map(|splits| { - tracing::debug!("spawning connector split reader for split {:?}", splits); + tracing::debug!(?splits, ?prop, "spawning connector split reader"); let props = prop.clone(); let data_gen_columns = data_gen_columns.clone(); let parser_config = parser_config.clone(); From de8d2172feb03a338f334ce257e7c014c5384a93 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 26 Oct 2023 18:23:14 +0800 Subject: [PATCH 37/48] fix(standalone): hide etcd pwd in logs (#13034) --- Cargo.lock | 7 +++++++ src/cmd_all/src/standalone.rs | 6 +++--- src/meta/node/Cargo.toml | 1 + src/meta/node/src/lib.rs | 9 +++++++-- 4 files changed, 18 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index dbce3dfae9f3b..f03347d622208 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6645,6 +6645,12 @@ dependencies = [ "zstd-sys", ] +[[package]] +name = "redact" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b97c0a6319ae55341eb213c8ef97002630a3a5bd6f287f0124d077121d3f2a5" + [[package]] name = "redis" version = "0.23.3" @@ -7753,6 +7759,7 @@ dependencies = [ "madsim-tokio", "madsim-tonic", "prometheus-http-query", + "redact", "regex", "risingwave_common", "risingwave_common_heap_profiling", diff --git a/src/cmd_all/src/standalone.rs b/src/cmd_all/src/standalone.rs index 8ebe2c7112c49..15a46d789ff3d 100644 --- a/src/cmd_all/src/standalone.rs +++ b/src/cmd_all/src/standalone.rs @@ -195,7 +195,7 @@ mod test { // Test parsing into standalone-level opts. let raw_opts = " --compute-opts=--listen-addr 127.0.0.1:8000 --total-memory-bytes 34359738368 --parallelism 10 ---meta-opts=--advertise-addr 127.0.0.1:9999 --data-directory \"some path with spaces\" --listen-addr 127.0.0.1:8001 +--meta-opts=--advertise-addr 127.0.0.1:9999 --data-directory \"some path with spaces\" --listen-addr 127.0.0.1:8001 --etcd-password 1234 --frontend-opts=--config-path=src/config/original.toml --prometheus-listener-addr=127.0.0.1:1234 --config-path=src/config/test.toml @@ -203,7 +203,7 @@ mod test { let actual = StandaloneOpts::parse_from(raw_opts.lines()); let opts = StandaloneOpts { compute_opts: Some("--listen-addr 127.0.0.1:8000 --total-memory-bytes 34359738368 --parallelism 10".into()), - meta_opts: Some("--advertise-addr 127.0.0.1:9999 --data-directory \"some path with spaces\" --listen-addr 127.0.0.1:8001".into()), + meta_opts: Some("--advertise-addr 127.0.0.1:9999 --data-directory \"some path with spaces\" --listen-addr 127.0.0.1:8001 --etcd-password 1234".into()), frontend_opts: Some("--config-path=src/config/original.toml".into()), compactor_opts: None, prometheus_listener_addr: Some("127.0.0.1:1234".into()), @@ -228,7 +228,7 @@ mod test { etcd_endpoints: "", etcd_auth: false, etcd_username: "", - etcd_password: "", + etcd_password: [REDACTED alloc::string::String], sql_endpoint: None, dashboard_ui_path: None, prometheus_endpoint: None, diff --git a/src/meta/node/Cargo.toml b/src/meta/node/Cargo.toml index 84793a74591c8..dcfa053fdfc4b 100644 --- a/src/meta/node/Cargo.toml +++ b/src/meta/node/Cargo.toml @@ -21,6 +21,7 @@ etcd-client = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } itertools = "0.11" prometheus-http-query = "0.7" +redact = "0.1.5" regex = "1" risingwave_common = { workspace = true } risingwave_common_heap_profiling = { workspace = true } diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index bf1bddad2070f..4be2bba4039b2 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -17,10 +17,12 @@ #![cfg_attr(coverage, feature(coverage_attribute))] mod server; + use std::time::Duration; use clap::Parser; pub use error::{MetaError, MetaResult}; +use redact::Secret; use risingwave_common::config::OverrideConfig; use risingwave_common::util::resource_util; use risingwave_common::{GIT_SHA, RW_VERSION}; @@ -71,7 +73,7 @@ pub struct MetaNodeOpts { /// Password of etcd, required when --etcd-auth is enabled. #[clap(long, env = "RW_ETCD_PASSWORD", default_value = "")] - etcd_password: String, + etcd_password: Secret, /// Endpoint of the SQL service, make it non-option when SQL service is required. #[clap(long, env = "RW_SQL_ENDPOINT")] @@ -196,7 +198,10 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { .map(|x| x.to_string()) .collect(), credentials: match opts.etcd_auth { - true => Some((opts.etcd_username, opts.etcd_password)), + true => Some(( + opts.etcd_username, + opts.etcd_password.expose_secret().to_string(), + )), false => None, }, }, From 46d2ff9d9ba9a501b37bd15dd144333c4bf0913f Mon Sep 17 00:00:00 2001 From: August Date: Thu, 26 Oct 2023 18:26:16 +0800 Subject: [PATCH 38/48] feat: use bit flags for user privilege check (#13033) --- Cargo.lock | 21 +++ proto/user.proto | 1 + src/common/Cargo.toml | 1 + src/common/src/acl/mod.rs | 144 +++++++++++++++ src/common/src/lib.rs | 1 + .../src/catalog/system_catalog/mod.rs | 49 ++--- src/frontend/src/handler/alter_user.rs | 7 +- src/frontend/src/handler/create_index.rs | 5 +- src/frontend/src/handler/create_mv.rs | 4 +- src/frontend/src/handler/create_schema.rs | 5 +- src/frontend/src/handler/create_user.rs | 3 +- src/frontend/src/handler/drop_user.rs | 10 +- src/frontend/src/handler/handle_privilege.rs | 22 +-- src/frontend/src/handler/privilege.rs | 51 +++--- src/frontend/src/session.rs | 14 +- src/frontend/src/test_utils.rs | 35 +--- src/frontend/src/user/mod.rs | 1 + src/frontend/src/user/user_catalog.rs | 171 ++++++++++++++++++ src/frontend/src/user/user_manager.rs | 17 +- src/frontend/src/user/user_privilege.rs | 90 ++++----- 20 files changed, 465 insertions(+), 187 deletions(-) create mode 100644 src/common/src/acl/mod.rs create mode 100644 src/frontend/src/user/user_catalog.rs diff --git a/Cargo.lock b/Cargo.lock index f03347d622208..4e969a94fc134 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2735,6 +2735,26 @@ dependencies = [ "syn 2.0.37", ] +[[package]] +name = "enumflags2" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5998b4f30320c9d93aed72f63af821bfdac50465b75428fce77b48ec482c3939" +dependencies = [ + "enumflags2_derive", +] + +[[package]] +name = "enumflags2_derive" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f95e2801cd355d4a1a3e3953ce6ee5ae9603a5c833455343a8bfe3f44d418246" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.37", +] + [[package]] name = "env_logger" version = "0.10.0" @@ -7068,6 +7088,7 @@ dependencies = [ "educe", "either", "enum-as-inner", + "enumflags2", "ethnum", "fixedbitset", "fs-err", diff --git a/proto/user.proto b/proto/user.proto index 7468147cad271..c998f66d15133 100644 --- a/proto/user.proto +++ b/proto/user.proto @@ -43,6 +43,7 @@ message GrantPrivilege { DELETE = 4; CREATE = 5; CONNECT = 6; + USAGE = 7; } message ActionWithGrantOption { diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 168ba836d4c1b..f44c0c9ba8a5d 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -38,6 +38,7 @@ easy-ext = "1" educe = "0.4" either = "1" enum-as-inner = "0.6" +enumflags2 = { version = "0.7.8" } ethnum = { version = "1", features = ["serde"] } fixedbitset = { version = "0.4", features = ["std"] } fs-err = "2" diff --git a/src/common/src/acl/mod.rs b/src/common/src/acl/mod.rs new file mode 100644 index 0000000000000..929577437571d --- /dev/null +++ b/src/common/src/acl/mod.rs @@ -0,0 +1,144 @@ +// 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. + +//! `Acl` defines all grantable privileges. + +use std::convert::Into; +use std::fmt::Formatter; +use std::sync::LazyLock; + +use enumflags2::{bitflags, make_bitflags, BitFlags}; +use parse_display::Display; +use risingwave_pb::user::grant_privilege::PbAction; + +#[bitflags] +#[repr(u64)] +#[derive(Clone, Copy, Debug, Display, Eq, PartialEq)] +pub enum AclMode { + #[display("a")] + Insert = 1 << 0, // formerly known as "append". + #[display("r")] + Select = 1 << 1, // formerly known as "read". + #[display("w")] + Update = 1 << 2, // formerly known as "write". + #[display("d")] + Delete = 1 << 3, + #[display("D")] + Truncate = 1 << 4, // super-delete, as it were + #[display("x")] + References = 1 << 5, + #[display("t")] + Trigger = 1 << 6, + #[display("X")] + Execute = 1 << 7, // For functions + #[display("U")] + Usage = 1 << 8, // For various object types + #[display("C")] + Create = 1 << 9, // For namespaces and databases + #[display("T")] + CreateTemp = 1 << 10, // For databases + #[display("c")] + Connect = 1 << 11, // For databases + #[display("s")] + Set = 1 << 12, // For configuration parameters + #[display("A")] + AlterSystem = 1 << 13, // For configuration parameters + #[display("m")] + Maintain = 1 << 14, // For relations +} + +impl From for AclMode { + fn from(action: PbAction) -> Self { + match action { + PbAction::Unspecified => unreachable!(), + PbAction::Select => AclMode::Select, + PbAction::Insert => AclMode::Insert, + PbAction::Update => AclMode::Update, + PbAction::Delete => AclMode::Delete, + PbAction::Create => AclMode::Create, + PbAction::Connect => AclMode::Connect, + PbAction::Usage => AclMode::Usage, + } + } +} + +impl From for PbAction { + fn from(val: AclMode) -> Self { + match val { + AclMode::Select => PbAction::Select, + AclMode::Insert => PbAction::Insert, + AclMode::Update => PbAction::Update, + AclMode::Delete => PbAction::Delete, + AclMode::Create => PbAction::Create, + AclMode::Connect => PbAction::Connect, + AclMode::Usage => PbAction::Usage, + _ => unreachable!(), + } + } +} + +/// `AclModeSet` defines a set of `AclMode`s. +#[derive(Clone, Debug)] +pub struct AclModeSet { + pub modes: BitFlags, +} + +pub static ALL_AVAILABLE_DATABASE_MODES: LazyLock = + LazyLock::new(|| make_bitflags!(AclMode::{Create | Connect}).into()); +pub static ALL_AVAILABLE_SCHEMA_MODES: LazyLock = + LazyLock::new(|| make_bitflags!(AclMode::{Create | Usage}).into()); +pub static ALL_AVAILABLE_TABLE_MODES: LazyLock = + LazyLock::new(|| make_bitflags!(AclMode::{Select | Insert | Update | Delete}).into()); +pub static ALL_AVAILABLE_SOURCE_MODES: LazyLock = LazyLock::new(AclModeSet::readonly); +pub static ALL_AVAILABLE_MVIEW_MODES: LazyLock = LazyLock::new(AclModeSet::readonly); +pub static ALL_AVAILABLE_VIEW_MODES: LazyLock = LazyLock::new(AclModeSet::readonly); +pub static ALL_AVAILABLE_SINK_MODES: LazyLock = LazyLock::new(AclModeSet::empty); +pub static ALL_AVAILABLE_FUNCTION_MODES: LazyLock = + LazyLock::new(|| BitFlags::from(AclMode::Execute).into()); +pub static ALL_AVAILABLE_CONNECTION_MODES: LazyLock = + LazyLock::new(|| BitFlags::from(AclMode::Usage).into()); + +impl AclModeSet { + pub fn empty() -> Self { + Self { + modes: BitFlags::empty(), + } + } + + pub fn readonly() -> Self { + Self { + modes: BitFlags::from(AclMode::Select), + } + } + + pub fn has_mode(&self, mode: AclMode) -> bool { + self.modes.contains(mode) + } + + pub fn iter(&self) -> impl Iterator + '_ { + self.modes.iter() + } +} + +impl From> for AclModeSet { + fn from(modes: BitFlags) -> Self { + Self { modes } + } +} + +impl std::fmt::Display for AclModeSet { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.modes) + } +} diff --git a/src/common/src/lib.rs b/src/common/src/lib.rs index fbcd3854fa572..408c8823d397f 100644 --- a/src/common/src/lib.rs +++ b/src/common/src/lib.rs @@ -70,6 +70,7 @@ pub mod system_param; pub mod telemetry; pub mod transaction; +pub mod acl; pub mod metrics; pub mod test_utils; pub mod types; diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index d64db79b8ced1..656960bffb906 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -21,6 +21,7 @@ use std::sync::{Arc, LazyLock}; use async_trait::async_trait; use itertools::Itertools; +use risingwave_common::acl::AclMode; use risingwave_common::catalog::{ ColumnCatalog, ColumnDesc, Field, SysCatalogReader, TableDesc, TableId, DEFAULT_SUPER_USER_ID, NON_RESERVED_SYS_CATALOG_ID, @@ -28,8 +29,7 @@ use risingwave_common::catalog::{ use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use risingwave_pb::user::grant_privilege::{Action, Object}; -use risingwave_pb::user::UserInfo; +use risingwave_pb::user::grant_privilege::Object; use crate::catalog::catalog_service::CatalogReader; use crate::catalog::system_catalog::information_schema::*; @@ -39,6 +39,7 @@ use crate::catalog::view_catalog::ViewCatalog; use crate::meta_client::FrontendMetaClient; use crate::scheduler::worker_node_manager::WorkerNodeManagerRef; use crate::session::AuthContext; +use crate::user::user_catalog::UserCatalog; use crate::user::user_privilege::available_prost_privilege; use crate::user::user_service::UserInfoReader; use crate::user::UserId; @@ -212,17 +213,17 @@ fn infer_dummy_view_sql(columns: &[SystemCatalogColumnsDef<'_>]) -> String { fn get_acl_items( object: &Object, for_dml_table: bool, - users: &Vec, + users: &Vec, username_map: &HashMap, ) -> String { let mut res = String::from("{"); let mut empty_flag = true; let super_privilege = available_prost_privilege(object.clone(), for_dml_table); for user in users { - let privileges = if user.get_is_super() { + let privileges = if user.is_super { vec![&super_privilege] } else { - user.get_grant_privileges() + user.grant_privileges .iter() .filter(|&privilege| privilege.object.as_ref().unwrap() == object) .collect_vec() @@ -233,43 +234,29 @@ fn get_acl_items( let mut grantor_map = HashMap::new(); privileges.iter().for_each(|&privilege| { privilege.action_with_opts.iter().for_each(|ao| { - grantor_map.entry(ao.granted_by).or_insert_with(Vec::new); grantor_map - .get_mut(&ao.granted_by) - .unwrap() - .push((ao.action, ao.with_grant_option)); + .entry(ao.granted_by) + .or_insert_with(Vec::new) + .push((ao.get_action().unwrap(), ao.with_grant_option)); }) }); - for key in grantor_map.keys() { + for (granted_by, actions) in grantor_map { if empty_flag { empty_flag = false; } else { res.push(','); } - res.push_str(user.get_name()); + res.push_str(&user.name); res.push('='); - grantor_map - .get(key) - .unwrap() - .iter() - .for_each(|(action, option)| { - let str = match Action::try_from(*action).unwrap() { - Action::Select => "r", - Action::Insert => "a", - Action::Update => "w", - Action::Delete => "d", - Action::Create => "C", - Action::Connect => "c", - _ => unreachable!(), - }; - res.push_str(str); - if *option { - res.push('*'); - } - }); + for (action, option) in actions { + res.push_str(&AclMode::from(action).to_string()); + if option { + res.push('*'); + } + } res.push('/'); // should be able to query grantor's name - res.push_str(username_map.get(key).as_ref().unwrap()); + res.push_str(username_map.get(&granted_by).unwrap()); } } res.push('}'); diff --git a/src/frontend/src/handler/alter_user.rs b/src/frontend/src/handler/alter_user.rs index 5c91df3888b71..0d83c3ae867d5 100644 --- a/src/frontend/src/handler/alter_user.rs +++ b/src/frontend/src/handler/alter_user.rs @@ -24,11 +24,12 @@ use crate::binder::Binder; use crate::catalog::CatalogError; use crate::handler::HandlerArgs; use crate::user::user_authentication::encrypted_password; +use crate::user::user_catalog::UserCatalog; fn alter_prost_user_info( mut user_info: UserInfo, options: &UserOptions, - session_user: &UserInfo, + session_user: &UserCatalog, ) -> Result<(UserInfo, Vec)> { if !session_user.is_super { let require_super = user_info.is_super @@ -116,7 +117,7 @@ fn alter_prost_user_info( fn alter_rename_prost_user_info( mut user_info: UserInfo, new_name: ObjectName, - session_user: &UserInfo, + session_user: &UserCatalog, ) -> Result<(UserInfo, Vec)> { if session_user.id == user_info.id { return Err(InternalError("session user cannot be renamed".to_string()).into()); @@ -153,7 +154,7 @@ pub async fn handle_alter_user( let old_info = user_reader .get_user_by_name(&user_name) .ok_or(CatalogError::NotFound("user", user_name))? - .clone(); + .to_prost(); let session_user = user_reader .get_user_by_name(session.user_name()) diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index a5a002d3b3d79..bed35eadec9ae 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -18,12 +18,13 @@ use std::rc::Rc; use fixedbitset::FixedBitSet; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::acl::AclMode; use risingwave_common::catalog::{IndexId, TableDesc, TableId}; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::catalog::{PbIndex, PbStreamJobStatus, PbTable}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; -use risingwave_pb::user::grant_privilege::{Action, Object}; +use risingwave_pb::user::grant_privilege::Object; use risingwave_sqlparser::ast; use risingwave_sqlparser::ast::{Ident, ObjectName, OrderByExpr}; @@ -74,7 +75,7 @@ pub(crate) fn gen_create_index_plan( session.check_privileges(&[ObjectCheckItem::new( table.owner, - Action::Select, + AclMode::Select, Object::TableId(table.id.table_id), )])?; diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index 053ba5aa30f19..75474ca576dd8 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -14,10 +14,10 @@ use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::acl::AclMode; use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::catalog::{CreateType, PbTable}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; -use risingwave_pb::user::grant_privilege::Action; use risingwave_sqlparser::ast::{EmitMode, Ident, ObjectName, Query}; use super::privilege::resolve_relation_privileges; @@ -66,7 +66,7 @@ pub(super) fn get_column_names( } if let Some(relation) = &select.from { let mut check_items = Vec::new(); - resolve_relation_privileges(relation, Action::Select, &mut check_items); + resolve_relation_privileges(relation, AclMode::Select, &mut check_items); session.check_privileges(&check_items)?; } } diff --git a/src/frontend/src/handler/create_schema.rs b/src/frontend/src/handler/create_schema.rs index 12970193fe33e..962d59178bf0c 100644 --- a/src/frontend/src/handler/create_schema.rs +++ b/src/frontend/src/handler/create_schema.rs @@ -13,9 +13,10 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::acl::AclMode; use risingwave_common::catalog::RESERVED_PG_SCHEMA_PREFIX; use risingwave_common::error::{ErrorCode, Result}; -use risingwave_pb::user::grant_privilege::{Action, Object}; +use risingwave_pb::user::grant_privilege::Object; use risingwave_sqlparser::ast::ObjectName; use super::RwPgResponse; @@ -63,7 +64,7 @@ pub async fn handle_create_schema( session.check_privileges(&[ObjectCheckItem::new( db_owner, - Action::Create, + AclMode::Create, Object::DatabaseId(db_id), )])?; diff --git a/src/frontend/src/handler/create_user.rs b/src/frontend/src/handler/create_user.rs index a4f8eaecc405e..8659e1b647c33 100644 --- a/src/frontend/src/handler/create_user.rs +++ b/src/frontend/src/handler/create_user.rs @@ -24,11 +24,12 @@ use crate::binder::Binder; use crate::catalog::{CatalogError, DatabaseId}; use crate::handler::HandlerArgs; use crate::user::user_authentication::encrypted_password; +use crate::user::user_catalog::UserCatalog; fn make_prost_user_info( user_name: String, options: &UserOptions, - session_user: &UserInfo, + session_user: &UserCatalog, database_id: DatabaseId, ) -> Result { if !session_user.is_super { diff --git a/src/frontend/src/handler/drop_user.rs b/src/frontend/src/handler/drop_user.rs index 11232faa64a95..5b0ae7d55596c 100644 --- a/src/frontend/src/handler/drop_user.rs +++ b/src/frontend/src/handler/drop_user.rs @@ -34,14 +34,14 @@ pub async fn handle_drop_user( let user_name = Binder::resolve_user_name(user_name)?; let user_info_reader = session.env().user_info_reader(); - let user = user_info_reader + let user_id = user_info_reader .read_guard() .get_user_by_name(&user_name) - .cloned(); - match user { - Some(user) => { + .map(|u| u.id); + match user_id { + Some(user_id) => { let user_info_writer = session.user_info_writer()?; - user_info_writer.drop_user(user.id).await?; + user_info_writer.drop_user(user_id).await?; } None => { return if if_exists { diff --git a/src/frontend/src/handler/handle_privilege.rs b/src/frontend/src/handler/handle_privilege.rs index 2f166e9fa1824..07b87fa3bc3d2 100644 --- a/src/frontend/src/handler/handle_privilege.rs +++ b/src/frontend/src/handler/handle_privilege.rs @@ -39,7 +39,10 @@ fn make_prost_privilege( let reader = catalog_reader.read_guard(); let actions = match privileges { Privileges::All { .. } => available_privilege_actions(&objects)?, - Privileges::Actions(actions) => actions, + Privileges::Actions(actions) => actions + .into_iter() + .map(|action| get_prost_action(&action)) + .collect(), }; let mut grant_objs = vec![]; match objects { @@ -147,14 +150,11 @@ fn make_prost_privilege( } }; let action_with_opts = actions - .iter() - .map(|action| { - let prost_action = get_prost_action(action); - ActionWithGrantOption { - action: prost_action as i32, - granted_by: session.user_id(), - ..Default::default() - } + .into_iter() + .map(|action| ActionWithGrantOption { + action: action as i32, + granted_by: session.user_id(), + ..Default::default() }) .collect::>(); @@ -318,12 +318,12 @@ mod tests { PbGrantPrivilege { action_with_opts: vec![ ActionWithGrantOption { - action: Action::Connect as i32, + action: Action::Create as i32, with_grant_option: true, granted_by: DEFAULT_SUPER_USER_ID, }, ActionWithGrantOption { - action: Action::Create as i32, + action: Action::Connect as i32, with_grant_option: true, granted_by: DEFAULT_SUPER_USER_ID, } diff --git a/src/frontend/src/handler/privilege.rs b/src/frontend/src/handler/privilege.rs index dcd8696b9670f..24227a46ff2a6 100644 --- a/src/frontend/src/handler/privilege.rs +++ b/src/frontend/src/handler/privilege.rs @@ -12,9 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::acl::AclMode; use risingwave_common::error::ErrorCode::PermissionDenied; use risingwave_common::error::Result; -use risingwave_pb::user::grant_privilege::{PbAction, PbObject}; +use risingwave_pb::user::grant_privilege::PbObject; use crate::binder::{BoundQuery, BoundStatement, Relation}; use crate::catalog::OwnedByUserCatalog; @@ -24,15 +25,16 @@ use crate::user::UserId; #[derive(Debug)] pub struct ObjectCheckItem { owner: UserId, - action: PbAction, + mode: AclMode, + // todo: change it to object id. object: PbObject, } impl ObjectCheckItem { - pub fn new(owner: UserId, action: PbAction, object: PbObject) -> Self { + pub fn new(owner: UserId, mode: AclMode, object: PbObject) -> Self { Self { owner, - action, + mode, object, } } @@ -41,14 +43,14 @@ impl ObjectCheckItem { /// resolve privileges in `relation` pub(crate) fn resolve_relation_privileges( relation: &Relation, - action: PbAction, + mode: AclMode, objects: &mut Vec, ) { match relation { Relation::Source(source) => { let item = ObjectCheckItem { owner: source.catalog.owner, - action, + mode, object: PbObject::SourceId(source.catalog.id), }; objects.push(item); @@ -56,7 +58,7 @@ pub(crate) fn resolve_relation_privileges( Relation::BaseTable(table) => { let item = ObjectCheckItem { owner: table.table_catalog.owner, - action, + mode, object: PbObject::TableId(table.table_id.table_id), }; objects.push(item); @@ -64,16 +66,16 @@ pub(crate) fn resolve_relation_privileges( Relation::Subquery(query) => { if let crate::binder::BoundSetExpr::Select(select) = &query.query.body { if let Some(sub_relation) = &select.from { - resolve_relation_privileges(sub_relation, action, objects); + resolve_relation_privileges(sub_relation, mode, objects); } } } Relation::Join(join) => { - resolve_relation_privileges(&join.left, action, objects); - resolve_relation_privileges(&join.right, action, objects); + resolve_relation_privileges(&join.left, mode, objects); + resolve_relation_privileges(&join.right, mode, objects); } Relation::WindowTableFunction(table) => { - resolve_relation_privileges(&table.input, action, objects) + resolve_relation_privileges(&table.input, mode, objects) } _ => {} }; @@ -86,20 +88,20 @@ pub(crate) fn resolve_privileges(stmt: &BoundStatement) -> Vec BoundStatement::Insert(ref insert) => { let object = ObjectCheckItem { owner: insert.owner, - action: PbAction::Insert, + mode: AclMode::Insert, object: PbObject::TableId(insert.table_id.table_id), }; objects.push(object); if let crate::binder::BoundSetExpr::Select(select) = &insert.source.body { if let Some(sub_relation) = &select.from { - resolve_relation_privileges(sub_relation, PbAction::Select, &mut objects); + resolve_relation_privileges(sub_relation, AclMode::Select, &mut objects); } } } BoundStatement::Delete(ref delete) => { let object = ObjectCheckItem { owner: delete.owner, - action: PbAction::Delete, + mode: AclMode::Delete, object: PbObject::TableId(delete.table_id.table_id), }; objects.push(object); @@ -107,7 +109,7 @@ pub(crate) fn resolve_privileges(stmt: &BoundStatement) -> Vec BoundStatement::Update(ref update) => { let object = ObjectCheckItem { owner: update.owner, - action: PbAction::Update, + mode: AclMode::Update, object: PbObject::TableId(update.table_id.table_id), }; objects.push(object); @@ -122,7 +124,7 @@ pub(crate) fn resolve_query_privileges(query: &BoundQuery) -> Vec Result<()> { let mut lock = self.user_info.write(); let id = update_user.get_id(); - let old_name = lock.get_user_name_by_id(id).unwrap(); - let mut user_info = lock.get_user_by_name(&old_name).unwrap().clone(); + let Some(old_name) = lock.get_user_name_by_id(id) else { + return Ok(()); + }; + let mut user_info = lock.get_user_by_name(&old_name).unwrap().to_prost(); update_fields.into_iter().for_each(|field| match field { UpdateField::Super => user_info.is_super = update_user.is_super, UpdateField::Login => user_info.can_login = update_user.can_login, @@ -679,7 +681,7 @@ impl UserInfoWriter for MockUserInfoWriter { .collect::>(); for user_id in users { if let Some(u) = self.user_info.write().get_user_mut(user_id) { - u.grant_privileges.extend(privileges.clone()); + u.extend_privileges(privileges.clone()); } } Ok(()) @@ -698,32 +700,7 @@ impl UserInfoWriter for MockUserInfoWriter { ) -> Result<()> { for user_id in users { if let Some(u) = self.user_info.write().get_user_mut(user_id) { - u.grant_privileges.iter_mut().for_each(|p| { - for rp in &privileges { - if rp.object != p.object { - continue; - } - if revoke_grant_option { - for ao in &mut p.action_with_opts { - if rp - .action_with_opts - .iter() - .any(|rao| rao.action == ao.action) - { - ao.with_grant_option = false; - } - } - } else { - p.action_with_opts.retain(|po| { - rp.action_with_opts - .iter() - .all(|rao| rao.action != po.action) - }); - } - } - }); - u.grant_privileges - .retain(|p| !p.action_with_opts.is_empty()); + u.revoke_privileges(privileges.clone(), revoke_grant_option); } } Ok(()) diff --git a/src/frontend/src/user/mod.rs b/src/frontend/src/user/mod.rs index c0e23f2a69d92..6dfec976ac7f8 100644 --- a/src/frontend/src/user/mod.rs +++ b/src/frontend/src/user/mod.rs @@ -13,6 +13,7 @@ // limitations under the License. pub(crate) mod user_authentication; +pub(crate) mod user_catalog; pub(crate) mod user_manager; pub mod user_privilege; pub(crate) mod user_service; diff --git a/src/frontend/src/user/user_catalog.rs b/src/frontend/src/user/user_catalog.rs new file mode 100644 index 0000000000000..e1b5f85d446b1 --- /dev/null +++ b/src/frontend/src/user/user_catalog.rs @@ -0,0 +1,171 @@ +// 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::hash_map::Entry; +use std::collections::HashMap; + +use risingwave_common::acl::{AclMode, AclModeSet}; +use risingwave_pb::user::grant_privilege::{Object as GrantObject, Object}; +use risingwave_pb::user::{PbAuthInfo, PbGrantPrivilege, PbUserInfo}; + +use crate::catalog::{DatabaseId, SchemaId}; +use crate::user::UserId; + +/// `UserCatalog` is responsible for managing user's information. +#[derive(Clone, Debug)] +pub struct UserCatalog { + pub id: UserId, + pub name: String, + pub is_super: bool, + pub can_create_db: bool, + pub can_create_user: bool, + pub can_login: bool, + pub auth_info: Option, + pub grant_privileges: Vec, + + // User owned acl mode set, group by object id. + // TODO: merge it after we fully migrate to sql-backend. + pub database_acls: HashMap, + pub schema_acls: HashMap, + pub object_acls: HashMap, +} + +impl From for UserCatalog { + fn from(user: PbUserInfo) -> Self { + let mut user_catalog = Self { + id: user.id, + name: user.name, + is_super: user.is_super, + can_create_db: user.can_create_db, + can_create_user: user.can_create_user, + can_login: user.can_login, + auth_info: user.auth_info, + grant_privileges: user.grant_privileges, + database_acls: Default::default(), + schema_acls: Default::default(), + object_acls: Default::default(), + }; + user_catalog.refresh_acl_modes(); + + user_catalog + } +} + +impl UserCatalog { + pub fn to_prost(&self) -> PbUserInfo { + PbUserInfo { + id: self.id, + name: self.name.clone(), + is_super: self.is_super, + can_create_db: self.can_create_db, + can_create_user: self.can_create_user, + can_login: self.can_login, + auth_info: self.auth_info.clone(), + grant_privileges: self.grant_privileges.clone(), + } + } + + fn get_acl_entry(&mut self, object: GrantObject) -> Entry<'_, u32, AclModeSet> { + match object { + Object::DatabaseId(id) => self.database_acls.entry(id), + Object::SchemaId(id) => self.schema_acls.entry(id), + Object::TableId(id) => self.object_acls.entry(id), + Object::SourceId(id) => self.object_acls.entry(id), + Object::SinkId(id) => self.object_acls.entry(id), + Object::ViewId(id) => self.object_acls.entry(id), + Object::FunctionId(_) => { + unreachable!("grant privilege on function is not supported yet.") + } + _ => unreachable!(""), + } + } + + fn get_acl(&self, object: &GrantObject) -> Option<&AclModeSet> { + match object { + Object::DatabaseId(id) => self.database_acls.get(id), + Object::SchemaId(id) => self.schema_acls.get(id), + Object::TableId(id) => self.object_acls.get(id), + Object::SourceId(id) => self.object_acls.get(id), + Object::SinkId(id) => self.object_acls.get(id), + Object::ViewId(id) => self.object_acls.get(id), + Object::FunctionId(_) => { + unreachable!("grant privilege on function is not supported yet.") + } + _ => unreachable!("unexpected object type."), + } + } + + fn refresh_acl_modes(&mut self) { + self.database_acls.clear(); + self.schema_acls.clear(); + self.object_acls.clear(); + let privileges = self.grant_privileges.clone(); + for privilege in privileges { + let entry = self + .get_acl_entry(privilege.object.unwrap()) + .or_insert(AclModeSet::empty()); + for awo in privilege.action_with_opts { + entry + .modes + .insert::(awo.get_action().unwrap().into()); + } + } + } + + // Only for test, used in `MockUserInfoWriter`. + pub fn extend_privileges(&mut self, privileges: Vec) { + self.grant_privileges.extend(privileges); + self.refresh_acl_modes(); + } + + // Only for test, used in `MockUserInfoWriter`. + pub fn revoke_privileges( + &mut self, + privileges: Vec, + revoke_grant_option: bool, + ) { + self.grant_privileges.iter_mut().for_each(|p| { + for rp in &privileges { + if rp.object != p.object { + continue; + } + if revoke_grant_option { + for ao in &mut p.action_with_opts { + if rp + .action_with_opts + .iter() + .any(|rao| rao.action == ao.action) + { + ao.with_grant_option = false; + } + } + } else { + p.action_with_opts.retain(|po| { + rp.action_with_opts + .iter() + .all(|rao| rao.action != po.action) + }); + } + } + }); + self.grant_privileges + .retain(|p| !p.action_with_opts.is_empty()); + self.refresh_acl_modes(); + } + + pub fn check_privilege(&self, object: &GrantObject, mode: AclMode) -> bool { + self.get_acl(object) + .map_or(false, |acl_set| acl_set.has_mode(mode)) + } +} diff --git a/src/frontend/src/user/user_manager.rs b/src/frontend/src/user/user_manager.rs index 3620eef51114a..d42c764c0b5ed 100644 --- a/src/frontend/src/user/user_manager.rs +++ b/src/frontend/src/user/user_manager.rs @@ -17,12 +17,13 @@ use std::collections::HashMap; use itertools::Itertools; use risingwave_pb::user::{GrantPrivilege, UserInfo}; +use crate::user::user_catalog::UserCatalog; use crate::user::{UserId, UserInfoVersion}; /// `UserInfoManager` is responsible for managing users. pub struct UserInfoManager { version: UserInfoVersion, - user_by_name: HashMap, + user_by_name: HashMap, user_name_by_id: HashMap, } @@ -38,16 +39,16 @@ impl Default for UserInfoManager { } impl UserInfoManager { - pub fn get_user_mut(&mut self, id: UserId) -> Option<&mut UserInfo> { + pub fn get_user_mut(&mut self, id: UserId) -> Option<&mut UserCatalog> { let name = self.user_name_by_id.get(&id)?; self.user_by_name.get_mut(name) } - pub fn get_all_users(&self) -> Vec { + pub fn get_all_users(&self) -> Vec { self.user_by_name.values().cloned().collect_vec() } - pub fn get_user_by_name(&self, user_name: &str) -> Option<&UserInfo> { + pub fn get_user_by_name(&self, user_name: &str) -> Option<&UserCatalog> { self.user_by_name.get(user_name) } @@ -63,7 +64,7 @@ impl UserInfoManager { let id = user_info.id; let name = user_info.name.clone(); self.user_by_name - .try_insert(name.clone(), user_info) + .try_insert(name.clone(), user_info.into()) .unwrap(); self.user_name_by_id.try_insert(id, name).unwrap(); } @@ -78,9 +79,11 @@ impl UserInfoManager { let name = user_info.name.clone(); if let Some(old_name) = self.get_user_name_by_id(id) { self.user_by_name.remove(&old_name); - self.user_by_name.insert(name.clone(), user_info); + self.user_by_name.insert(name.clone(), user_info.into()); } else { - self.user_by_name.insert(name.clone(), user_info).unwrap(); + self.user_by_name + .insert(name.clone(), user_info.into()) + .unwrap(); } self.user_name_by_id.insert(id, name).unwrap(); } diff --git a/src/frontend/src/user/user_privilege.rs b/src/frontend/src/user/user_privilege.rs index 69ea6ce6830f0..104f1b28aa5be 100644 --- a/src/frontend/src/user/user_privilege.rs +++ b/src/frontend/src/user/user_privilege.rs @@ -13,54 +13,23 @@ // limitations under the License. use itertools::Itertools; +use risingwave_common::acl; +use risingwave_common::acl::{AclMode, AclModeSet}; use risingwave_common::catalog::DEFAULT_SUPER_USER_ID; use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::user::grant_privilege::{ActionWithGrantOption, PbAction, PbObject}; use risingwave_pb::user::PbGrantPrivilege; use risingwave_sqlparser::ast::{Action, GrantObjects, Privileges}; -// TODO: add user_privilege mod under user manager and move check and expand logic there, and bitmap -// impl for privilege check. -static AVAILABLE_ACTION_ON_DATABASE: &[Action] = &[Action::Connect, Action::Create]; -static AVAILABLE_ACTION_ON_SCHEMA: &[Action] = &[Action::Create]; - -static AVAILABLE_ACTION_ON_TABLE: &[Action] = &[ - Action::Select { columns: None }, - Action::Update { columns: None }, - Action::Insert { columns: None }, - Action::Delete, -]; -static AVAILABLE_ACTION_ON_MVIEW: &[Action] = &[Action::Select { columns: None }]; -static AVAILABLE_ACTION_ON_VIEW: &[Action] = AVAILABLE_ACTION_ON_MVIEW; -static AVAILABLE_ACTION_ON_SOURCE: &[Action] = AVAILABLE_ACTION_ON_MVIEW; -static AVAILABLE_ACTION_ON_SINK: &[Action] = &[]; -static AVAILABLE_ACTION_ON_FUNCTION: &[Action] = &[]; - pub fn check_privilege_type(privilege: &Privileges, objects: &GrantObjects) -> Result<()> { match privilege { Privileges::All { .. } => Ok(()), Privileges::Actions(actions) => { - let valid = match objects { - GrantObjects::Databases(_) => actions - .iter() - .all(|action| AVAILABLE_ACTION_ON_DATABASE.contains(action)), - GrantObjects::Schemas(_) => actions - .iter() - .all(|action| AVAILABLE_ACTION_ON_SCHEMA.contains(action)), - GrantObjects::Sources(_) | GrantObjects::AllSourcesInSchema { .. } => actions - .iter() - .all(|action| AVAILABLE_ACTION_ON_SOURCE.contains(action)), - GrantObjects::Mviews(_) | GrantObjects::AllMviewsInSchema { .. } => actions - .iter() - .all(|action| AVAILABLE_ACTION_ON_MVIEW.contains(action)), - GrantObjects::Tables(_) | GrantObjects::AllTablesInSchema { .. } => actions - .iter() - .all(|action| AVAILABLE_ACTION_ON_TABLE.contains(action)), - GrantObjects::Sinks(_) => actions - .iter() - .all(|action| AVAILABLE_ACTION_ON_SINK.contains(action)), - GrantObjects::Sequences(_) | GrantObjects::AllSequencesInSchema { .. } => true, - }; + let acl_sets = get_all_available_modes(objects)?; + let valid = actions + .iter() + .map(get_prost_action) + .all(|action| acl_sets.has_mode(action.into())); if !valid { return Err(ErrorCode::BindError( "Invalid privilege type for the given object.".to_string(), @@ -73,25 +42,31 @@ pub fn check_privilege_type(privilege: &Privileges, objects: &GrantObjects) -> R } } -pub fn available_privilege_actions(objects: &GrantObjects) -> Result> { - match objects { - GrantObjects::Databases(_) => Ok(AVAILABLE_ACTION_ON_DATABASE.to_vec()), - GrantObjects::Schemas(_) => Ok(AVAILABLE_ACTION_ON_SCHEMA.to_vec()), +fn get_all_available_modes(object: &GrantObjects) -> Result<&AclModeSet> { + match object { + GrantObjects::Databases(_) => Ok(&acl::ALL_AVAILABLE_DATABASE_MODES), + GrantObjects::Schemas(_) => Ok(&acl::ALL_AVAILABLE_SCHEMA_MODES), GrantObjects::Sources(_) | GrantObjects::AllSourcesInSchema { .. } => { - Ok(AVAILABLE_ACTION_ON_SOURCE.to_vec()) + Ok(&acl::ALL_AVAILABLE_SOURCE_MODES) } GrantObjects::Mviews(_) | GrantObjects::AllMviewsInSchema { .. } => { - Ok(AVAILABLE_ACTION_ON_MVIEW.to_vec()) + Ok(&acl::ALL_AVAILABLE_MVIEW_MODES) } GrantObjects::Tables(_) | GrantObjects::AllTablesInSchema { .. } => { - Ok(AVAILABLE_ACTION_ON_TABLE.to_vec()) + Ok(&acl::ALL_AVAILABLE_TABLE_MODES) } + GrantObjects::Sinks(_) => Ok(&acl::ALL_AVAILABLE_SINK_MODES), _ => Err( ErrorCode::BindError("Invalid privilege type for the given object.".to_string()).into(), ), } } +pub fn available_privilege_actions(objects: &GrantObjects) -> Result> { + let acl_sets = get_all_available_modes(objects)?; + Ok(acl_sets.iter().map(Into::into).collect_vec()) +} + #[inline(always)] pub fn get_prost_action(action: &Action) -> PbAction { match action { @@ -101,31 +76,32 @@ pub fn get_prost_action(action: &Action) -> PbAction { Action::Delete { .. } => PbAction::Delete, Action::Connect => PbAction::Connect, Action::Create => PbAction::Create, + Action::Usage => PbAction::Usage, _ => unreachable!(), } } pub fn available_prost_privilege(object: PbObject, for_dml_table: bool) -> PbGrantPrivilege { - let actions = match object { - PbObject::DatabaseId(_) => AVAILABLE_ACTION_ON_DATABASE.to_vec(), - PbObject::SchemaId(_) => AVAILABLE_ACTION_ON_SCHEMA.to_vec(), - PbObject::SourceId(_) => AVAILABLE_ACTION_ON_SOURCE.to_vec(), + let acl_set = match object { + PbObject::DatabaseId(_) => &acl::ALL_AVAILABLE_DATABASE_MODES, + PbObject::SchemaId(_) => &acl::ALL_AVAILABLE_SCHEMA_MODES, + PbObject::SourceId(_) => &acl::ALL_AVAILABLE_SOURCE_MODES, PbObject::TableId(_) => { if for_dml_table { - AVAILABLE_ACTION_ON_TABLE.to_vec() + &acl::ALL_AVAILABLE_TABLE_MODES } else { - AVAILABLE_ACTION_ON_MVIEW.to_vec() + &acl::ALL_AVAILABLE_MVIEW_MODES } } - PbObject::ViewId(_) => AVAILABLE_ACTION_ON_VIEW.to_vec(), - PbObject::SinkId(_) => AVAILABLE_ACTION_ON_SINK.to_vec(), - PbObject::FunctionId(_) => AVAILABLE_ACTION_ON_FUNCTION.to_vec(), + PbObject::ViewId(_) => &acl::ALL_AVAILABLE_VIEW_MODES, + PbObject::SinkId(_) => &acl::ALL_AVAILABLE_SINK_MODES, + PbObject::FunctionId(_) => &acl::ALL_AVAILABLE_FUNCTION_MODES, _ => unreachable!("Invalid object type"), }; - let actions = actions + let actions = acl_set .iter() - .map(|action| ActionWithGrantOption { - action: get_prost_action(action) as i32, + .map(|mode| ActionWithGrantOption { + action: >::into(mode) as i32, with_grant_option: false, granted_by: DEFAULT_SUPER_USER_ID, }) From bb100a743b593e1c95527a4fe9728c8f43df0be7 Mon Sep 17 00:00:00 2001 From: August Date: Thu, 26 Oct 2023 20:35:54 +0800 Subject: [PATCH 39/48] feat: implement comment function in catalog controller (#13084) --- .../migration/src/m20230908_072257_init.rs | 2 + src/meta/model_v2/src/table.rs | 1 + src/meta/src/controller/catalog.rs | 66 ++++++++++++++++++- 3 files changed, 66 insertions(+), 3 deletions(-) diff --git a/src/meta/model_v2/migration/src/m20230908_072257_init.rs b/src/meta/model_v2/migration/src/m20230908_072257_init.rs index c9559bd6feda2..59a30149a1158 100644 --- a/src/meta/model_v2/migration/src/m20230908_072257_init.rs +++ b/src/meta/model_v2/migration/src/m20230908_072257_init.rs @@ -472,6 +472,7 @@ impl MigrationTrait for Migration { ) .col(ColumnDef::new(Table::JobStatus).string().not_null()) .col(ColumnDef::new(Table::CreateType).string().not_null()) + .col(ColumnDef::new(Table::Description).string()) .col(ColumnDef::new(Table::Version).json().not_null()) .foreign_key( &mut ForeignKey::create() @@ -887,6 +888,7 @@ enum Table { CleanedByWatermark, JobStatus, CreateType, + Description, Version, } diff --git a/src/meta/model_v2/src/table.rs b/src/meta/model_v2/src/table.rs index a335f41023442..78602110788a6 100644 --- a/src/meta/model_v2/src/table.rs +++ b/src/meta/model_v2/src/table.rs @@ -94,6 +94,7 @@ pub struct Model { pub cleaned_by_watermark: bool, pub job_status: JobStatus, pub create_type: CreateType, + pub description: Option, pub version: TableVersion, } diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index daaa9b684850c..9cf2ac987cbc9 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -14,6 +14,7 @@ use std::iter; +use anyhow::anyhow; use itertools::Itertools; use risingwave_common::bail; use risingwave_common::catalog::{DEFAULT_SCHEMA_NAME, SYSTEM_SCHEMAS}; @@ -21,11 +22,12 @@ use risingwave_meta_model_v2::object::ObjectType; use risingwave_meta_model_v2::prelude::*; use risingwave_meta_model_v2::{ connection, database, function, index, object, object_dependency, schema, sink, source, table, - view, ConnectionId, DatabaseId, FunctionId, ObjectId, PrivateLinkService, SchemaId, SourceId, - TableId, UserId, + view, ColumnCatalogArray, ConnectionId, DatabaseId, FunctionId, ObjectId, PrivateLinkService, + SchemaId, SourceId, TableId, UserId, }; use risingwave_pb::catalog::{ - PbConnection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, + PbComment, PbConnection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, + PbView, }; use risingwave_pb::meta::relation::PbRelationInfo; use risingwave_pb::meta::subscribe_response::{ @@ -490,6 +492,64 @@ impl CatalogController { Ok(version) } + pub async fn comment_on(&self, comment: PbComment) -> MetaResult { + let inner = self.inner.write().await; + let txn = inner.db.begin().await?; + ensure_object_id(ObjectType::Database, comment.database_id, &txn).await?; + ensure_object_id(ObjectType::Schema, comment.schema_id, &txn).await?; + let table_obj = Object::find_by_id(comment.table_id) + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("table", comment.table_id))?; + + let table = if let Some(col_idx) = comment.column_index { + let mut columns: ColumnCatalogArray = Table::find_by_id(comment.table_id) + .select_only() + .column(table::Column::Columns) + .into_tuple() + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("table", comment.table_id))?; + let column = columns + .0 + .get_mut(col_idx as usize) + .ok_or_else(|| MetaError::catalog_id_not_found("column", col_idx))?; + let column_desc = column.column_desc.as_mut().ok_or_else(|| { + anyhow!( + "column desc at index {} for table id {} not found", + col_idx, + comment.table_id + ) + })?; + column_desc.description = comment.description; + table::ActiveModel { + table_id: ActiveValue::Set(comment.table_id), + columns: ActiveValue::Set(columns), + ..Default::default() + } + .update(&txn) + .await? + } else { + table::ActiveModel { + table_id: ActiveValue::Set(comment.table_id), + description: ActiveValue::Set(comment.description), + ..Default::default() + } + .update(&txn) + .await? + }; + txn.commit().await?; + + let version = self + .notify_frontend_relation_info( + NotificationOperation::Update, + PbRelationInfo::Table(ObjectModel(table, table_obj).into()), + ) + .await; + + Ok(version) + } + pub async fn drop_relation( &self, object_type: ObjectType, From 8eedf8d0862a005d68a06ca22ef2cc13f4561d91 Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Thu, 26 Oct 2023 22:28:15 +0800 Subject: [PATCH 40/48] fix(object_store): fix a corner case with retry (#13078) --- Cargo.lock | 1 + e2e_test/sink/kafka/avro.slt | 2 +- e2e_test/sink/kafka/protobuf.slt | 2 +- src/object_store/Cargo.toml | 1 + src/object_store/src/object/s3.rs | 48 ++++++++++++++++++++++++------- 5 files changed, 41 insertions(+), 13 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4e969a94fc134..a2a8941e5905a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7832,6 +7832,7 @@ dependencies = [ "aws-smithy-types", "bytes", "crc32fast", + "either", "fail", "futures", "hyper", diff --git a/e2e_test/sink/kafka/avro.slt b/e2e_test/sink/kafka/avro.slt index e1b09e3608e37..a30d8b70fd4ba 100644 --- a/e2e_test/sink/kafka/avro.slt +++ b/e2e_test/sink/kafka/avro.slt @@ -57,7 +57,7 @@ select timestamp_millis_field, date_field, time_micros_field, - time_millis_field from from_kafka; + time_millis_field from from_kafka order by string_field; ---- t Rising \x6130 3.5 4.25 22 23 NULL {{NULL,3},NULL,{7,NULL,2}} 2006-01-02 22:04:05+00:00 NULL NULL 12:34:56.123456 NULL f Wave \x5a4446 1.5 NULL 11 12 (NULL,foo) NULL NULL 2006-01-02 22:04:05+00:00 2021-04-01 NULL 23:45:16.654 diff --git a/e2e_test/sink/kafka/protobuf.slt b/e2e_test/sink/kafka/protobuf.slt index 2f827aeda9fc0..c6ccb2ac24416 100644 --- a/e2e_test/sink/kafka/protobuf.slt +++ b/e2e_test/sink/kafka/protobuf.slt @@ -61,7 +61,7 @@ select nested_message_field, repeated_int_field, timestamp_field, - oneof_int32 from from_kafka; + oneof_int32 from from_kafka order by string_field; ---- t Rising \x6130 3.5 4.25 22 23 24 0 26 27 (1,) {4,0,4} (1136239445,0) 42 f Wave \x5a4446 1.5 0 11 12 13 14 15 16 (4,foo) {} (0,0) 0 diff --git a/src/object_store/Cargo.toml b/src/object_store/Cargo.toml index 88f73ca1d3c7d..f117c272a9afc 100644 --- a/src/object_store/Cargo.toml +++ b/src/object_store/Cargo.toml @@ -18,6 +18,7 @@ aws-smithy-http = { workspace = true } aws-smithy-types = { workspace = true } bytes = { version = "1", features = ["serde"] } crc32fast = "1.3.2" +either = "1" fail = "0.5" futures = { version = "0.3", default-features = false, features = ["alloc"] } hyper = "0.14" diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index 69e7f3687fdeb..89f9aa5a053d5 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -24,7 +24,7 @@ use aws_sdk_s3::operation::get_object::builders::GetObjectFluentBuilder; use aws_sdk_s3::operation::get_object::GetObjectError; use aws_sdk_s3::operation::list_objects_v2::ListObjectsV2Error; use aws_sdk_s3::operation::upload_part::UploadPartOutput; -use aws_sdk_s3::primitives::ByteStream; +use aws_sdk_s3::primitives::{ByteStream, ByteStreamError}; use aws_sdk_s3::types::{ AbortIncompleteMultipartUpload, BucketLifecycleConfiguration, CompletedMultipartUpload, CompletedPart, Delete, ExpirationStatus, LifecycleRule, LifecycleRuleFilter, ObjectIdentifier, @@ -34,6 +34,7 @@ use aws_smithy_client::http_connector::{ConnectorSettings, HttpConnector}; use aws_smithy_http::body::SdkBody; use aws_smithy_http::result::SdkError; use aws_smithy_types::retry::RetryConfig; +use either::Either; use fail::fail_point; use futures::future::{try_join_all, BoxFuture, FutureExt}; use futures::{stream, Stream}; @@ -354,11 +355,19 @@ impl ObjectStore for S3ObjectStore { ))); // retry if occurs AWS EC2 HTTP timeout error. - let resp = tokio_retry::RetryIf::spawn( + let val = tokio_retry::RetryIf::spawn( self.config.get_retry_strategy(), || async { match self.obj_store_request(path, range.clone()).send().await { - Ok(resp) => Ok(resp), + Ok(resp) => { + let val = resp + .body + .collect() + .await + .map_err(either::Right)? + .into_bytes(); + Ok(val) + } Err(err) => { if let SdkError::DispatchFailure(e) = &err && e.is_timeout() @@ -369,7 +378,7 @@ impl ObjectStore for S3ObjectStore { .inc(); } - Err(err) + Err(either::Left(err)) } } }, @@ -377,8 +386,6 @@ impl ObjectStore for S3ObjectStore { ) .await?; - let val = resp.body.collect().await?.into_bytes(); - if let Some(len) = range.len() && len != val.len() { return Err(ObjectError::internal(format!( "mismatched size: expected {}, found {} when reading {} at {:?}", @@ -445,7 +452,7 @@ impl ObjectStore for S3ObjectStore { .inc(); } - Err(err) + Err(either::Left(err)) } } }, @@ -768,13 +775,23 @@ impl S3ObjectStore { } #[inline(always)] - fn should_retry(err: &SdkError) -> bool { - if let SdkError::DispatchFailure(e) = err { - if e.is_timeout() { - tracing::warn!(target: "http_timeout_retry", "{:?} occurs, trying to retry S3 get_object request.", e); + fn should_retry(err: &Either, ByteStreamError>) -> bool { + match err { + Either::Left(err) => { + if let SdkError::DispatchFailure(e) = err { + if e.is_timeout() { + tracing::warn!(target: "http_timeout_retry", "{:?} occurs, trying to retry S3 get_object request.", e); + return true; + } + } + } + Either::Right(_) => { + // Unfortunately `ErrorKind` of `ByteStreamError` is not accessible. + // Always returns true and relies on req_retry_max_attempts to avoid infinite loop. return true; } } + false } } @@ -914,6 +931,15 @@ impl Stream for S3ObjectIter { } } +impl From, ByteStreamError>> for ObjectError { + fn from(e: Either, ByteStreamError>) -> Self { + match e { + Either::Left(e) => e.into(), + Either::Right(e) => e.into(), + } + } +} + #[cfg(test)] #[cfg(not(madsim))] mod tests { From a27e876cdb20692fcac2862130ac6a1c0454368c Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Thu, 26 Oct 2023 11:18:21 -0400 Subject: [PATCH 41/48] fix(stream): fix the minput's indicies when it's for distinct call (#13015) Signed-off-by: Richard Chien Co-authored-by: Richard Chien Co-authored-by: Bugen Zhao --- e2e_test/streaming/bug_fixes/issue_12140.slt | 75 +++++++++++++ proto/stream_plan.proto | 12 +++ .../tests/testdata/input/agg.yaml | 6 ++ .../tests/testdata/output/agg.yaml | 102 ++++++++++++++++++ .../src/optimizer/plan_node/generic/agg.rs | 19 +++- .../optimizer/plan_node/stream_hash_agg.rs | 1 + .../optimizer/plan_node/stream_simple_agg.rs | 1 + .../plan_node/stream_stateless_simple_agg.rs | 1 + src/stream/src/executor/agg_common.rs | 3 + .../src/executor/aggregation/agg_group.rs | 5 + .../src/executor/aggregation/agg_state.rs | 3 + src/stream/src/executor/aggregation/minput.rs | 38 ++++++- src/stream/src/executor/hash_agg.rs | 7 ++ src/stream/src/executor/simple_agg.rs | 33 +++--- src/stream/src/executor/test_utils.rs | 5 + src/stream/src/from_proto/hash_agg.rs | 2 + src/stream/src/from_proto/simple_agg.rs | 2 + 17 files changed, 296 insertions(+), 19 deletions(-) create mode 100644 e2e_test/streaming/bug_fixes/issue_12140.slt diff --git a/e2e_test/streaming/bug_fixes/issue_12140.slt b/e2e_test/streaming/bug_fixes/issue_12140.slt new file mode 100644 index 0000000000000..2240762868832 --- /dev/null +++ b/e2e_test/streaming/bug_fixes/issue_12140.slt @@ -0,0 +1,75 @@ +# https://github.com/risingwavelabs/risingwave/issues/12140 + +statement ok +CREATE TABLE t (c3 INT, c9 CHARACTER VARYING); + +statement ok +INSERT INTO t VALUES (1, 'interesting'), (2, 'interesting'), (3, 'interesting'), (4, 'IwfwuseZmg'), (5, 'ZuT4aIQVhA'); + +statement ok +CREATE MATERIALIZED VIEW mv AS +SELECT + first_value(DISTINCT t.c9 ORDER BY t.c9 ASC) +FROM + t; + +statement ok +DELETE FROM t WHERE c3 = 1; + +statement ok +DELETE FROM t WHERE c3 = 2; + +statement ok +DELETE FROM t WHERE c3 = 3; + +statement ok +drop materialized view mv; + +statement ok +drop table t; + +statement ok +CREATE TABLE t (c3 INT, c9 CHARACTER VARYING); + +statement ok +INSERT INTO t VALUES (1, 'interesting'), (2, 'interesting'), (3, 'interesting'), (1, 'boring'), (2, 'boring'), (3, 'boring'), (1, 'exciting'), (2, 'exciting'), (3, 'exciting'), (4, 'IwfwuseZmg'), (5, 'ZuT4aIQVhA'); + +statement ok +CREATE MATERIALIZED VIEW mv AS +SELECT + first_value(DISTINCT t.c9 ORDER BY t.c9 ASC), last_value(distinct c3 order by c3 asc) +FROM + t; + +statement ok +DELETE FROM t WHERE c3 = 1 and c9 = 'interesting'; + +statement ok +DELETE FROM t WHERE c3 = 2 and c9 = 'interesting'; + +statement ok +DELETE FROM t WHERE c3 = 3 and c9 = 'interesting'; + +statement ok +DELETE FROM t WHERE c3 = 1 and c9 = 'boring'; + +statement ok +DELETE FROM t WHERE c3 = 1 and c9 = 'exciting'; + +statement ok +DELETE FROM t WHERE c3 = 2 and c9 = 'boring'; + +statement ok +DELETE FROM t WHERE c3 = 2 and c9 = 'exciting'; + +statement ok +DELETE FROM t WHERE c3 = 3 and c9 = 'boring'; + +statement ok +DELETE FROM t WHERE c3 = 3 and c9 = 'exciting'; + +statement ok +drop materialized view mv; + +statement ok +drop table t; \ No newline at end of file diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 683a43ef6e9be..97f0c72937cda 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -268,6 +268,16 @@ message AggCallState { reserved "table_state"; } +enum AggNodeVersion { + AGG_NODE_VERSION_UNSPECIFIED = 0; + + // https://github.com/risingwavelabs/risingwave/issues/12140#issuecomment-1776289808 + AGG_NODE_VERSION_ISSUE_12140 = 1; + + // Used for test only. + AGG_NODE_VERSION_MAX = 2147483647; +} + message SimpleAggNode { repeated expr.AggCall agg_calls = 1; // Only used for stateless simple agg. @@ -279,6 +289,7 @@ message SimpleAggNode { bool is_append_only = 5; map distinct_dedup_tables = 6; uint32 row_count_index = 7; + AggNodeVersion version = 8; } message HashAggNode { @@ -292,6 +303,7 @@ message HashAggNode { map distinct_dedup_tables = 6; uint32 row_count_index = 7; bool emit_on_window_close = 8; + AggNodeVersion version = 9; } message TopNNode { diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index 99aa94ff773b9..2b69b5a53145d 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -912,6 +912,12 @@ expected_outputs: - batch_plan - stream_plan + - stream_dist_plan # check the state table schema +- sql: | + create table t (x int, y int); + select first_value(distinct x order by x asc) from t; + expected_outputs: + - stream_dist_plan # check the state table schema - sql: | create table t (x int, y int); select last_value(x order by y desc nulls last) from t; diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index baa77dc79c89b..058fa2a794d1e 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1613,6 +1613,108 @@ └─StreamSimpleAgg { aggs: [first_value(t.x order_by(t.y ASC)), count] } └─StreamExchange { dist: Single } └─StreamTableScan { table: t, columns: [t.x, t.y, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [first_value], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── materialized table: 4294967294 + └── StreamProject { exprs: [first_value(t.x order_by(t.y ASC))] } + └── StreamSimpleAgg { aggs: [first_value(t.x order_by(t.y ASC)), count] } + ├── intermediate state table: 1 + ├── state tables: [ 0 ] + ├── distinct tables: [] + └── StreamExchange Single from 1 + + Fragment 1 + Chain { table: t, columns: [t.x, t.y, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + ├── state table: 2 + ├── Upstream + └── BatchPlanNode + + Table 0 + ├── columns: [ t_y, t__row_id, t_x ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 1 + ├── columns: [ first_value(t_x order_by(t_y ASC)), count ] + ├── primary key: [] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 2 + ├── columns: [ vnode, _row_id, t_backfill_finished, t_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 4294967294 + ├── columns: [ first_value ] + ├── primary key: [] + ├── value indices: [ 0 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + +- sql: | + create table t (x int, y int); + select first_value(distinct x order by x asc) from t; + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [first_value], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── materialized table: 4294967294 + └── StreamProject { exprs: [first_value(distinct t.x order_by(t.x ASC))] } + └── StreamSimpleAgg { aggs: [first_value(distinct t.x order_by(t.x ASC)), count] } + ├── intermediate state table: 1 + ├── state tables: [ 0 ] + ├── distinct tables: [ (distinct key: t.x, table id: 2) ] + └── StreamExchange Single from 1 + + Fragment 1 + Chain { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + ├── state table: 3 + ├── Upstream + └── BatchPlanNode + + Table 0 + ├── columns: [ t_x ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 1 + ├── columns: [ first_value(distinct t_x order_by(t_x ASC)), count ] + ├── primary key: [] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 2 + ├── columns: [ t_x, count_for_agg_call_0 ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 1 + + Table 3 + ├── columns: [ vnode, _row_id, t_backfill_finished, t_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 4294967294 + ├── columns: [ first_value ] + ├── primary key: [] + ├── value indices: [ 0 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + - sql: | create table t (x int, y int); select last_value(x order by y desc nulls last) from t; diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 98109e695110f..c7a0ea4f39389 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -13,7 +13,7 @@ // limitations under the License. use std::collections::{BTreeMap, BTreeSet, HashMap}; -use std::fmt; +use std::{fmt, vec}; use fixedbitset::FixedBitSet; use itertools::{Either, Itertools}; @@ -348,6 +348,7 @@ impl Agg { let in_dist_key = self.input.distribution().dist_column_indices().to_vec(); let gen_materialized_input_state = |sort_keys: Vec<(OrderType, usize)>, + extra_keys: Vec, include_keys: Vec| -> MaterializedInputState { let (mut table_builder, mut included_upstream_indices, mut column_mapping) = @@ -375,7 +376,7 @@ impl Agg { for (order_type, idx) in sort_keys { add_column(idx, Some(order_type), true, &mut table_builder); } - for &idx in &in_pks { + for idx in extra_keys { add_column(idx, Some(OrderType::ascending()), true, &mut table_builder); } for idx in include_keys { @@ -458,6 +459,17 @@ impl Agg { _ => unreachable!(), } }; + + // columns to ensure each row unique + let extra_keys = if agg_call.distinct { + // if distinct, use distinct keys as extra keys + let distinct_key = agg_call.inputs[0].index; + vec![distinct_key] + } else { + // if not distinct, use primary keys as extra keys + in_pks.clone() + }; + // other columns that should be contained in state table let include_keys = match agg_call.agg_kind { AggKind::FirstValue @@ -470,7 +482,8 @@ impl Agg { } _ => vec![], }; - let state = gen_materialized_input_state(sort_keys, include_keys); + + let state = gen_materialized_input_state(sort_keys, extra_keys, include_keys); AggCallState::MaterializedInput(Box::new(state)) } agg_kinds::rewritten!() => { diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 6ef8849b1e142..cb181677c6aaa 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -217,6 +217,7 @@ impl StreamNode for StreamHashAgg { .collect(), row_count_index: self.row_count_idx as u32, emit_on_window_close: self.base.emit_on_window_close(), + version: PbAggNodeVersion::Issue12140 as _, }) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs index 06aa227d7e316..ff590e9695bb9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -128,6 +128,7 @@ impl StreamNode for StreamSimpleAgg { }) .collect(), row_count_index: self.row_count_idx as u32, + version: PbAggNodeVersion::Issue12140 as _, }) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs index ca076132f3a55..e9db33b13b626 100644 --- a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs @@ -100,6 +100,7 @@ impl StreamNode for StreamStatelessSimpleAgg { intermediate_state_table: None, is_append_only: self.input().append_only(), distinct_dedup_tables: Default::default(), + version: AggNodeVersion::Issue12140 as _, }) } } diff --git a/src/stream/src/executor/agg_common.rs b/src/stream/src/executor/agg_common.rs index fbaa80c3fbeb7..d1ea23068d430 100644 --- a/src/stream/src/executor/agg_common.rs +++ b/src/stream/src/executor/agg_common.rs @@ -16,6 +16,7 @@ use std::collections::HashMap; use std::sync::Arc; use risingwave_expr::aggregate::AggCall; +use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; use super::aggregation::AggStateStorage; @@ -27,6 +28,8 @@ use crate::task::AtomicU64Ref; /// Arguments needed to construct an `XxxAggExecutor`. pub struct AggExecutorArgs { + pub version: PbAggNodeVersion, + // basic pub input: Box, pub actor_ctx: ActorContextRef, diff --git a/src/stream/src/executor/aggregation/agg_group.rs b/src/stream/src/executor/aggregation/agg_group.rs index d854969120919..d0e97cd4783e9 100644 --- a/src/stream/src/executor/aggregation/agg_group.rs +++ b/src/stream/src/executor/aggregation/agg_group.rs @@ -25,6 +25,7 @@ use risingwave_common::must_match; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::aggregate::{AggCall, BoxedAggregateFunction}; +use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; use super::agg_state::{AggState, AggStateStorage}; @@ -192,6 +193,7 @@ impl AggGroup { /// For [`crate::executor::SimpleAggExecutor`], the `group_key` should be `None`. #[allow(clippy::too_many_arguments)] pub async fn create( + version: PbAggNodeVersion, group_key: Option, agg_calls: &[AggCall], agg_funcs: &[BoxedAggregateFunction], @@ -212,6 +214,7 @@ impl AggGroup { let mut states = Vec::with_capacity(agg_calls.len()); for (idx, (agg_call, agg_func)) in agg_calls.iter().zip_eq_fast(agg_funcs).enumerate() { let state = AggState::create( + version, agg_call, agg_func, &storages[idx], @@ -242,6 +245,7 @@ impl AggGroup { /// Create a group from encoded states for EOWC. The previous output is set to `None`. #[allow(clippy::too_many_arguments)] pub fn create_eowc( + version: PbAggNodeVersion, group_key: Option, agg_calls: &[AggCall], agg_funcs: &[BoxedAggregateFunction], @@ -255,6 +259,7 @@ impl AggGroup { let mut states = Vec::with_capacity(agg_calls.len()); for (idx, (agg_call, agg_func)) in agg_calls.iter().zip_eq_fast(agg_funcs).enumerate() { let state = AggState::create( + version, agg_call, agg_func, &storages[idx], diff --git a/src/stream/src/executor/aggregation/agg_state.rs b/src/stream/src/executor/aggregation/agg_state.rs index 0c1932c58831c..a0413ed4491d2 100644 --- a/src/stream/src/executor/aggregation/agg_state.rs +++ b/src/stream/src/executor/aggregation/agg_state.rs @@ -19,6 +19,7 @@ use risingwave_common::estimate_size::EstimateSize; use risingwave_common::must_match; use risingwave_common::types::Datum; use risingwave_expr::aggregate::{AggCall, AggregateState, BoxedAggregateFunction}; +use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; use super::minput::MaterializedInputState; @@ -65,6 +66,7 @@ impl AggState { /// Create an [`AggState`] from a given [`AggCall`]. #[allow(clippy::too_many_arguments)] pub fn create( + version: PbAggNodeVersion, agg_call: &AggCall, agg_func: &BoxedAggregateFunction, storage: &AggStateStorage, @@ -83,6 +85,7 @@ impl AggState { } AggStateStorage::MaterializedInput { mapping, .. } => { Self::MaterializedInput(Box::new(MaterializedInputState::new( + version, agg_call, pk_indices, mapping, diff --git a/src/stream/src/executor/aggregation/minput.rs b/src/stream/src/executor/aggregation/minput.rs index 1329f08eb6d99..2392e53d7d49a 100644 --- a/src/stream/src/executor/aggregation/minput.rs +++ b/src/stream/src/executor/aggregation/minput.rs @@ -25,6 +25,7 @@ use risingwave_common::types::Datum; use risingwave_common::util::row_serde::OrderedRowSerde; use risingwave_common::util::sort_util::OrderType; use risingwave_expr::aggregate::{AggCall, AggKind, BoxedAggregateFunction}; +use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; @@ -68,6 +69,7 @@ pub struct MaterializedInputState { impl MaterializedInputState { /// Create an instance from [`AggCall`]. pub fn new( + version: PbAggNodeVersion, agg_call: &AggCall, pk_indices: &PkIndices, col_mapping: &StateTableColumnMapping, @@ -102,9 +104,26 @@ impl MaterializedInputState { .unzip() }; - let pk_len = pk_indices.len(); - order_col_indices.extend(pk_indices.iter()); - order_types.extend(itertools::repeat_n(OrderType::ascending(), pk_len)); + if agg_call.distinct { + if version < PbAggNodeVersion::Issue12140 { + panic!( + "RisingWave versions before issue #12140 is resolved has critical bug, you must re-create current MV to ensure correctness." + ); + } + + // If distinct, we need to materialize input with the distinct keys + // As we only support single-column distinct for now, we use the + // `agg_call.args.val_indices()[0]` as the distinct key. + if !order_col_indices.contains(&agg_call.args.val_indices()[0]) { + order_col_indices.push(agg_call.args.val_indices()[0]); + order_types.push(OrderType::ascending()); + } + } else { + // If not distinct, we need to materialize input with the primary keys + let pk_len = pk_indices.len(); + order_col_indices.extend(pk_indices.iter()); + order_types.extend(itertools::repeat_n(OrderType::ascending(), pk_len)); + } // map argument columns to state table column indices let state_table_arg_col_indices = arg_col_indices @@ -255,6 +274,7 @@ mod tests { use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::sort_util::OrderType; use risingwave_expr::aggregate::{build_append_only, AggCall}; + use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::StateStore; @@ -327,6 +347,7 @@ mod tests { .await; let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -379,6 +400,7 @@ mod tests { { // test recovery (cold start) let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -420,6 +442,7 @@ mod tests { .await; let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -472,6 +495,7 @@ mod tests { { // test recovery (cold start) let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -529,6 +553,7 @@ mod tests { table_2.init_epoch(epoch); let mut state_1 = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call_1, &input_pk_indices, &mapping_1, @@ -538,6 +563,7 @@ mod tests { .unwrap(); let mut state_2 = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call_2, &input_pk_indices, &mapping_2, @@ -621,6 +647,7 @@ mod tests { .await; let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -672,6 +699,7 @@ mod tests { { // test recovery (cold start) let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -715,6 +743,7 @@ mod tests { table.init_epoch(epoch); let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -814,6 +843,7 @@ mod tests { .await; let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -921,6 +951,7 @@ mod tests { .await; let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -1000,6 +1031,7 @@ mod tests { .await; let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index cb62e8d8f94aa..8d02cc328fa43 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -29,6 +29,7 @@ use risingwave_common::types::ScalarImpl; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::aggregate::{build_retractable, AggCall, BoxedAggregateFunction}; +use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; use super::agg_common::{AggExecutorArgs, HashAggExecutorExtraArgs}; @@ -81,6 +82,9 @@ pub struct HashAggExecutor { struct ExecutorInner { _phantom: PhantomData, + /// Version of aggregation executors. + version: PbAggNodeVersion, + actor_ctx: ActorContextRef, info: ExecutorInfo, @@ -233,6 +237,7 @@ impl HashAggExecutor { input: args.input, inner: ExecutorInner { _phantom: PhantomData, + version: args.version, actor_ctx: args.actor_ctx, info: ExecutorInfo { schema, @@ -318,6 +323,7 @@ impl HashAggExecutor { // Create `AggGroup` for the current group if not exists. This will // restore agg states from the intermediate state table. let agg_group = AggGroup::create( + this.version, Some(GroupKey::new( key.deserialize(group_key_types)?, Some(this.group_key_table_pk_projection.clone()), @@ -466,6 +472,7 @@ impl HashAggExecutor { let states = row.into_iter().skip(this.group_key_indices.len()).collect(); let mut agg_group = AggGroup::create_eowc( + this.version, Some(GroupKey::new( group_key, Some(this.group_key_table_pk_projection.clone()), diff --git a/src/stream/src/executor/simple_agg.rs b/src/stream/src/executor/simple_agg.rs index 6e88241f48433..92730218ca148 100644 --- a/src/stream/src/executor/simple_agg.rs +++ b/src/stream/src/executor/simple_agg.rs @@ -18,6 +18,7 @@ use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::aggregate::{build_retractable, AggCall, BoxedAggregateFunction}; +use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; use super::agg_common::{AggExecutorArgs, SimpleAggExecutorExtraArgs}; @@ -52,6 +53,9 @@ pub struct SimpleAggExecutor { } struct ExecutorInner { + /// Version of aggregation executors. + version: PbAggNodeVersion, + actor_ctx: ActorContextRef, info: ExecutorInfo, @@ -135,6 +139,7 @@ impl SimpleAggExecutor { Ok(Self { input: args.input, inner: ExecutorInner { + version: args.version, actor_ctx: args.actor_ctx, info: ExecutorInfo { schema, @@ -257,9 +262,23 @@ impl SimpleAggExecutor { table.init_epoch(barrier.epoch); }); + let mut distinct_dedup = DistinctDeduplicater::new( + &this.agg_calls, + &this.watermark_epoch, + &this.distinct_dedup_tables, + this.actor_ctx.id, + this.metrics.clone(), + ); + distinct_dedup.dedup_caches_mut().for_each(|cache| { + cache.update_epoch(barrier.epoch.curr); + }); + + yield Message::Barrier(barrier); + let mut vars = ExecutionVars { // This will fetch previous agg states from the intermediate state table. agg_group: AggGroup::create( + this.version, None, &this.agg_calls, &this.agg_funcs, @@ -271,22 +290,10 @@ impl SimpleAggExecutor { &this.input_schema, ) .await?, - distinct_dedup: DistinctDeduplicater::new( - &this.agg_calls, - &this.watermark_epoch, - &this.distinct_dedup_tables, - this.actor_ctx.id, - this.metrics.clone(), - ), + distinct_dedup, state_changed: false, }; - vars.distinct_dedup.dedup_caches_mut().for_each(|cache| { - cache.update_epoch(barrier.epoch.curr); - }); - - yield Message::Barrier(barrier); - #[for_await] for msg in input { let msg = msg?; diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index 13a9237cf0159..2ae461e62351c 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -280,6 +280,7 @@ pub mod agg_executor { use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; use risingwave_expr::aggregate::{AggCall, AggKind}; + use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; @@ -444,6 +445,8 @@ pub mod agg_executor { .await; HashAggExecutor::::new(AggExecutorArgs { + version: PbAggNodeVersion::Max, + input, actor_ctx: ActorContext::create(123), pk_indices, @@ -507,6 +510,8 @@ pub mod agg_executor { .await; SimpleAggExecutor::new(AggExecutorArgs { + version: PbAggNodeVersion::Max, + input, actor_ctx, pk_indices, diff --git a/src/stream/src/from_proto/hash_agg.rs b/src/stream/src/from_proto/hash_agg.rs index a369f8124ebfb..faf8a1f7fdad1 100644 --- a/src/stream/src/from_proto/hash_agg.rs +++ b/src/stream/src/from_proto/hash_agg.rs @@ -97,6 +97,8 @@ impl ExecutorBuilder for HashAggExecutorBuilder { HashAggExecutorDispatcherArgs { args: AggExecutorArgs { + version: node.version(), + input, actor_ctx: params.actor_context, pk_indices: params.pk_indices, diff --git a/src/stream/src/from_proto/simple_agg.rs b/src/stream/src/from_proto/simple_agg.rs index 5423e4fd2043f..fdd6d877b99ed 100644 --- a/src/stream/src/from_proto/simple_agg.rs +++ b/src/stream/src/from_proto/simple_agg.rs @@ -58,6 +58,8 @@ impl ExecutorBuilder for SimpleAggExecutorBuilder { .await; Ok(SimpleAggExecutor::new(AggExecutorArgs { + version: node.version(), + input, actor_ctx: params.actor_context, pk_indices: params.pk_indices, From 4a8b0cb8d0da71963c7b449898c342a6c0353e89 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 26 Oct 2023 23:30:04 +0800 Subject: [PATCH 42/48] feat(expr): `substring`/`substr` for `bytea` (#13088) --- e2e_test/batch/functions/substr.slt.part | 10 +++++++++ src/expr/impl/src/scalar/substr.rs | 26 ++++++++++++++++++++++-- 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/e2e_test/batch/functions/substr.slt.part b/e2e_test/batch/functions/substr.slt.part index 4f87043da3e86..22a495a3f8a87 100644 --- a/e2e_test/batch/functions/substr.slt.part +++ b/e2e_test/batch/functions/substr.slt.part @@ -27,3 +27,13 @@ query T select substr('a', 2147483646, 1); ---- (empty) + +query T +select substr('abcde'::bytea, 2, 7); +---- +\x62636465 + +query T +select substr('abcde'::bytea, -2, 5); +---- +\x6162 diff --git a/src/expr/impl/src/scalar/substr.rs b/src/expr/impl/src/scalar/substr.rs index dc2829c5d8a52..2b65575c91c19 100644 --- a/src/expr/impl/src/scalar/substr.rs +++ b/src/expr/impl/src/scalar/substr.rs @@ -28,8 +28,14 @@ pub fn substr_start(s: &str, start: i32, writer: &mut impl Write) -> Result<()> Ok(()) } -#[function("substr(varchar, int4, int4) -> varchar")] -pub fn substr_start_for(s: &str, start: i32, count: i32, writer: &mut impl Write) -> Result<()> { +#[function("substr(bytea, int4) -> bytea")] +pub fn substr_start_bytea(s: &[u8], start: i32) -> Box<[u8]> { + let skip = start.saturating_sub(1).max(0) as usize; + + s.iter().copied().skip(skip).collect() +} + +fn convert_args(start: i32, count: i32) -> Result<(usize, usize)> { if count < 0 { return Err(ExprError::InvalidParam { name: "length", @@ -44,6 +50,15 @@ pub fn substr_start_for(s: &str, start: i32, count: i32, writer: &mut impl Write count.saturating_add(start.saturating_sub(1)).max(0) as usize }; + // The returned args may still go out of bounds. + // So `skip` and `take` on iterator is safer than `[skip..(skip+take)]` + Ok((skip, take)) +} + +#[function("substr(varchar, int4, int4) -> varchar")] +pub fn substr_start_for(s: &str, start: i32, count: i32, writer: &mut impl Write) -> Result<()> { + let (skip, take) = convert_args(start, count)?; + let substr = s.chars().skip(skip).take(take); for char in substr { writer.write_char(char).unwrap(); @@ -52,6 +67,13 @@ pub fn substr_start_for(s: &str, start: i32, count: i32, writer: &mut impl Write Ok(()) } +#[function("substr(bytea, int4, int4) -> bytea")] +pub fn substr_start_for_bytea(s: &[u8], start: i32, count: i32) -> Result> { + let (skip, take) = convert_args(start, count)?; + + Ok(s.iter().copied().skip(skip).take(take).collect()) +} + #[cfg(test)] mod tests { use super::*; From da6626da5245c363614069ca60f8e7af4d5c6f65 Mon Sep 17 00:00:00 2001 From: TennyZhuang Date: Fri, 27 Oct 2023 01:12:49 +0800 Subject: [PATCH 43/48] chore(deps): bump rustix 0.37.23 to 0.37.26 reported by dependabot alerts (#13092) Signed-off-by: TennyZhuang Co-authored-by: xxchan --- Cargo.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a2a8941e5905a..b303658d3c57f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -480,7 +480,7 @@ dependencies = [ "log", "parking", "polling", - "rustix 0.37.23", + "rustix 0.37.26", "slab", "socket2 0.4.9", "waker-fn", @@ -8384,9 +8384,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.37.23" +version = "0.37.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d69718bf81c6127a49dc64e44a742e8bb9213c0ff8869a22c308f84c1d4ab06" +checksum = "84f3f8f960ed3b5a59055428714943298bf3fa2d4a1d53135084e0544829d995" dependencies = [ "bitflags 1.3.2", "errno", From 23b69c47290a29267694a1ea362b848faaa792d6 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Fri, 27 Oct 2023 11:15:47 +0800 Subject: [PATCH 44/48] feat(expr): support `int8send` and `int8recv` (#13077) --- .../batch/functions/pgwire_send_recv.slt.part | 40 +++++++++++++++++++ proto/expr.proto | 2 + src/expr/impl/src/scalar/cast.rs | 14 +++++++ src/frontend/src/binder/expr/function.rs | 13 ++++++ src/frontend/src/expr/pure.rs | 2 + src/tests/sqlsmith/src/validation.rs | 5 +++ 6 files changed, 76 insertions(+) create mode 100644 e2e_test/batch/functions/pgwire_send_recv.slt.part diff --git a/e2e_test/batch/functions/pgwire_send_recv.slt.part b/e2e_test/batch/functions/pgwire_send_recv.slt.part new file mode 100644 index 0000000000000..47fea3a4fb19b --- /dev/null +++ b/e2e_test/batch/functions/pgwire_send_recv.slt.part @@ -0,0 +1,40 @@ +query TT +select int8send(2147483647); +---- +\x000000007fffffff + +query I +select int8recv(int8send(76)); +---- +76 + +query I +select int8recv(' a'::bytea); +---- +2314885530818453601 + +statement error could not convert slice to array +select int8recv('a'::bytea); + +query I +select int8recv(decode(substr(md5(''), 3, 16), 'hex')); +---- +2129315932054619369 + +query I +select int8recv(substr(decode(md5(''), 'hex'), 2, 8)); +---- +2129315932054619369 + +query I +select int8recv(substr(sha256(''), 2, 8)); +---- +-5709365202766785382 + +statement error decode +select int8recv(to_hex(2129315932054619369)); + +query I +select int8recv(decode(to_hex(2129315932054619369), 'hex')); +---- +2129315932054619369 diff --git a/proto/expr.proto b/proto/expr.proto index 2f252d67c8400..9c6f3598032f9 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -173,6 +173,8 @@ message ExprNode { LEFT = 317; RIGHT = 318; FORMAT = 319; + PGWIRE_SEND = 320; + PGWIRE_RECV = 321; // Unary operators NEG = 401; diff --git a/src/expr/impl/src/scalar/cast.rs b/src/expr/impl/src/scalar/cast.rs index c173c76c330c5..f22b643bb9a09 100644 --- a/src/expr/impl/src/scalar/cast.rs +++ b/src/expr/impl/src/scalar/cast.rs @@ -47,6 +47,14 @@ where .map_err(|err: ::Err| ExprError::Parse(err.to_string().into())) } +// TODO: introduce `FromBinary` and support all types +#[function("pgwire_recv(bytea) -> int8")] +pub fn pgwire_recv(elem: &[u8]) -> Result { + let fixed_length = + <[u8; 8]>::try_from(elem).map_err(|e| ExprError::Parse(e.to_string().into()))?; + Ok(i64::from_be_bytes(fixed_length)) +} + #[function("cast(int2) -> int256")] #[function("cast(int4) -> int256")] #[function("cast(int8) -> int256")] @@ -156,6 +164,12 @@ pub fn general_to_text(elem: impl ToText, mut writer: &mut impl Write) { elem.write(&mut writer).unwrap(); } +// TODO: use `ToBinary` and support all types +#[function("pgwire_send(int8) -> bytea")] +fn pgwire_send(elem: i64) -> Box<[u8]> { + elem.to_be_bytes().into() +} + #[function("cast(boolean) -> varchar")] pub fn bool_to_varchar(input: bool, writer: &mut impl Write) { writer diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 18438b28c0a98..0b8457bc1cb82 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -802,6 +802,19 @@ impl Binder { ("sha512", raw_call(ExprType::Sha512)), ("left", raw_call(ExprType::Left)), ("right", raw_call(ExprType::Right)), + ("int8send", raw_call(ExprType::PgwireSend)), + ("int8recv", guard_by_len(1, raw(|_binder, mut inputs| { + // Similar to `cast` from string, return type is set explicitly rather than inferred. + let hint = if !inputs[0].is_untyped() && inputs[0].return_type() == DataType::Varchar { + " Consider `decode` or cast." + } else { + "" + }; + inputs[0].cast_implicit_mut(DataType::Bytea).map_err(|e| { + ErrorCode::BindError(format!("{e} in `recv`.{hint}")) + })?; + Ok(FunctionCall::new_unchecked(ExprType::PgwireRecv, inputs, DataType::Int64).into()) + }))), // array ("array_cat", raw_call(ExprType::ArrayCat)), ("array_append", raw_call(ExprType::ArrayAppend)), diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index e5d698c2ce172..470e1efc6aba0 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -200,6 +200,8 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::ArrayPositions | expr_node::Type::StringToArray | expr_node::Type::Format + | expr_node::Type::PgwireSend + | expr_node::Type::PgwireRecv | expr_node::Type::ArrayTransform | expr_node::Type::Greatest | expr_node::Type::Least => diff --git a/src/tests/sqlsmith/src/validation.rs b/src/tests/sqlsmith/src/validation.rs index 7c88320b8d4ca..5195444d079f5 100644 --- a/src/tests/sqlsmith/src/validation.rs +++ b/src/tests/sqlsmith/src/validation.rs @@ -30,6 +30,10 @@ fn is_numeric_out_of_range_err(db_error: &str) -> bool { || db_error.contains("Casting to u32 out of range") } +fn is_parse_err(db_error: &str) -> bool { + db_error.contains("Parse error") +} + /// Skip queries with unimplemented features fn is_unimplemented_error(db_error: &str) -> bool { db_error.contains("not yet implemented") @@ -106,6 +110,7 @@ pub fn is_neg_exp_error(db_error: &str) -> bool { pub fn is_permissible_error(db_error: &str) -> bool { is_numeric_out_of_range_err(db_error) || is_zero_err(db_error) + || is_parse_err(db_error) || is_unimplemented_error(db_error) || not_unique_error(db_error) || is_window_error(db_error) From b2eeca1176de9ba4c87f2040b44c02f125241c0e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 27 Oct 2023 12:25:45 +0800 Subject: [PATCH 45/48] chore(deps): Bump ethnum from 1.4.0 to 1.5.0 (#13096) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b303658d3c57f..a928849d0e03c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2832,9 +2832,9 @@ dependencies = [ [[package]] name = "ethnum" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c8ff382b2fa527fb7fb06eeebfc5bbb3f17e3cc6b9d70b006c41daa8824adac" +checksum = "b90ca2580b73ab6a1f724b76ca11ab632df820fd6040c336200d2c1df7b3c82c" dependencies = [ "serde", ] From 00df63562dea12faddd4a2ccd87fe7ae08cf9ba7 Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 27 Oct 2023 12:26:35 +0800 Subject: [PATCH 46/48] chore: disable tests by default for some heavy lib/bins without UTs (#12950) --- src/cmd/Cargo.toml | 8 ++++++++ src/cmd_all/Cargo.toml | 1 + src/meta/node/Cargo.toml | 3 +++ 3 files changed, 12 insertions(+) diff --git a/src/cmd/Cargo.toml b/src/cmd/Cargo.toml index 8ba72d6a24af4..de69cc3574509 100644 --- a/src/cmd/Cargo.toml +++ b/src/cmd/Cargo.toml @@ -49,25 +49,33 @@ workspace-hack = { path = "../workspace-hack" } [target.'cfg(enable_task_local_alloc)'.dependencies] task_stats_alloc = { path = "../utils/task_stats_alloc" } +[lib] +test = false + [[bin]] name = "frontend" path = "src/bin/frontend_node.rs" +test = false [[bin]] name = "meta-node" path = "src/bin/meta_node.rs" +test = false [[bin]] name = "compute-node" path = "src/bin/compute_node.rs" +test = false [[bin]] name = "compactor" path = "src/bin/compactor.rs" +test = false [[bin]] name = "risectl" path = "src/bin/ctl.rs" +test = false [lints] workspace = true diff --git a/src/cmd_all/Cargo.toml b/src/cmd_all/Cargo.toml index 9a4b34c094196..e5aa9e3c68d04 100644 --- a/src/cmd_all/Cargo.toml +++ b/src/cmd_all/Cargo.toml @@ -69,6 +69,7 @@ task_stats_alloc = { path = "../utils/task_stats_alloc" } [[bin]] name = "risingwave" path = "src/bin/risingwave.rs" +test = false [lints] workspace = true diff --git a/src/meta/node/Cargo.toml b/src/meta/node/Cargo.toml index dcfa053fdfc4b..e7597c0d041f3 100644 --- a/src/meta/node/Cargo.toml +++ b/src/meta/node/Cargo.toml @@ -54,5 +54,8 @@ workspace-hack = { path = "../../workspace-hack" } [dev-dependencies] +[lib] +test = false + [lints] workspace = true From 6b7f863e51601f1098caf7abcada94eef1fc70de Mon Sep 17 00:00:00 2001 From: Dylan Date: Fri, 27 Oct 2023 13:34:34 +0800 Subject: [PATCH 47/48] fix(binder): only table-in-out functions can have subquery parameters (#13098) --- .../tests/testdata/input/subquery.yaml | 10 ++++++++++ .../tests/testdata/output/subquery.yaml | 13 +++++++++++++ .../src/binder/relation/table_function.rs | 16 +++++++++++++++- 3 files changed, 38 insertions(+), 1 deletion(-) diff --git a/src/frontend/planner_test/tests/testdata/input/subquery.yaml b/src/frontend/planner_test/tests/testdata/input/subquery.yaml index b1d58c5e422ed..47785f0234271 100644 --- a/src/frontend/planner_test/tests/testdata/input/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/input/subquery.yaml @@ -291,3 +291,13 @@ expected_outputs: - batch_plan - stream_plan +- name: Only table-in-out functions can have subquery parameters. + sql: | + SELECT * FROM generate_series(1, (select 1)); + expected_outputs: + - binder_error +- name: While this one is allowed. + sql: | + SELECT generate_series(1, (select 1)); + expected_outputs: + - batch_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index e07e84e040929..914d9b764c5e1 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -864,3 +864,16 @@ └─StreamHopWindow { time_col: t1.ts, slide: 00:10:00, size: 00:30:00, output: all } └─StreamFilter { predicate: IsNotNull(t1.ts) } └─StreamTableScan { table: t1, columns: [t1.k, t1.ts], pk: [t1.k], dist: UpstreamHashShard(t1.k) } +- name: Only table-in-out functions can have subquery parameters. + sql: | + SELECT * FROM generate_series(1, (select 1)); + binder_error: 'Invalid input syntax: Only table-in-out functions can have subquery parameters, generate_series only accepts constant parameters' +- name: While this one is allowed. + sql: | + SELECT generate_series(1, (select 1)); + batch_plan: |- + BatchProject { exprs: [GenerateSeries(1:Int32, $0)] } + └─BatchProjectSet { select_list: [GenerateSeries(1:Int32, $0)] } + └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } + ├─BatchValues { rows: [[]] } + └─BatchValues { rows: [[1:Int32]] } diff --git a/src/frontend/src/binder/relation/table_function.rs b/src/frontend/src/binder/relation/table_function.rs index 988ea0561a860..032791bfab30c 100644 --- a/src/frontend/src/binder/relation/table_function.rs +++ b/src/frontend/src/binder/relation/table_function.rs @@ -28,7 +28,7 @@ use crate::binder::bind_context::Clause; use crate::catalog::system_catalog::pg_catalog::{ PG_GET_KEYWORDS_FUNC_NAME, PG_KEYWORDS_TABLE_NAME, }; -use crate::expr::Expr; +use crate::expr::{Expr, ExprImpl}; impl Binder { /// Binds a table function AST, which is a function call in a relation position. @@ -125,6 +125,20 @@ impl Binder { self.pop_context()?; let func = func?; + if let ExprImpl::TableFunction(func) = &func { + if func + .args + .iter() + .any(|arg| matches!(arg, ExprImpl::Subquery(_))) + { + // Same error reports as DuckDB. + return Err(ErrorCode::InvalidInputSyntax( + format!("Only table-in-out functions can have subquery parameters, {} only accepts constant parameters", func.name()), + ) + .into()); + } + } + // bool indicates if the field is hidden let mut columns = if let DataType::Struct(s) = func.return_type() { // If the table function returns a struct, it will be flattened into multiple columns. From abf3a787a74a5576ddcee0c5e5169134f5f6e18f Mon Sep 17 00:00:00 2001 From: TennyZhuang Date: Fri, 27 Oct 2023 15:02:24 +0800 Subject: [PATCH 48/48] chore(deps): bump madsim-tokio to 0.2.24 (#13091) Signed-off-by: TennyZhuang --- Cargo.lock | 4 ++-- src/tests/e2e_extended_mode/Cargo.toml | 2 +- src/tests/simulation/Cargo.toml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a928849d0e03c..8f6acdbeae3d1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4604,9 +4604,9 @@ dependencies = [ [[package]] name = "madsim-tokio" -version = "0.2.23" +version = "0.2.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3b6df81cfc5c561311fa2e85ed52e8c6f3990b700751be914797addf984e72d" +checksum = "e5611fd0eb96867dd03a9fd2494d4c1bb126f413519673195065b6ea011e8c68" dependencies = [ "madsim", "spin 0.9.8", diff --git a/src/tests/e2e_extended_mode/Cargo.toml b/src/tests/e2e_extended_mode/Cargo.toml index ea83c5069c774..56012a64a25dc 100644 --- a/src/tests/e2e_extended_mode/Cargo.toml +++ b/src/tests/e2e_extended_mode/Cargo.toml @@ -19,7 +19,7 @@ chrono = { version = "0.4", features = ['serde'] } clap = { version = "4", features = ["derive"] } pg_interval = "0.4" rust_decimal ={ version = "1.32", features = ["db-postgres"] } -tokio = { version = "0.2.23", package = "madsim-tokio", features = ["rt", "macros","rt-multi-thread"] } +tokio = { version = "0.2.24", package = "madsim-tokio", features = ["rt", "macros","rt-multi-thread"] } tokio-postgres = { version = "0.7", features = ["with-chrono-0_4"] } tracing = "0.1" tracing-subscriber = "0.3.17" diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index b81be65edae42..734eccadd4c16 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -48,7 +48,7 @@ serde_json = "1.0.107" sqllogictest = "0.17.0" tempfile = "3" tikv-jemallocator = { workspace = true } -tokio = { version = "0.2.23", package = "madsim-tokio" } +tokio = { version = "0.2.24", package = "madsim-tokio" } tokio-postgres = "0.7" tokio-stream = "0.1" tracing = "0.1"