From ba2dbbb0f6e7733a1f8c2ae98929d4b87f3abb19 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 26 Jan 2023 16:46:15 +0800 Subject: [PATCH 01/22] impl --- .../optimizer/plan_node/expr_rewritable.rs | 39 +++++++++++++++++++ .../src/optimizer/plan_node/generic/agg.rs | 11 +++++- .../src/optimizer/plan_node/generic/filter.rs | 7 ++++ .../src/optimizer/plan_node/generic/join.rs | 7 ++++ .../optimizer/plan_node/generic/project.rs | 8 +++- .../plan_node/generic/project_set.rs | 8 +++- .../src/optimizer/plan_node/generic/scan.rs | 7 ++++ .../src/optimizer/plan_node/logical_agg.rs | 10 ++++- .../src/optimizer/plan_node/logical_apply.rs | 10 ++++- .../src/optimizer/plan_node/logical_delete.rs | 9 ++++- .../src/optimizer/plan_node/logical_expand.rs | 9 ++++- .../src/optimizer/plan_node/logical_filter.rs | 13 ++++++- .../optimizer/plan_node/logical_hop_window.rs | 10 ++++- .../src/optimizer/plan_node/logical_insert.rs | 9 ++++- .../src/optimizer/plan_node/logical_join.rs | 10 ++++- .../src/optimizer/plan_node/logical_limit.rs | 9 ++++- .../optimizer/plan_node/logical_multi_join.rs | 11 +++++- .../src/optimizer/plan_node/logical_now.rs | 8 +++- .../optimizer/plan_node/logical_over_agg.rs | 9 ++++- .../optimizer/plan_node/logical_project.rs | 10 ++++- .../plan_node/logical_project_set.rs | 10 ++++- .../src/optimizer/plan_node/logical_scan.rs | 10 ++++- .../src/optimizer/plan_node/logical_share.rs | 11 +++++- .../src/optimizer/plan_node/logical_source.rs | 9 ++++- .../plan_node/logical_table_function.rs | 12 +++++- .../src/optimizer/plan_node/logical_topn.rs | 9 ++++- .../src/optimizer/plan_node/logical_union.rs | 9 ++++- .../src/optimizer/plan_node/logical_update.rs | 12 +++++- .../src/optimizer/plan_node/logical_values.rs | 14 ++++++- src/frontend/src/optimizer/plan_node/mod.rs | 3 ++ .../plan_node/stream_dynamic_filter.rs | 11 +----- 31 files changed, 281 insertions(+), 43 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_node/expr_rewritable.rs diff --git a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs new file mode 100644 index 0000000000000..775347e5ed969 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs @@ -0,0 +1,39 @@ +use std::ops::Deref; +use super::*; +use paste::paste; +use crate::expr::ExprRewriter; +use crate::{for_batch_plan_nodes, for_stream_plan_nodes}; + +pub trait ExprRewritable { + // Default is to do nothing + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef; +} + +impl ExprRewritable for PlanRef { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let inputs: Vec = self + .inputs() + .iter() + .map(|plan_ref| plan_ref.rewrite_exprs(r)) + .collect(); + let new = self.clone_with_inputs(&inputs[..]); + let dyn_t = new.deref(); + dyn_t.rewrite_exprs(r) + } +} + + +macro_rules! ban_expr_rewritable { + ($( { $convention:ident, $name:ident }),*) => { + paste!{ + $(impl ExprRewritable for [<$convention $name>] { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + unimplemented!() + } + })* + } + } +} +for_batch_plan_nodes! {ban_expr_rewritable} +for_stream_plan_nodes! {ban_expr_rewritable} +// for_logical_plan_nodes! {ban_expr_rewritable} \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 4013cc4dc97c2..93c8ee7f912bf 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -46,6 +46,14 @@ pub struct Agg { pub input: PlanRef, } +impl Agg { + pub(crate) fn rewrite_exprs(&mut self, r: &mut dyn ExprRewriter) { + self.agg_calls.iter_mut().for_each(|call| { + call.filter = call.filter.clone().rewrite_expr(r); + }); + } +} + impl GenericPlanNode for Agg { fn schema(&self) -> Schema { let fields = self @@ -120,6 +128,7 @@ impl AggCallState { }), } } + } pub struct TableState { @@ -657,4 +666,4 @@ impl fmt::Debug for PlanAggCallDisplay<'_> { } Ok(()) } -} +} \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/generic/filter.rs b/src/frontend/src/optimizer/plan_node/generic/filter.rs index fdb8b793310dc..1e5dee8c5b470 100644 --- a/src/frontend/src/optimizer/plan_node/generic/filter.rs +++ b/src/frontend/src/optimizer/plan_node/generic/filter.rs @@ -15,6 +15,7 @@ use risingwave_common::catalog::Schema; use super::{GenericPlanNode, GenericPlanRef}; +use crate::expr::ExprRewriter; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::utils::Condition; @@ -41,3 +42,9 @@ impl GenericPlanNode for Filter { self.input.ctx() } } + +impl Filter { + pub(crate) fn rewrite_exprs(&mut self, r: &mut dyn ExprRewriter) { + self.predicate = self.predicate.clone().rewrite_expr(r); + } +} diff --git a/src/frontend/src/optimizer/plan_node/generic/join.rs b/src/frontend/src/optimizer/plan_node/generic/join.rs index 9e4b394b55bfb..3032a441bee9e 100644 --- a/src/frontend/src/optimizer/plan_node/generic/join.rs +++ b/src/frontend/src/optimizer/plan_node/generic/join.rs @@ -16,6 +16,7 @@ use risingwave_common::catalog::Schema; use risingwave_pb::plan_common::JoinType; use super::{EqJoinPredicate, GenericPlanNode, GenericPlanRef}; +use crate::expr::ExprRewriter; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::utils::{ColIndexMapping, Condition}; @@ -34,6 +35,12 @@ pub struct Join { pub output_indices: Vec, } +impl Join { + pub(crate) fn rewrite_exprs(&mut self, r: &mut dyn ExprRewriter) { + self.on = self.on.clone().rewrite_expr(r); + } +} + impl GenericPlanNode for Join { fn schema(&self) -> Schema { let left_schema = self.left.schema(); diff --git a/src/frontend/src/optimizer/plan_node/generic/project.rs b/src/frontend/src/optimizer/plan_node/generic/project.rs index e16618f43cd62..c77611a17727c 100644 --- a/src/frontend/src/optimizer/plan_node/generic/project.rs +++ b/src/frontend/src/optimizer/plan_node/generic/project.rs @@ -20,7 +20,7 @@ use itertools::Itertools; use risingwave_common::catalog::{Field, Schema}; use super::{GenericPlanNode, GenericPlanRef}; -use crate::expr::{assert_input_ref, Expr, ExprDisplay, ExprImpl, InputRef}; +use crate::expr::{assert_input_ref, Expr, ExprDisplay, ExprImpl, InputRef, ExprRewriter}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::utils::ColIndexMapping; @@ -50,6 +50,12 @@ pub struct Project { _private: (), } +impl Project { + pub(crate) fn rewrite_exprs(&mut self, r: &mut dyn ExprRewriter) { + self.exprs = self.exprs.iter().map(|e| r.rewrite_expr(e.clone())).collect(); + } +} + impl GenericPlanNode for Project { fn schema(&self) -> Schema { let o2i = self.o2i_col_mapping(); diff --git a/src/frontend/src/optimizer/plan_node/generic/project_set.rs b/src/frontend/src/optimizer/plan_node/generic/project_set.rs index 2bd4d69759c00..072e3a4ba2318 100644 --- a/src/frontend/src/optimizer/plan_node/generic/project_set.rs +++ b/src/frontend/src/optimizer/plan_node/generic/project_set.rs @@ -16,7 +16,7 @@ use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::DataType; use super::{GenericPlanNode, GenericPlanRef}; -use crate::expr::{Expr, ExprDisplay, ExprImpl}; +use crate::expr::{Expr, ExprDisplay, ExprImpl, ExprRewriter}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::utils::ColIndexMapping; @@ -34,6 +34,12 @@ pub struct ProjectSet { pub input: PlanRef, } +impl ProjectSet { + pub(crate) fn rewrite_exprs(&mut self, r: &mut dyn ExprRewriter) { + self.select_list = self.select_list.iter().map(|e| r.rewrite_expr(e.clone())).collect(); + } +} + impl GenericPlanNode for ProjectSet { fn schema(&self) -> Schema { let input_schema = self.input.schema(); diff --git a/src/frontend/src/optimizer/plan_node/generic/scan.rs b/src/frontend/src/optimizer/plan_node/generic/scan.rs index 1e0a72bba783b..06d529f9786e9 100644 --- a/src/frontend/src/optimizer/plan_node/generic/scan.rs +++ b/src/frontend/src/optimizer/plan_node/generic/scan.rs @@ -19,6 +19,7 @@ use risingwave_common::catalog::{ColumnDesc, Field, Schema, TableDesc}; use super::GenericPlanNode; use crate::catalog::{ColumnId, IndexCatalog}; +use crate::expr::ExprRewriter; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::utils::Condition; @@ -38,6 +39,12 @@ pub struct Scan { pub predicate: Condition, } +impl Scan { + pub(crate) fn rewrite_exprs(&mut self, r: &mut dyn ExprRewriter) { + self.predicate = self.predicate.clone().rewrite_expr(r); + } +} + impl GenericPlanNode for Scan { fn schema(&self) -> Schema { let fields = self diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index d0d545ae27e53..d7e37402c9f24 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -27,7 +27,7 @@ use super::generic::{ use super::{ BatchHashAgg, BatchSimpleAgg, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamGlobalSimpleAgg, StreamHashAgg, StreamLocalSimpleAgg, StreamProject, - ToBatch, ToStream, + ToBatch, ToStream, ExprRewritable, }; use crate::catalog::table_catalog::TableCatalog; use crate::expr::{ @@ -788,6 +788,14 @@ impl fmt::Display for LogicalAgg { } } +impl ExprRewritable for LogicalAgg { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut new = self.clone(); + new.core.rewrite_exprs(r); + new.into() + } +} + impl ColPrunable for LogicalAgg { fn prune_col(&self, required_cols: &[usize], ctx: &mut ColumnPruningContext) -> PlanRef { let group_key_required_cols = FixedBitSet::from_iter(self.group_key().iter().copied()); diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index 610ac069b8d47..7101b30a07537 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -24,7 +24,7 @@ use super::{ PredicatePushdown, ToBatch, ToStream, }; use crate::expr::{CorrelatedId, Expr, ExprImpl, ExprRewriter, InputRef}; -use crate::optimizer::plan_node::{ +use crate::optimizer::plan_node::{ExprRewritable, ColumnPruningContext, LogicalFilter, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -295,6 +295,14 @@ impl ColPrunable for LogicalApply { } } +impl ExprRewritable for LogicalApply { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut new = self.clone(); + new.on = new.on.rewrite_expr(r); + new.into() + } +} + impl PredicatePushdown for LogicalApply { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/logical_delete.rs b/src/frontend/src/optimizer/plan_node/logical_delete.rs index 698dded990e94..970b8a5e95ccf 100644 --- a/src/frontend/src/optimizer/plan_node/logical_delete.rs +++ b/src/frontend/src/optimizer/plan_node/logical_delete.rs @@ -20,9 +20,10 @@ use risingwave_common::types::DataType; use super::{ gen_filter_and_pushdown, BatchDelete, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, + PredicatePushdown, ToBatch, ToStream, ExprRewritable, }; use crate::catalog::TableId; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -126,6 +127,12 @@ impl ColPrunable for LogicalDelete { } } +impl ExprRewritable for LogicalDelete { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + self.clone().into() + } +} + impl PredicatePushdown for LogicalDelete { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index 4ef6b09c83cea..a73d8505ab758 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -21,8 +21,9 @@ use risingwave_common::error::Result; use super::generic::GenericPlanNode; use super::{ gen_filter_and_pushdown, generic, BatchExpand, ColPrunable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamExpand, ToBatch, ToStream, + PlanTreeNodeUnary, PredicatePushdown, StreamExpand, ToBatch, ToStream, ExprRewritable, }; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -157,6 +158,12 @@ impl ColPrunable for LogicalExpand { } } +impl ExprRewritable for LogicalExpand { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + self.clone().into() + } +} + impl PredicatePushdown for LogicalExpand { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/logical_filter.rs b/src/frontend/src/optimizer/plan_node/logical_filter.rs index 9d6c54bc9e8ae..fb822588f3cee 100644 --- a/src/frontend/src/optimizer/plan_node/logical_filter.rs +++ b/src/frontend/src/optimizer/plan_node/logical_filter.rs @@ -22,9 +22,9 @@ use risingwave_common::error::Result; use super::generic::{self, GenericPlanNode}; use super::{ ColPrunable, CollectInputRef, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, + PredicatePushdown, ToBatch, ToStream, ExprRewritable, }; -use crate::expr::{assert_input_ref, ExprImpl}; +use crate::expr::{assert_input_ref, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::{ BatchFilter, ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, StreamFilter, ToStreamContext, @@ -175,6 +175,15 @@ impl ColPrunable for LogicalFilter { } } + +impl ExprRewritable for LogicalFilter { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut new = self.clone(); + new.core.rewrite_exprs(r); + new.into() + } +} + impl PredicatePushdown for LogicalFilter { fn predicate_pushdown( &self, 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 9d17805d944da..daa139e85bf65 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -23,9 +23,9 @@ use risingwave_common::types::{DataType, IntervalUnit}; use super::generic::GenericPlanNode; use super::{ gen_filter_and_pushdown, generic, BatchHopWindow, ColPrunable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, ToBatch, ToStream, + PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, ToBatch, ToStream, ExprRewritable, }; -use crate::expr::InputRef; +use crate::expr::{InputRef, ExprRewriter}; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -327,6 +327,12 @@ impl ColPrunable for LogicalHopWindow { } } +impl ExprRewritable for LogicalHopWindow { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + self.clone().into() + } +} + impl PredicatePushdown for LogicalHopWindow { /// Keep predicate on time window parameters (`window_start`, `window_end`), /// the rest may be pushed-down. diff --git a/src/frontend/src/optimizer/plan_node/logical_insert.rs b/src/frontend/src/optimizer/plan_node/logical_insert.rs index e8c1e88e2b993..137f28f5fb5b0 100644 --- a/src/frontend/src/optimizer/plan_node/logical_insert.rs +++ b/src/frontend/src/optimizer/plan_node/logical_insert.rs @@ -20,9 +20,10 @@ use risingwave_common::types::DataType; use super::{ gen_filter_and_pushdown, BatchInsert, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, + PredicatePushdown, ToBatch, ToStream, ExprRewritable, }; use crate::catalog::TableId; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -160,6 +161,12 @@ impl ColPrunable for LogicalInsert { } } +impl ExprRewritable for LogicalInsert { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + self.clone().into() + } +} + impl PredicatePushdown for LogicalInsert { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index 713aeff6f17f4..89a60e5641640 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -25,7 +25,7 @@ use risingwave_pb::plan_common::JoinType; use super::generic::GenericPlanNode; use super::{ generic, BatchProject, ColPrunable, CollectInputRef, LogicalProject, PlanBase, PlanRef, - PlanTreeNodeBinary, PredicatePushdown, StreamHashJoin, StreamProject, ToBatch, ToStream, + PlanTreeNodeBinary, PredicatePushdown, StreamHashJoin, StreamProject, ToBatch, ToStream, ExprRewritable, }; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, InputRef}; use crate::optimizer::plan_node::generic::GenericPlanRef; @@ -784,6 +784,14 @@ impl ColPrunable for LogicalJoin { } } +impl ExprRewritable for LogicalJoin { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut new = self.clone(); + new.core.rewrite_exprs(r); + new.into() + } +} + fn is_pure_fn_except_for_input_ref(expr: &ExprImpl) -> bool { match expr { ExprImpl::Literal(_) => true, diff --git a/src/frontend/src/optimizer/plan_node/logical_limit.rs b/src/frontend/src/optimizer/plan_node/logical_limit.rs index c1814f65f1105..8ce88f88abbdb 100644 --- a/src/frontend/src/optimizer/plan_node/logical_limit.rs +++ b/src/frontend/src/optimizer/plan_node/logical_limit.rs @@ -18,8 +18,9 @@ use risingwave_common::error::{ErrorCode, Result, RwError}; use super::{ gen_filter_and_pushdown, BatchLimit, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, + PredicatePushdown, ToBatch, ToStream, ExprRewritable, }; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -99,6 +100,12 @@ impl ColPrunable for LogicalLimit { } } +impl ExprRewritable for LogicalLimit { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + self.clone().into() + } +} + impl PredicatePushdown for LogicalLimit { fn predicate_pushdown( &self, 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 d778ad98d979f..4f90f84d08c84 100644 --- a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs @@ -21,7 +21,7 @@ use risingwave_pb::plan_common::JoinType; use super::{ ColPrunable, LogicalFilter, LogicalJoin, LogicalProject, PlanBase, PlanNodeType, PlanRef, - PlanTreeNodeBinary, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + PlanTreeNodeBinary, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, ExprRewritable, }; use crate::expr::{ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::{ @@ -522,6 +522,15 @@ impl ColPrunable for LogicalMultiJoin { } } +impl ExprRewritable for LogicalMultiJoin { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + panic!( + "Method not available for `LogicalMultiJoin` which is a placeholder node with \ + a temporary lifetime. It only facilitates join reordering during logical planning." + ) + } +} + impl PredicatePushdown for LogicalMultiJoin { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/logical_now.rs b/src/frontend/src/optimizer/plan_node/logical_now.rs index cd90fe9f62d1b..7db3433dc18d0 100644 --- a/src/frontend/src/optimizer/plan_node/logical_now.rs +++ b/src/frontend/src/optimizer/plan_node/logical_now.rs @@ -23,8 +23,9 @@ use risingwave_common::types::DataType; use super::utils::IndicesDisplay; use super::{ ColPrunable, ColumnPruningContext, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, - RewriteStreamContext, StreamNow, ToBatch, ToStream, ToStreamContext, + RewriteStreamContext, StreamNow, ToBatch, ToStream, ToStreamContext, ExprRewritable, }; +use crate::expr::ExprRewriter; use crate::optimizer::property::FunctionalDependencySet; use crate::utils::ColIndexMapping; use crate::OptimizerContextRef; @@ -69,6 +70,11 @@ impl fmt::Display for LogicalNow { impl_plan_tree_node_for_leaf! { LogicalNow } +impl ExprRewritable for LogicalNow { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + self.clone().into() + }} + impl PredicatePushdown for LogicalNow { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/logical_over_agg.rs b/src/frontend/src/optimizer/plan_node/logical_over_agg.rs index 3af433c6f0dce..d65e51c7bd7fc 100644 --- a/src/frontend/src/optimizer/plan_node/logical_over_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_over_agg.rs @@ -23,9 +23,9 @@ use risingwave_common::types::DataType; use super::generic::{PlanAggOrderByField, PlanAggOrderByFieldDisplay}; use super::{ gen_filter_and_pushdown, ColPrunable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, + PredicatePushdown, ToBatch, ToStream, ExprRewritable, }; -use crate::expr::{Expr, ExprImpl, InputRef, InputRefDisplay, WindowFunction, WindowFunctionType}; +use crate::expr::{Expr, ExprImpl, InputRef, InputRefDisplay, WindowFunction, WindowFunctionType, ExprRewriter}; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -264,6 +264,11 @@ impl ColPrunable for LogicalOverAgg { } } +impl ExprRewritable for LogicalOverAgg { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + self.clone().into() + }} + impl PredicatePushdown for LogicalOverAgg { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index a6183994cd6db..ac12f866f8f8c 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -21,7 +21,7 @@ use risingwave_common::error::Result; use super::generic::{self, GenericPlanNode, Project}; use super::{ gen_filter_and_pushdown, BatchProject, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, StreamProject, ToBatch, ToStream, + PredicatePushdown, StreamProject, ToBatch, ToStream, ExprRewritable, }; use crate::expr::{ExprImpl, ExprRewriter, ExprVisitor, InputRef}; use crate::optimizer::plan_node::{ @@ -204,6 +204,14 @@ impl ColPrunable for LogicalProject { } } +impl ExprRewritable for LogicalProject { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut new = self.clone(); + new.core.rewrite_exprs(r); + new.into() + } +} + impl PredicatePushdown for LogicalProject { fn predicate_pushdown( &self, 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 bc56312a6b7a5..098f9b258dc15 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project_set.rs @@ -18,7 +18,7 @@ use risingwave_common::error::Result; use super::{ generic, BatchProjectSet, ColPrunable, LogicalFilter, LogicalProject, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, ToBatch, ToStream, + PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, ToBatch, ToStream, ExprRewritable, }; use crate::expr::{Expr, ExprImpl, ExprRewriter, FunctionCall, InputRef, TableFunction}; use crate::optimizer::plan_node::generic::GenericPlanNode; @@ -257,6 +257,14 @@ impl ColPrunable for LogicalProjectSet { } } +impl ExprRewritable for LogicalProjectSet { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut new = self.clone(); + new.core.rewrite_exprs(r); + new.into() + } +} + impl PredicatePushdown for LogicalProjectSet { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 29214c91c8195..886fffcea9ed2 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::OrderType; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::{ generic, BatchFilter, BatchProject, ColPrunable, PlanBase, PlanRef, PredicatePushdown, - StreamTableScan, ToBatch, ToStream, + StreamTableScan, ToBatch, ToStream, ExprRewritable, }; use crate::catalog::{ColumnId, IndexCatalog}; use crate::expr::{ @@ -450,6 +450,14 @@ impl ColPrunable for LogicalScan { } } +impl ExprRewritable for LogicalScan { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut new = self.clone(); + new.core.rewrite_exprs(r); + new.into() + } +} + impl PredicatePushdown for LogicalScan { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs index dfea313066e26..b3f59a39aa619 100644 --- a/src/frontend/src/optimizer/plan_node/logical_share.rs +++ b/src/frontend/src/optimizer/plan_node/logical_share.rs @@ -20,8 +20,9 @@ use risingwave_common::error::Result; use super::generic::{self, GenericPlanNode}; use super::{ - ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, ExprRewritable, }; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, @@ -118,6 +119,14 @@ impl ColPrunable for LogicalShare { } } +impl ExprRewritable for LogicalShare { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + unimplemented!( + "call rewrite_exprs of the PlanRef instead of calling directly on LogicalShare" + ) + } +} + impl PredicatePushdown for LogicalShare { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 680e1bba82f33..caa44cba6c97f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -25,11 +25,11 @@ use risingwave_connector::source::DataType; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::{ generic, BatchSource, ColPrunable, LogicalFilter, LogicalProject, PlanBase, PlanRef, - PredicatePushdown, StreamRowIdGen, StreamSource, ToBatch, ToStream, + PredicatePushdown, StreamRowIdGen, StreamSource, ToBatch, ToStream, ExprRewritable, }; use crate::catalog::source_catalog::SourceCatalog; use crate::catalog::ColumnId; -use crate::expr::{Expr, ExprImpl, ExprType}; +use crate::expr::{Expr, ExprImpl, ExprType, ExprRewriter}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, @@ -161,6 +161,11 @@ impl ColPrunable for LogicalSource { } } +impl ExprRewritable for LogicalSource { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + self.clone().into() + }} + /// A util function to extract kafka offset timestamp range. /// /// Currently we only support limiting kafka offset timestamp range using literals, e.g. we only 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 f5f900272c9a8..c7d19947e558e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/logical_table_function.rs @@ -17,8 +17,8 @@ use std::fmt; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::{ErrorCode, Result}; -use super::{ColPrunable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream}; -use crate::expr::{Expr, TableFunction}; +use super::{ColPrunable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, ExprRewritable}; +use crate::expr::{Expr, TableFunction, ExprRewriter}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::{ BatchTableFunction, ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, @@ -68,6 +68,14 @@ impl ColPrunable for LogicalTableFunction { } } +impl ExprRewritable for LogicalTableFunction { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut new = self.clone(); + new.table_function.args = new.table_function.args.into_iter().map(|e| r.rewrite_expr(e)).collect(); + new.into() + } +} + impl PredicatePushdown for LogicalTableFunction { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/logical_topn.rs b/src/frontend/src/optimizer/plan_node/logical_topn.rs index 38dba4e90f641..ca8e82e53f188 100644 --- a/src/frontend/src/optimizer/plan_node/logical_topn.rs +++ b/src/frontend/src/optimizer/plan_node/logical_topn.rs @@ -21,9 +21,9 @@ use risingwave_common::error::{ErrorCode, Result, RwError}; use super::generic::GenericPlanNode; use super::{ gen_filter_and_pushdown, generic, BatchGroupTopN, ColPrunable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamGroupTopN, StreamProject, ToBatch, ToStream, + PlanTreeNodeUnary, PredicatePushdown, StreamGroupTopN, StreamProject, ToBatch, ToStream, ExprRewritable, }; -use crate::expr::{ExprType, FunctionCall, InputRef}; +use crate::expr::{ExprType, FunctionCall, InputRef, ExprRewriter}; use crate::optimizer::plan_node::{ BatchTopN, ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, StreamTopN, ToStreamContext, @@ -353,6 +353,11 @@ impl ColPrunable for LogicalTopN { } } +impl ExprRewritable for LogicalTopN { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + self.clone().into() + }} + impl PredicatePushdown for LogicalTopN { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/logical_union.rs b/src/frontend/src/optimizer/plan_node/logical_union.rs index 2d8d357a8cb41..b83ea989b2756 100644 --- a/src/frontend/src/optimizer/plan_node/logical_union.rs +++ b/src/frontend/src/optimizer/plan_node/logical_union.rs @@ -18,8 +18,8 @@ use itertools::Itertools; use risingwave_common::error::Result; use risingwave_common::types::{DataType, Scalar}; -use super::{ColPrunable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream}; -use crate::expr::{ExprImpl, InputRef, Literal}; +use super::{ColPrunable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, ExprRewritable}; +use crate::expr::{ExprImpl, InputRef, Literal, ExprRewriter}; use crate::optimizer::plan_node::generic::{GenericPlanNode, GenericPlanRef}; use crate::optimizer::plan_node::stream_union::StreamUnion; use crate::optimizer::plan_node::{ @@ -109,6 +109,11 @@ impl ColPrunable for LogicalUnion { } } +impl ExprRewritable for LogicalUnion { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + self.clone().into() + }} + impl PredicatePushdown for LogicalUnion { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/logical_update.rs b/src/frontend/src/optimizer/plan_node/logical_update.rs index ad3c48c2ebfea..5e8dd0fb46570 100644 --- a/src/frontend/src/optimizer/plan_node/logical_update.rs +++ b/src/frontend/src/optimizer/plan_node/logical_update.rs @@ -20,10 +20,10 @@ use risingwave_common::types::DataType; use super::{ gen_filter_and_pushdown, BatchUpdate, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, + PredicatePushdown, ToBatch, ToStream, ExprRewritable, }; use crate::catalog::TableId; -use crate::expr::ExprImpl; +use crate::expr::{ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -135,6 +135,14 @@ impl fmt::Display for LogicalUpdate { } } +impl ExprRewritable for LogicalUpdate { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut new = self.clone(); + new.exprs = new.exprs.into_iter().map(|e| r.rewrite_expr(e)).collect(); + new.into() + } +} + impl ColPrunable for LogicalUpdate { fn prune_col(&self, _required_cols: &[usize], ctx: &mut ColumnPruningContext) -> PlanRef { let required_cols: Vec<_> = (0..self.input.schema().len()).collect(); diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index a2ef0269632f1..1613aedf6aa71 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -20,9 +20,9 @@ use risingwave_common::error::{ErrorCode, Result, RwError}; use super::{ BatchValues, ColPrunable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, ToBatch, - ToStream, + ToStream, ExprRewritable, }; -use crate::expr::{Expr, ExprImpl}; +use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, @@ -76,6 +76,14 @@ impl fmt::Display for LogicalValues { } } +impl ExprRewritable for LogicalValues { + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut new = self.clone(); + new.rows = new.rows.iter().map(|exprs| exprs.iter().map(|e| r.rewrite_expr(e.clone())).collect::>()).collect::>().into(); + new.into() + } +} + impl ColPrunable for LogicalValues { fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef { let rows = self @@ -91,6 +99,8 @@ impl ColPrunable for LogicalValues { } } + + impl PredicatePushdown for LogicalValues { fn predicate_pushdown( &self, diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index ccded2343f7b1..9943ad6621aa9 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -60,6 +60,7 @@ pub trait PlanNode: + Display + Downcast + ColPrunable + + ExprRewritable + ToBatch + ToStream + ToDistributedBatch @@ -407,6 +408,8 @@ mod plan_tree_node; pub use plan_tree_node::*; mod col_pruning; pub use col_pruning::*; +mod expr_rewritable; +pub use expr_rewritable::*; mod convert; pub use convert::*; mod eq_join_predicate; 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 09644694e22d0..dd50f331fb042 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -77,10 +77,6 @@ impl StreamDynamicFilter { pub fn left_index(&self) -> usize { self.core.left_index } - - pub fn comparator(&self) -> &ExprType { - &self.core.comparator - } } impl fmt::Display for StreamDynamicFilter { @@ -136,12 +132,9 @@ impl_plan_tree_node_for_binary! { StreamDynamicFilter } impl StreamNode for StreamDynamicFilter { fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> NodeBody { use generic::dynamic_filter::*; - let condition = self.core.predicate().as_expr_unless_true().map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x) + let condition = self.core.predicate().as_expr_unless_true().map(|x| x .to_expr_proto() - }); + ); let left_index = self.core.left_index; let left_table = infer_left_internal_table_catalog(&self.base, left_index) .with_id(state.gen_table_id_wrapped()); From 70301936ce0c0a63db12a20b3e943921458cce81 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 26 Jan 2023 18:05:16 +0800 Subject: [PATCH 02/22] commit --- e2e_test/batch/functions/now.slt.part | 7 ++ .../planner_test/tests/testdata/array.yaml | 14 ++-- .../tests/testdata/basic_query.yaml | 10 +-- .../planner_test/tests/testdata/cast.yaml | 4 +- .../planner_test/tests/testdata/explain.yaml | 11 +++ .../planner_test/tests/testdata/expr.yaml | 78 +++++++++++-------- .../tests/testdata/index_selection.yaml | 6 +- .../planner_test/tests/testdata/insert.yaml | 2 +- .../planner_test/tests/testdata/order_by.yaml | 10 +-- .../tests/testdata/predicate_pushdown.yaml | 10 +-- .../tests/testdata/project_set.yaml | 12 +-- .../tests/testdata/range_scan.yaml | 4 +- .../tests/testdata/temporal_filter.yaml | 2 +- src/frontend/src/optimizer/mod.rs | 20 +++++ .../plan_expr_rewriter/const_eval_rewriter.rs | 40 ++++++++++ .../src/optimizer/plan_expr_rewriter/mod.rs | 3 + .../optimizer/plan_node/expr_rewritable.rs | 9 +-- .../src/optimizer/plan_node/generic/agg.rs | 3 +- .../optimizer/plan_node/generic/project.rs | 8 +- .../plan_node/generic/project_set.rs | 6 +- .../src/optimizer/plan_node/logical_agg.rs | 6 +- .../src/optimizer/plan_node/logical_apply.rs | 6 +- .../src/optimizer/plan_node/logical_delete.rs | 6 +- .../src/optimizer/plan_node/logical_expand.rs | 6 +- .../src/optimizer/plan_node/logical_filter.rs | 5 +- .../optimizer/plan_node/logical_hop_window.rs | 8 +- .../src/optimizer/plan_node/logical_insert.rs | 6 +- .../src/optimizer/plan_node/logical_join.rs | 5 +- .../src/optimizer/plan_node/logical_limit.rs | 6 +- .../optimizer/plan_node/logical_multi_join.rs | 7 +- .../src/optimizer/plan_node/logical_now.rs | 9 ++- .../optimizer/plan_node/logical_over_agg.rs | 13 ++-- .../optimizer/plan_node/logical_project.rs | 4 +- .../plan_node/logical_project_set.rs | 4 +- .../src/optimizer/plan_node/logical_scan.rs | 4 +- .../src/optimizer/plan_node/logical_share.rs | 5 +- .../src/optimizer/plan_node/logical_source.rs | 11 +-- .../plan_node/logical_table_function.rs | 14 +++- .../src/optimizer/plan_node/logical_topn.rs | 12 +-- .../src/optimizer/plan_node/logical_union.rs | 9 ++- .../src/optimizer/plan_node/logical_update.rs | 4 +- .../src/optimizer/plan_node/logical_values.rs | 18 +++-- .../plan_node/stream_dynamic_filter.rs | 8 +- 43 files changed, 280 insertions(+), 155 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs create mode 100644 src/frontend/src/optimizer/plan_expr_rewriter/mod.rs diff --git a/e2e_test/batch/functions/now.slt.part b/e2e_test/batch/functions/now.slt.part index ee706e80e2eb9..5adb66c83db2a 100644 --- a/e2e_test/batch/functions/now.slt.part +++ b/e2e_test/batch/functions/now.slt.part @@ -10,6 +10,13 @@ create table t (a timestamp); statement ok insert into t values(now()); +# constant eval of now in batch plan +query T +explain select now() + interval '1 hour' = now() + interval '30 minutes' + interval '30 minutes' true; +---- +BatchProject { exprs: [true:Boolean] } + └─BatchValues { rows: [[]] } + statement ok drop table tz diff --git a/src/frontend/planner_test/tests/testdata/array.yaml b/src/frontend/planner_test/tests/testdata/array.yaml index b436cbdff2d4e..fd26d4a94ac4a 100644 --- a/src/frontend/planner_test/tests/testdata/array.yaml +++ b/src/frontend/planner_test/tests/testdata/array.yaml @@ -4,13 +4,13 @@ logical_plan: | LogicalValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Varchar }] } } batch_plan: | - BatchValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]] } + BatchValues { rows: [[ARRAY[foo, bar]:List { datatype: Varchar }]] } - sql: | values (ARRAY[1, 2+3, 4*5+1]); logical_plan: | LogicalValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Int32 }] } } batch_plan: | - BatchValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]] } + BatchValues { rows: [[ARRAY[1, 5, 21]:List { datatype: Int32 }]] } - sql: | create table t (v1 int); select (ARRAY[1, v1]) from t; @@ -52,7 +52,7 @@ LogicalProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32))] } + BatchProject { exprs: [ARRAY[66, 123]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[array[66]], array[233]); @@ -60,7 +60,7 @@ LogicalProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32))] } + BatchProject { exprs: [ARRAY[{66}, {233}]:List { datatype: List { datatype: Int32 } }] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[233], array[array[66]]); @@ -68,7 +68,7 @@ LogicalProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32)))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32)))] } + BatchProject { exprs: [ARRAY[{233}, {66}]:List { datatype: List { datatype: Int32 } }] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[233], array[array[array[66]]]); @@ -85,7 +85,7 @@ LogicalProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32)] } + BatchProject { exprs: [ARRAY[66, 123]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - sql: | select array_append(123, 234); @@ -102,7 +102,7 @@ LogicalProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32))] } + BatchProject { exprs: [ARRAY[123, 66]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - sql: | select array_prepend(123, 234); diff --git a/src/frontend/planner_test/tests/testdata/basic_query.yaml b/src/frontend/planner_test/tests/testdata/basic_query.yaml index 5b141abcc0d1f..51ac43d99d7fd 100644 --- a/src/frontend/planner_test/tests/testdata/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/basic_query.yaml @@ -1,7 +1,7 @@ # This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - sql: values (11, 22), (33+(1+2), 44); batch_plan: | - BatchValues { rows: [[11:Int32, 22:Int32], [(33:Int32 + (1:Int32 + 2:Int32)), 44:Int32]] } + BatchValues { rows: [[11:Int32, 22:Int32], [36:Int32, 44:Int32]] } - sql: select * from t binder_error: 'Catalog error: table or source not found: t' - sql: | @@ -22,11 +22,11 @@ select * from t where 1>2 and 1=1 and 3<1 and 4<>1 or 1=1 and 2>=1 and 1<=2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } + └─BatchFilter { predicate: true:Boolean AND true:Boolean } └─BatchScan { table: t, columns: [], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } + └─StreamFilter { predicate: true:Boolean AND true:Boolean } └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int); @@ -130,11 +130,11 @@ - sql: | select * from unnest(Array[1,2,3]); batch_plan: | - BatchTableFunction { Unnest(Array(1:Int32, 2:Int32, 3:Int32)) } + BatchTableFunction { Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 }) } - sql: | select * from unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchTableFunction { Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32))) } + BatchTableFunction { Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } }) } - sql: | create table t1 (x int); select * from t1 where EXISTS(select * where t1.x=1); diff --git a/src/frontend/planner_test/tests/testdata/cast.yaml b/src/frontend/planner_test/tests/testdata/cast.yaml index e122dd0ace778..b9385ce48355e 100644 --- a/src/frontend/planner_test/tests/testdata/cast.yaml +++ b/src/frontend/planner_test/tests/testdata/cast.yaml @@ -18,7 +18,7 @@ sql: | select case when NULL then 1 end; batch_plan: | - BatchProject { exprs: [Case(null:Boolean, 1:Int32)] } + BatchProject { exprs: [null:Int32] } └─BatchValues { rows: [[]] } - name: implicit cast boolean (JOIN ON NULL) sql: | @@ -65,7 +65,7 @@ sql: | select case when 'y' then 1 end; batch_plan: | - BatchProject { exprs: [Case(true:Boolean, 1:Int32)] } + BatchProject { exprs: [1:Int32] } └─BatchValues { rows: [[]] } - name: implicit cast boolean (JOIN ON with literal 'y' of unknown type) sql: | diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index c89502dce3ade..ea082c6c69bd8 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -36,6 +36,11 @@ LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + Const eval exprs: + + LogicalProject { exprs: [1:Int32] } + └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + To Batch Physical Plan: BatchProject { exprs: [1:Int32] } @@ -165,6 +170,12 @@ ├─LogicalScan { table: t1, columns: [v1] } └─LogicalScan { table: t2, columns: [v2] } + Const eval exprs: + + LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } + ├─LogicalScan { table: t1, columns: [v1] } + └─LogicalScan { table: t2, columns: [v2] } + - sql: | explain (logical) create table t1(v1 int); explain_output: | diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 0420e84d1e2ca..286ae7e5c65b0 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -19,17 +19,17 @@ - sql: | values(cast(1 as bigint)); batch_plan: | - BatchValues { rows: [[1:Int32::Int64]] } + BatchValues { rows: [[1:Int64]] } - sql: | values(not true); batch_plan: | - BatchValues { rows: [[Not(true:Boolean)]] } + BatchValues { rows: [[false:Boolean]] } - sql: | create table t (); select (((((false is not true) is true) is not false) is false) is not null) is null from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [IsNull(IsNotNull(IsFalse(IsNotFalse(IsTrue(IsNotTrue(false:Boolean))))))] } + └─BatchProject { exprs: [false:Boolean] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: bind between sql: | @@ -65,13 +65,13 @@ sql: | SELECT 1::real in (3, 1.0, 2); batch_plan: | - BatchProject { exprs: [In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32)] } + BatchProject { exprs: [true:Boolean] } └─BatchValues { rows: [[]] } - name: not in-list with aligned types sql: | SELECT 1::real not in (3, 1.0, 2); batch_plan: | - BatchProject { exprs: [Not(In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32))] } + BatchProject { exprs: [false:Boolean] } └─BatchValues { rows: [[]] } - name: in-list with misaligned types sql: | @@ -82,7 +82,7 @@ create table t (v1 int); SELECT 1 in (3, 0.5*2, min(v1)) from t; batch_plan: | - BatchProject { exprs: [(In(1:Int32::Decimal, 3:Int32::Decimal, (0.5:Decimal * 2:Int32)) OR (1:Int32 = min(min(t.v1))))] } + BatchProject { exprs: [(true:Boolean OR (1:Int32 = min(min(t.v1))))] } └─BatchSimpleAgg { aggs: [min(min(t.v1))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [min(t.v1)] } @@ -108,7 +108,7 @@ create table b (b1 int, b2 int); SELECT b2 from b where exists (select 2 from t where v1 in (3, 1.0, b1)); batch_plan: | - BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } + BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b1, b.b2], distribution: SomeShard } └─BatchExchange { order: [], dist: Single } @@ -122,19 +122,19 @@ - sql: | values(round(42.4382, 2)); batch_plan: | - BatchValues { rows: [[RoundDigit(42.4382:Decimal, 2:Int32)]] } + BatchValues { rows: [[42.44:Decimal]] } - sql: | values(round(42.4382)); batch_plan: | - BatchValues { rows: [[Round(42.4382:Decimal)]] } + BatchValues { rows: [[42:Decimal]] } - sql: | values(round(42)); batch_plan: | - BatchValues { rows: [[Round(42:Int32::Float64)]] } + BatchValues { rows: [[42:Float64]] } - sql: | values(round(42, 2)); batch_plan: | - BatchValues { rows: [[RoundDigit(42:Int32::Decimal, 2:Int32)]] } + BatchValues { rows: [[42:Decimal]] } - sql: | values(round(true)); binder_error: |- @@ -144,20 +144,19 @@ -- Single quoted literal can be treated as number without error. values(round('123')); batch_plan: | - BatchValues { rows: [[Round(123:Float64)]] } + BatchValues { rows: [[123:Float64]] } - sql: | -- When it is invalid, PostgreSQL reports error during explain, but we have to wait until execution as of now. #4235 values(round('abc')); - batch_plan: | - BatchValues { rows: [[Round('abc':Varchar::Float64)]] } + batch_error: 'Expr error: Parse error: risingwave_common::types::ordered_float::OrderedFloat' - sql: | values(extract(hour from timestamp '2001-02-16 20:38:40')); batch_plan: | - BatchValues { rows: [[Extract('HOUR':Varchar, '2001-02-16 20:38:40':Timestamp)]] } + BatchValues { rows: [[20:Decimal]] } - sql: | values('Postgres' not like 'Post%'); batch_plan: | - BatchValues { rows: [[Not(Like('Postgres':Varchar, 'Post%':Varchar))]] } + BatchValues { rows: [[false:Boolean]] } - sql: | values(1 not like 1.23); binder_error: |- @@ -166,12 +165,12 @@ - sql: | select length(trim(trailing '1' from '12'))+length(trim(leading '2' from '23'))+length(trim(both '3' from '34')); batch_plan: | - BatchProject { exprs: [((Length(Rtrim('12':Varchar, '1':Varchar)) + Length(Ltrim('23':Varchar, '2':Varchar))) + Length(Trim('34':Varchar, '3':Varchar)))] } + BatchProject { exprs: [4:Int32] } └─BatchValues { rows: [[]] } - sql: | select position(replace('1','1','2'),'123') where '12' like '%1'; batch_plan: | - BatchProject { exprs: [Position(Replace('1':Varchar, '1':Varchar, '2':Varchar), '123':Varchar)] } + BatchProject { exprs: [0:Int32] } └─BatchFilter { predicate: Like('12':Varchar, '%1':Varchar) } └─BatchValues { rows: [[]] } - name: case searched form with else @@ -180,11 +179,11 @@ select (case when v1=1 then 1 when v1=2 then 2 else 0.0 end) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal), t._row_id] } + └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal), t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: case searched form without else sql: | @@ -192,7 +191,7 @@ select (case when v1=1 then 1 when v1=2 then 2.1 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2.1:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2.1:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case simple form sql: | @@ -200,7 +199,7 @@ select (case v1 when 1 then 1 when 2.0 then 2 else 0.0 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2.0:Decimal), 2:Int32::Decimal, 0.0:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2.0:Decimal), 2:Decimal, 0.0:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case misaligned result types sql: | @@ -266,18 +265,18 @@ select concat_ws(v1, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar)] } + └─BatchProject { exprs: [ConcatWs(t.v1, '1':Varchar)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar), t._row_id] } + └─StreamProject { exprs: [ConcatWs(t.v1, '1':Varchar), t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 varchar); select concat_ws(v1, 1.2) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, 1.2:Decimal::Varchar)] } + └─BatchProject { exprs: [ConcatWs(t.v1, '1.2':Varchar)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - sql: | create table t (v1 int); @@ -292,11 +291,11 @@ select concat(v1, v2, v3, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar)] } + └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar)] } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar), t._row_id] } + └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar), t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 float); @@ -312,22 +311,22 @@ - sql: | select concat(':', true); batch_plan: | - BatchProject { exprs: [ConcatWs('':Varchar, ':':Varchar, BoolOut(true:Boolean))] } + BatchProject { exprs: [':t':Varchar] } └─BatchValues { rows: [[]] } - sql: | select ':' || true; batch_plan: | - BatchProject { exprs: [ConcatOp(':':Varchar, true:Boolean::Varchar)] } + BatchProject { exprs: [':true':Varchar] } └─BatchValues { rows: [[]] } - sql: | select substr('hello', NULL); batch_plan: | - BatchProject { exprs: [Substr('hello':Varchar, null:Int32)] } + BatchProject { exprs: [null:Varchar] } └─BatchValues { rows: [[]] } - sql: | select substr(NULL, 1); batch_plan: | - BatchProject { exprs: [Substr(null:Varchar, 1:Int32)] } + BatchProject { exprs: [null:Varchar] } └─BatchValues { rows: [[]] } - sql: | select pg_typeof('123'); @@ -421,7 +420,7 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [Some((1:Int32 < ArrayCat(Array(1:Int32), Array(2:Int32))))] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat(Array(1:Int32), ARRAY[2]:List { datatype: Int32 })))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchProject { exprs: [Array(1:Int32)] } @@ -445,7 +444,7 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [All((1:Int32 < ArrayCat(Array(1:Int32), Array(2:Int32))))] } + BatchProject { exprs: [All((1:Int32 < ArrayCat(Array(1:Int32), ARRAY[2]:List { datatype: Int32 })))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchProject { exprs: [Array(1:Int32)] } @@ -496,7 +495,7 @@ sql: | create table t (v1 timestamp with time zone, v2 timestamp with time zone); select * from t where v1 >= now() or v2 >= now(); - stream_error: 'internal error: Conditions containing now must be of the form `input_expr cmp now() [+- const_expr]` or `now() [+- const_expr] cmp input_expr`, where `input_expr` references a column and contains no `now()`.' + stream_error: 'Expr error: Expected epoch timestamp bound into Now' - name: now inside HAVING clause sql: | create table t (v1 timestamp with time zone, v2 int); @@ -526,3 +525,14 @@ create table t (v1 timestamp with time zone, v2 int); select sum(v2) filter (where v1 >= now()) as sum_v2 from t; stream_error: 'Invalid input syntax: For creation of materialized views, `NOW()` function is only allowed in `WHERE` and `HAVING`. Found in clause: Some(Filter)' +- name: const_eval of const expr + sql: | + create table t(v1 int); + select 1 + 2 + v1 from t; + optimized_logical_plan: | + LogicalProject { exprs: [(3:Int32 + t.v1)] } + └─LogicalScan { table: t, columns: [t.v1] } +- name: const_eval of division by 0 error + sql: | + select 1 / 0 t1; + stream_error: 'Expr error: Division by zero' diff --git a/src/frontend/planner_test/tests/testdata/index_selection.yaml b/src/frontend/planner_test/tests/testdata/index_selection.yaml index af18731a327f8..c98da2bf8e0bc 100644 --- a/src/frontend/planner_test/tests/testdata/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/index_selection.yaml @@ -77,7 +77,7 @@ select a,b from t1 where a in (1,2) and b in (2,3) batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: In(idx1.b, 2:Int32::Decimal, 3:Int32::Decimal) } + └─BatchFilter { predicate: In(idx1.b, 2:Decimal, 3:Decimal) } └─BatchScan { table: idx1, columns: [idx1.a, idx1.b], scan_ranges: [idx1.a = Int32(1) , idx1.a = Int32(2)], distribution: UpstreamHashShard(idx1.a, idx1.b) } - sql: | create table t1 (a int, b numeric, c bigint); @@ -213,13 +213,13 @@ update t1 set c = 3 where a = 1 and b = 2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int32::Int64, $3] } + └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } └─BatchExchange { order: [], dist: Single } └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } batch_local_plan: | - BatchUpdate { table: t1, exprs: [$0, $1, 3:Int32::Int64, $3] } + BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/insert.yaml b/src/frontend/planner_test/tests/testdata/insert.yaml index aa67c803d90a2..40a528fcfae51 100644 --- a/src/frontend/planner_test/tests/testdata/insert.yaml +++ b/src/frontend/planner_test/tests/testdata/insert.yaml @@ -93,7 +93,7 @@ sql: | values (1), (null), (2.3); batch_plan: | - BatchValues { rows: [[1:Int32::Decimal], [null:Decimal], [2.3:Decimal]] } + BatchValues { rows: [[1:Decimal], [null:Decimal], [2.3:Decimal]] } - name: rows of different number of columns sql: | values (1), (2, 3); diff --git a/src/frontend/planner_test/tests/testdata/order_by.yaml b/src/frontend/planner_test/tests/testdata/order_by.yaml index 4f460213c6791..5a74f834a74f5 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -59,13 +59,13 @@ select * from t order by 1+1; batch_plan: | BatchProject { exprs: [t.v1, t.v2] } - └─BatchExchange { order: [(1:Int32 + 1:Int32) ASC], dist: Single } - └─BatchSort { order: [(1:Int32 + 1:Int32) ASC] } - └─BatchProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32)] } + └─BatchExchange { order: [2:Int32 ASC], dist: Single } + └─BatchSort { order: [2:Int32 ASC] } + └─BatchProject { exprs: [t.v1, t.v2, 2:Int32] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, (1:Int32 + 1:Int32)(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [(1:Int32 + 1:Int32), t._row_id] } - └─StreamProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32), t._row_id] } + StreamMaterialize { columns: [v1, v2, 2:Int32(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [2:Int32, t._row_id] } + └─StreamProject { exprs: [t.v1, t.v2, 2:Int32, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 bigint, v2 double precision); diff --git a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml index 52ac449e4b5ae..f8b2fda0b1111 100644 --- a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml @@ -277,28 +277,28 @@ create table t2(v2 timestamp with time zone, v3 interval); select * from t1, t2 where v1 = v2 and v1 > now() + v3; optimized_logical_plan: | - LogicalFilter { predicate: (t1.v1 > (Now + t2.v3)) } + LogicalFilter { predicate: (t1.v1 > ('2021-04-01 00:00:00+00:00':Timestamptz + t2.v3)) } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2), output: all } ├─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalScan { table: t2, columns: [t2.v2, t2.v3] } - stream_error: 'internal error: Conditions containing now must be of the form `input_expr cmp now() [+- const_expr]` or `now() [+- const_expr] cmp input_expr`, where `input_expr` references a column and contains no `now()`.' + stream_error: 'Expr error: Expected epoch timestamp bound into Now' - name: now() in complex cmp expr pushed onto join ON clause results in dynamic filter sql: | create table t1(v1 timestamp with time zone); create table t2(v2 timestamp with time zone, v3 interval); select * from t1 join t2 where v1 = v2 and v1 > now() + v3; optimized_logical_plan: | - LogicalFilter { predicate: (t1.v1 > (Now + t2.v3)) } + LogicalFilter { predicate: (t1.v1 > ('2021-04-01 00:00:00+00:00':Timestamptz + t2.v3)) } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2), output: all } ├─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalScan { table: t2, columns: [t2.v2, t2.v3] } - stream_error: 'internal error: Conditions containing now must be of the form `input_expr cmp now() [+- const_expr]` or `now() [+- const_expr] cmp input_expr`, where `input_expr` references a column and contains no `now()`.' + stream_error: 'Expr error: Expected epoch timestamp bound into Now' - name: now() does not get pushed to scan, but others do sql: | create table t1(v1 timestamp with time zone, v2 int); select * from t1 where v1 > now() + '30 min' and v2 > 5; optimized_logical_plan: | - LogicalFilter { predicate: (t1.v1 > (Now + '00:30:00':Interval)) } + LogicalFilter { predicate: (t1.v1 > '2021-04-01 00:30:00+00:00':Timestamptz) } └─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: (t1.v2 > 5:Int32) } stream_plan: | StreamMaterialize { columns: [v1, v2, t1._row_id(hidden)], pk_columns: [t1._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/project_set.yaml b/src/frontend/planner_test/tests/testdata/project_set.yaml index 4adbeadc2bc68..fde593e397a9c 100644 --- a/src/frontend/planner_test/tests/testdata/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/project_set.yaml @@ -8,14 +8,14 @@ - sql: | select unnest(Array[1,2,3]); batch_plan: | - BatchProject { exprs: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } - └─BatchProjectSet { select_list: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } + BatchProject { exprs: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } + └─BatchProjectSet { select_list: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } └─BatchValues { rows: [[]] } - sql: | select unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchProject { exprs: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } - └─BatchProjectSet { select_list: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } + BatchProject { exprs: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } + └─BatchProjectSet { select_list: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } └─BatchValues { rows: [[]] } - sql: | create table t(x int[]); @@ -43,9 +43,9 @@ create table t(x int[]); select unnest(x), unnest(Array[1,2]) from t; batch_plan: | - BatchProject { exprs: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } + BatchProject { exprs: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } └─BatchExchange { order: [], dist: Single } - └─BatchProjectSet { select_list: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } + └─BatchProjectSet { select_list: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } - name: table functions as parameters of usual functions sql: | diff --git a/src/frontend/planner_test/tests/testdata/range_scan.yaml b/src/frontend/planner_test/tests/testdata/range_scan.yaml index df8dbca94c355..4c6678d46ccbd 100644 --- a/src/frontend/planner_test/tests/testdata/range_scan.yaml +++ b/src/frontend/planner_test/tests/testdata/range_scan.yaml @@ -87,7 +87,7 @@ SELECT * FROM orders_count_by_user WHERE user_id > 42 AND date = 1111 AND 2>1 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (2:Int32 > 1:Int32) AND (orders_count_by_user.date = 1111:Int32) } + └─BatchFilter { predicate: true:Boolean AND (orders_count_by_user.date = 1111:Int32) } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id > Int64(42)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv @@ -95,7 +95,7 @@ SELECT * FROM orders_count_by_user WHERE date > 1111 AND user_id = 42 AND 5<6 AND date <= 6666 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (5:Int32 < 6:Int32) } + └─BatchFilter { predicate: true:Boolean } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id = Int64(42) AND orders_count_by_user.date > Int32(1111) AND orders_count_by_user.date <= Int32(6666)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv diff --git a/src/frontend/planner_test/tests/testdata/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/temporal_filter.yaml index 98a55a6044fa0..91dfe76e3d6c1 100644 --- a/src/frontend/planner_test/tests/testdata/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/temporal_filter.yaml @@ -40,7 +40,7 @@ sql: |- create table t1 (ts timestamp with time zone); select * from t1 where now() - interval '15 minutes' > ts; - stream_error: 'internal error: All `now()` exprs were valid, but the condition must have at least one now expr as a lower bound.' + stream_error: 'Expr error: Expected epoch timestamp bound into Now' - name: Temporal filter reorders now expressions correctly sql: | create table t1 (ts timestamp with time zone); diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index e7128d2bf3061..c7979f8ae3979 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -24,6 +24,8 @@ mod plan_visitor; pub use plan_visitor::PlanVisitor; mod optimizer_context; mod rule; +mod plan_expr_rewriter; +use plan_expr_rewriter::ConstEvalRewriter; use fixedbitset::FixedBitSet; use itertools::Itertools as _; @@ -427,6 +429,14 @@ impl PlanRoot { ApplyOrder::TopDown, ); + // Const eval of exprs at the last minute + plan = const_eval_exprs(plan)?; + + if explain_trace { + ctx.trace("Const eval exprs:"); + ctx.trace(plan.explain_to_string().unwrap()); + } + #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); @@ -710,6 +720,16 @@ impl PlanRoot { } } +fn const_eval_exprs(plan: PlanRef) -> Result { + let mut const_eval_rewriter = ConstEvalRewriter { error: None }; + + let plan = plan.rewrite_exprs(&mut const_eval_rewriter); + if let Some(error) = const_eval_rewriter.error { + return Err(error); + } + Ok(plan) +} + #[cfg(test)] mod tests { use risingwave_common::catalog::Field; diff --git a/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs new file mode 100644 index 0000000000000..4f85bccf6086e --- /dev/null +++ b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs @@ -0,0 +1,40 @@ +use risingwave_common::error::RwError; + +use crate::expr::{ExprRewriter, ExprImpl, Literal, Expr}; + +pub(crate) struct ConstEvalRewriter { + pub(crate) error: Option, +} +impl ExprRewriter for ConstEvalRewriter { + fn rewrite_expr(&mut self, expr: ExprImpl) -> ExprImpl { + if self.error.is_some() { + return expr; + } + if expr.is_const() { //} && expr.count_nows() == 0 { + let data_type = expr.return_type(); + match expr.eval_row_const() { + Ok(datum) => Literal::new(datum, data_type).into(), + Err(e) => { + self.error = Some(e); + expr + } + } + } else { + match expr { + ExprImpl::InputRef(inner) => self.rewrite_input_ref(*inner), + ExprImpl::Literal(inner) => self.rewrite_literal(*inner), + ExprImpl::FunctionCall(inner) => self.rewrite_function_call(*inner), + ExprImpl::AggCall(inner) => self.rewrite_agg_call(*inner), + ExprImpl::Subquery(inner) => self.rewrite_subquery(*inner), + ExprImpl::CorrelatedInputRef(inner) => { + self.rewrite_correlated_input_ref(*inner) + } + ExprImpl::TableFunction(inner) => self.rewrite_table_function(*inner), + ExprImpl::WindowFunction(inner) => self.rewrite_window_function(*inner), + ExprImpl::UserDefinedFunction(inner) => { + self.rewrite_user_defined_function(*inner) + } + } + } + } +} diff --git a/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs b/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs new file mode 100644 index 0000000000000..f2187c55ea780 --- /dev/null +++ b/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs @@ -0,0 +1,3 @@ + +mod const_eval_rewriter; +pub(crate) use const_eval_rewriter::ConstEvalRewriter; \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs index 775347e5ed969..5efc91640ae53 100644 --- a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs +++ b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs @@ -1,11 +1,12 @@ use std::ops::Deref; -use super::*; + use paste::paste; + +use super::*; use crate::expr::ExprRewriter; use crate::{for_batch_plan_nodes, for_stream_plan_nodes}; pub trait ExprRewritable { - // Default is to do nothing fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef; } @@ -22,12 +23,11 @@ impl ExprRewritable for PlanRef { } } - macro_rules! ban_expr_rewritable { ($( { $convention:ident, $name:ident }),*) => { paste!{ $(impl ExprRewritable for [<$convention $name>] { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { unimplemented!() } })* @@ -36,4 +36,3 @@ macro_rules! ban_expr_rewritable { } for_batch_plan_nodes! {ban_expr_rewritable} for_stream_plan_nodes! {ban_expr_rewritable} -// for_logical_plan_nodes! {ban_expr_rewritable} \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 93c8ee7f912bf..4aae7434a26dd 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -128,7 +128,6 @@ impl AggCallState { }), } } - } pub struct TableState { @@ -666,4 +665,4 @@ impl fmt::Debug for PlanAggCallDisplay<'_> { } Ok(()) } -} \ No newline at end of file +} diff --git a/src/frontend/src/optimizer/plan_node/generic/project.rs b/src/frontend/src/optimizer/plan_node/generic/project.rs index c77611a17727c..cef9fdf09c343 100644 --- a/src/frontend/src/optimizer/plan_node/generic/project.rs +++ b/src/frontend/src/optimizer/plan_node/generic/project.rs @@ -20,7 +20,7 @@ use itertools::Itertools; use risingwave_common::catalog::{Field, Schema}; use super::{GenericPlanNode, GenericPlanRef}; -use crate::expr::{assert_input_ref, Expr, ExprDisplay, ExprImpl, InputRef, ExprRewriter}; +use crate::expr::{assert_input_ref, Expr, ExprDisplay, ExprImpl, ExprRewriter, InputRef}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::utils::ColIndexMapping; @@ -52,7 +52,11 @@ pub struct Project { impl Project { pub(crate) fn rewrite_exprs(&mut self, r: &mut dyn ExprRewriter) { - self.exprs = self.exprs.iter().map(|e| r.rewrite_expr(e.clone())).collect(); + self.exprs = self + .exprs + .iter() + .map(|e| r.rewrite_expr(e.clone())) + .collect(); } } diff --git a/src/frontend/src/optimizer/plan_node/generic/project_set.rs b/src/frontend/src/optimizer/plan_node/generic/project_set.rs index 072e3a4ba2318..088784bfcc987 100644 --- a/src/frontend/src/optimizer/plan_node/generic/project_set.rs +++ b/src/frontend/src/optimizer/plan_node/generic/project_set.rs @@ -36,7 +36,11 @@ pub struct ProjectSet { impl ProjectSet { pub(crate) fn rewrite_exprs(&mut self, r: &mut dyn ExprRewriter) { - self.select_list = self.select_list.iter().map(|e| r.rewrite_expr(e.clone())).collect(); + self.select_list = self + .select_list + .iter() + .map(|e| r.rewrite_expr(e.clone())) + .collect(); } } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index d7e37402c9f24..b814fbd2bde31 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -25,9 +25,9 @@ use super::generic::{ ProjectBuilder, }; use super::{ - BatchHashAgg, BatchSimpleAgg, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, StreamGlobalSimpleAgg, StreamHashAgg, StreamLocalSimpleAgg, StreamProject, - ToBatch, ToStream, ExprRewritable, + BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, PlanBase, PlanRef, + PlanTreeNodeUnary, PredicatePushdown, StreamGlobalSimpleAgg, StreamHashAgg, + StreamLocalSimpleAgg, StreamProject, ToBatch, ToStream, }; use crate::catalog::table_catalog::TableCatalog; use crate::expr::{ diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index 7101b30a07537..8cf261a935fb6 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -24,9 +24,9 @@ use super::{ PredicatePushdown, ToBatch, ToStream, }; use crate::expr::{CorrelatedId, Expr, ExprImpl, ExprRewriter, InputRef}; -use crate::optimizer::plan_node::{ExprRewritable, - ColumnPruningContext, LogicalFilter, PredicatePushdownContext, RewriteStreamContext, - ToStreamContext, +use crate::optimizer::plan_node::{ + ColumnPruningContext, ExprRewritable, LogicalFilter, PredicatePushdownContext, + RewriteStreamContext, ToStreamContext, }; use crate::optimizer::property::FunctionalDependencySet; use crate::utils::{ColIndexMapping, Condition, ConditionDisplay}; diff --git a/src/frontend/src/optimizer/plan_node/logical_delete.rs b/src/frontend/src/optimizer/plan_node/logical_delete.rs index 970b8a5e95ccf..5d1fcd2ca589c 100644 --- a/src/frontend/src/optimizer/plan_node/logical_delete.rs +++ b/src/frontend/src/optimizer/plan_node/logical_delete.rs @@ -19,8 +19,8 @@ use risingwave_common::error::Result; use risingwave_common::types::DataType; use super::{ - gen_filter_and_pushdown, BatchDelete, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, ExprRewritable, + gen_filter_and_pushdown, BatchDelete, ColPrunable, ExprRewritable, PlanBase, PlanRef, + PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; use crate::expr::ExprRewriter; @@ -128,7 +128,7 @@ impl ColPrunable for LogicalDelete { } impl ExprRewritable for LogicalDelete { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { self.clone().into() } } diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index a73d8505ab758..fda50a457d51b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -20,8 +20,8 @@ use risingwave_common::error::Result; use super::generic::GenericPlanNode; use super::{ - gen_filter_and_pushdown, generic, BatchExpand, ColPrunable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamExpand, ToBatch, ToStream, ExprRewritable, + gen_filter_and_pushdown, generic, BatchExpand, ColPrunable, ExprRewritable, PlanBase, PlanRef, + PlanTreeNodeUnary, PredicatePushdown, StreamExpand, ToBatch, ToStream, }; use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{ @@ -159,7 +159,7 @@ impl ColPrunable for LogicalExpand { } impl ExprRewritable for LogicalExpand { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { self.clone().into() } } diff --git a/src/frontend/src/optimizer/plan_node/logical_filter.rs b/src/frontend/src/optimizer/plan_node/logical_filter.rs index fb822588f3cee..eabea1f499fbe 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::error::Result; use super::generic::{self, GenericPlanNode}; use super::{ - ColPrunable, CollectInputRef, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, ExprRewritable, + ColPrunable, CollectInputRef, ExprRewritable, LogicalProject, PlanBase, PlanRef, + PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::expr::{assert_input_ref, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::{ @@ -175,7 +175,6 @@ impl ColPrunable for LogicalFilter { } } - impl ExprRewritable for LogicalFilter { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut new = self.clone(); 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 daa139e85bf65..7133d6301e377 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -22,10 +22,10 @@ use risingwave_common::types::{DataType, IntervalUnit}; use super::generic::GenericPlanNode; use super::{ - gen_filter_and_pushdown, generic, BatchHopWindow, ColPrunable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, ToBatch, ToStream, ExprRewritable, + gen_filter_and_pushdown, generic, BatchHopWindow, ColPrunable, ExprRewritable, PlanBase, + PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, ToBatch, ToStream, }; -use crate::expr::{InputRef, ExprRewriter}; +use crate::expr::{ExprRewriter, InputRef}; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -328,7 +328,7 @@ impl ColPrunable for LogicalHopWindow { } impl ExprRewritable for LogicalHopWindow { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { self.clone().into() } } diff --git a/src/frontend/src/optimizer/plan_node/logical_insert.rs b/src/frontend/src/optimizer/plan_node/logical_insert.rs index 137f28f5fb5b0..f9115f7507d5c 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 risingwave_common::types::DataType; use super::{ - gen_filter_and_pushdown, BatchInsert, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, ExprRewritable, + gen_filter_and_pushdown, BatchInsert, ColPrunable, ExprRewritable, PlanBase, PlanRef, + PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; use crate::expr::ExprRewriter; @@ -162,7 +162,7 @@ impl ColPrunable for LogicalInsert { } impl ExprRewritable for LogicalInsert { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { self.clone().into() } } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index 89a60e5641640..aca13e469bd9d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -24,8 +24,9 @@ use risingwave_pb::plan_common::JoinType; use super::generic::GenericPlanNode; use super::{ - generic, BatchProject, ColPrunable, CollectInputRef, LogicalProject, PlanBase, PlanRef, - PlanTreeNodeBinary, PredicatePushdown, StreamHashJoin, StreamProject, ToBatch, ToStream, ExprRewritable, + generic, BatchProject, ColPrunable, CollectInputRef, ExprRewritable, LogicalProject, PlanBase, + PlanRef, PlanTreeNodeBinary, PredicatePushdown, StreamHashJoin, StreamProject, ToBatch, + ToStream, }; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, InputRef}; use crate::optimizer::plan_node::generic::GenericPlanRef; diff --git a/src/frontend/src/optimizer/plan_node/logical_limit.rs b/src/frontend/src/optimizer/plan_node/logical_limit.rs index 8ce88f88abbdb..2600de15bd64c 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 std::fmt; use risingwave_common::error::{ErrorCode, Result, RwError}; use super::{ - gen_filter_and_pushdown, BatchLimit, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, ExprRewritable, + gen_filter_and_pushdown, BatchLimit, ColPrunable, ExprRewritable, PlanBase, PlanRef, + PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{ @@ -101,7 +101,7 @@ impl ColPrunable for LogicalLimit { } impl ExprRewritable for LogicalLimit { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { self.clone().into() } } 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 4f90f84d08c84..9b7f377b33d13 100644 --- a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs @@ -20,8 +20,9 @@ use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_pb::plan_common::JoinType; use super::{ - ColPrunable, LogicalFilter, LogicalJoin, LogicalProject, PlanBase, PlanNodeType, PlanRef, - PlanTreeNodeBinary, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, ExprRewritable, + ColPrunable, ExprRewritable, LogicalFilter, LogicalJoin, LogicalProject, PlanBase, + PlanNodeType, PlanRef, PlanTreeNodeBinary, PlanTreeNodeUnary, PredicatePushdown, ToBatch, + ToStream, }; use crate::expr::{ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::{ @@ -523,7 +524,7 @@ impl ColPrunable for LogicalMultiJoin { } impl ExprRewritable for LogicalMultiJoin { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { panic!( "Method not available for `LogicalMultiJoin` which is a placeholder node with \ a temporary lifetime. It only facilitates join reordering during logical planning." diff --git a/src/frontend/src/optimizer/plan_node/logical_now.rs b/src/frontend/src/optimizer/plan_node/logical_now.rs index 7db3433dc18d0..77a6004390bf5 100644 --- a/src/frontend/src/optimizer/plan_node/logical_now.rs +++ b/src/frontend/src/optimizer/plan_node/logical_now.rs @@ -22,8 +22,8 @@ use risingwave_common::types::DataType; use super::utils::IndicesDisplay; use super::{ - ColPrunable, ColumnPruningContext, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, - RewriteStreamContext, StreamNow, ToBatch, ToStream, ToStreamContext, ExprRewritable, + ColPrunable, ColumnPruningContext, ExprRewritable, LogicalFilter, PlanBase, PlanRef, + PredicatePushdown, RewriteStreamContext, StreamNow, ToBatch, ToStream, ToStreamContext, }; use crate::expr::ExprRewriter; use crate::optimizer::property::FunctionalDependencySet; @@ -71,9 +71,10 @@ impl fmt::Display for LogicalNow { impl_plan_tree_node_for_leaf! { LogicalNow } impl ExprRewritable for LogicalNow { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { self.clone().into() - }} + } +} impl PredicatePushdown for LogicalNow { fn predicate_pushdown( diff --git a/src/frontend/src/optimizer/plan_node/logical_over_agg.rs b/src/frontend/src/optimizer/plan_node/logical_over_agg.rs index d65e51c7bd7fc..b8364d095c6c9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_over_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_over_agg.rs @@ -22,10 +22,12 @@ use risingwave_common::types::DataType; use super::generic::{PlanAggOrderByField, PlanAggOrderByFieldDisplay}; use super::{ - gen_filter_and_pushdown, ColPrunable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, ExprRewritable, + gen_filter_and_pushdown, ColPrunable, ExprRewritable, LogicalProject, PlanBase, PlanRef, + PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, +}; +use crate::expr::{ + Expr, ExprImpl, ExprRewriter, InputRef, InputRefDisplay, WindowFunction, WindowFunctionType, }; -use crate::expr::{Expr, ExprImpl, InputRef, InputRefDisplay, WindowFunction, WindowFunctionType, ExprRewriter}; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -265,9 +267,10 @@ impl ColPrunable for LogicalOverAgg { } impl ExprRewritable for LogicalOverAgg { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { self.clone().into() - }} + } +} impl PredicatePushdown for LogicalOverAgg { fn predicate_pushdown( diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index ac12f866f8f8c..f71343fc6c225 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -20,8 +20,8 @@ use risingwave_common::error::Result; use super::generic::{self, GenericPlanNode, Project}; use super::{ - gen_filter_and_pushdown, BatchProject, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, StreamProject, ToBatch, ToStream, ExprRewritable, + gen_filter_and_pushdown, BatchProject, ColPrunable, ExprRewritable, PlanBase, PlanRef, + PlanTreeNodeUnary, PredicatePushdown, StreamProject, ToBatch, ToStream, }; use crate::expr::{ExprImpl, ExprRewriter, ExprVisitor, InputRef}; use crate::optimizer::plan_node::{ 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 098f9b258dc15..0c84b65c798e9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project_set.rs @@ -17,8 +17,8 @@ use std::fmt; use risingwave_common::error::Result; use super::{ - generic, BatchProjectSet, ColPrunable, LogicalFilter, LogicalProject, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, ToBatch, ToStream, ExprRewritable, + generic, BatchProjectSet, ColPrunable, ExprRewritable, LogicalFilter, LogicalProject, PlanBase, + PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, ToBatch, ToStream, }; use crate::expr::{Expr, ExprImpl, ExprRewriter, FunctionCall, InputRef, TableFunction}; use crate::optimizer::plan_node::generic::GenericPlanNode; diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 886fffcea9ed2..b35fcb3982572 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -24,8 +24,8 @@ use risingwave_common::util::sort_util::OrderType; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::{ - generic, BatchFilter, BatchProject, ColPrunable, PlanBase, PlanRef, PredicatePushdown, - StreamTableScan, ToBatch, ToStream, ExprRewritable, + generic, BatchFilter, BatchProject, ColPrunable, ExprRewritable, PlanBase, PlanRef, + PredicatePushdown, StreamTableScan, ToBatch, ToStream, }; use crate::catalog::{ColumnId, IndexCatalog}; use crate::expr::{ diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs index b3f59a39aa619..1e5be1df066c0 100644 --- a/src/frontend/src/optimizer/plan_node/logical_share.rs +++ b/src/frontend/src/optimizer/plan_node/logical_share.rs @@ -20,7 +20,8 @@ use risingwave_common::error::Result; use super::generic::{self, GenericPlanNode}; use super::{ - ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, ExprRewritable, + ColPrunable, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, + ToStream, }; use crate::expr::ExprRewriter; use crate::optimizer::plan_node::generic::GenericPlanRef; @@ -120,7 +121,7 @@ impl ColPrunable for LogicalShare { } impl ExprRewritable for LogicalShare { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { unimplemented!( "call rewrite_exprs of the PlanRef instead of calling directly on LogicalShare" ) diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index caa44cba6c97f..bf8cda4bd02fb 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -24,12 +24,12 @@ use risingwave_connector::source::DataType; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::{ - generic, BatchSource, ColPrunable, LogicalFilter, LogicalProject, PlanBase, PlanRef, - PredicatePushdown, StreamRowIdGen, StreamSource, ToBatch, ToStream, ExprRewritable, + generic, BatchSource, ColPrunable, ExprRewritable, LogicalFilter, LogicalProject, PlanBase, + PlanRef, PredicatePushdown, StreamRowIdGen, StreamSource, ToBatch, ToStream, }; use crate::catalog::source_catalog::SourceCatalog; use crate::catalog::ColumnId; -use crate::expr::{Expr, ExprImpl, ExprType, ExprRewriter}; +use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, @@ -162,9 +162,10 @@ impl ColPrunable for LogicalSource { } impl ExprRewritable for LogicalSource { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { self.clone().into() - }} + } +} /// A util function to extract kafka offset timestamp range. /// 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 c7d19947e558e..6235e31cc52f2 100644 --- a/src/frontend/src/optimizer/plan_node/logical_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/logical_table_function.rs @@ -17,8 +17,11 @@ use std::fmt; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::{ErrorCode, Result}; -use super::{ColPrunable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, ExprRewritable}; -use crate::expr::{Expr, TableFunction, ExprRewriter}; +use super::{ + ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, ToBatch, + ToStream, +}; +use crate::expr::{Expr, ExprRewriter, TableFunction}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::{ BatchTableFunction, ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, @@ -71,7 +74,12 @@ impl ColPrunable for LogicalTableFunction { impl ExprRewritable for LogicalTableFunction { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut new = self.clone(); - new.table_function.args = new.table_function.args.into_iter().map(|e| r.rewrite_expr(e)).collect(); + new.table_function.args = new + .table_function + .args + .into_iter() + .map(|e| r.rewrite_expr(e)) + .collect(); new.into() } } diff --git a/src/frontend/src/optimizer/plan_node/logical_topn.rs b/src/frontend/src/optimizer/plan_node/logical_topn.rs index ca8e82e53f188..bb1b60784d95d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_topn.rs +++ b/src/frontend/src/optimizer/plan_node/logical_topn.rs @@ -20,10 +20,11 @@ use risingwave_common::error::{ErrorCode, Result, RwError}; use super::generic::GenericPlanNode; use super::{ - gen_filter_and_pushdown, generic, BatchGroupTopN, ColPrunable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamGroupTopN, StreamProject, ToBatch, ToStream, ExprRewritable, + gen_filter_and_pushdown, generic, BatchGroupTopN, ColPrunable, ExprRewritable, PlanBase, + PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamGroupTopN, StreamProject, ToBatch, + ToStream, }; -use crate::expr::{ExprType, FunctionCall, InputRef, ExprRewriter}; +use crate::expr::{ExprRewriter, ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ BatchTopN, ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, StreamTopN, ToStreamContext, @@ -354,9 +355,10 @@ impl ColPrunable for LogicalTopN { } impl ExprRewritable for LogicalTopN { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { self.clone().into() - }} + } +} impl PredicatePushdown for LogicalTopN { fn predicate_pushdown( diff --git a/src/frontend/src/optimizer/plan_node/logical_union.rs b/src/frontend/src/optimizer/plan_node/logical_union.rs index b83ea989b2756..9a5fd102cd49a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_union.rs +++ b/src/frontend/src/optimizer/plan_node/logical_union.rs @@ -18,8 +18,8 @@ use itertools::Itertools; use risingwave_common::error::Result; use risingwave_common::types::{DataType, Scalar}; -use super::{ColPrunable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, ExprRewritable}; -use crate::expr::{ExprImpl, InputRef, Literal, ExprRewriter}; +use super::{ColPrunable, ExprRewritable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream}; +use crate::expr::{ExprImpl, ExprRewriter, InputRef, Literal}; use crate::optimizer::plan_node::generic::{GenericPlanNode, GenericPlanRef}; use crate::optimizer::plan_node::stream_union::StreamUnion; use crate::optimizer::plan_node::{ @@ -110,9 +110,10 @@ impl ColPrunable for LogicalUnion { } impl ExprRewritable for LogicalUnion { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { self.clone().into() - }} + } +} impl PredicatePushdown for LogicalUnion { fn predicate_pushdown( diff --git a/src/frontend/src/optimizer/plan_node/logical_update.rs b/src/frontend/src/optimizer/plan_node/logical_update.rs index 5e8dd0fb46570..3d0d1fcd2ef96 100644 --- a/src/frontend/src/optimizer/plan_node/logical_update.rs +++ b/src/frontend/src/optimizer/plan_node/logical_update.rs @@ -19,8 +19,8 @@ use risingwave_common::error::Result; use risingwave_common::types::DataType; use super::{ - gen_filter_and_pushdown, BatchUpdate, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, ExprRewritable, + gen_filter_and_pushdown, BatchUpdate, ColPrunable, ExprRewritable, PlanBase, PlanRef, + PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; use crate::expr::{ExprImpl, ExprRewriter}; diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index 1613aedf6aa71..2806f1c992962 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -19,8 +19,8 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result, RwError}; use super::{ - BatchValues, ColPrunable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, ToBatch, - ToStream, ExprRewritable, + BatchValues, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, + ToBatch, ToStream, }; use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::optimizer_context::OptimizerContextRef; @@ -79,7 +79,17 @@ impl fmt::Display for LogicalValues { impl ExprRewritable for LogicalValues { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut new = self.clone(); - new.rows = new.rows.iter().map(|exprs| exprs.iter().map(|e| r.rewrite_expr(e.clone())).collect::>()).collect::>().into(); + new.rows = new + .rows + .iter() + .map(|exprs| { + exprs + .iter() + .map(|e| r.rewrite_expr(e.clone())) + .collect::>() + }) + .collect::>() + .into(); new.into() } } @@ -99,8 +109,6 @@ impl ColPrunable for LogicalValues { } } - - impl PredicatePushdown for LogicalValues { fn predicate_pushdown( &self, 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 dd50f331fb042..11ebab408d1f0 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -132,9 +132,11 @@ impl_plan_tree_node_for_binary! { StreamDynamicFilter } impl StreamNode for StreamDynamicFilter { fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> NodeBody { use generic::dynamic_filter::*; - let condition = self.core.predicate().as_expr_unless_true().map(|x| x - .to_expr_proto() - ); + let condition = self + .core + .predicate() + .as_expr_unless_true() + .map(|x| x.to_expr_proto()); let left_index = self.core.left_index; let left_table = infer_left_internal_table_catalog(&self.base, left_index) .with_id(state.gen_table_id_wrapped()); From d5e05ef446993b4641644b403ac9eb2c59a3c111 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 26 Jan 2023 18:06:20 +0800 Subject: [PATCH 03/22] fmt --- src/frontend/src/optimizer/mod.rs | 5 ++--- .../plan_expr_rewriter/const_eval_rewriter.rs | 13 +++++-------- .../src/optimizer/plan_expr_rewriter/mod.rs | 3 +-- 3 files changed, 8 insertions(+), 13 deletions(-) diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index c7979f8ae3979..e7d4c82edb934 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -23,13 +23,12 @@ pub use plan_rewriter::PlanRewriter; mod plan_visitor; pub use plan_visitor::PlanVisitor; mod optimizer_context; -mod rule; mod plan_expr_rewriter; -use plan_expr_rewriter::ConstEvalRewriter; - +mod rule; use fixedbitset::FixedBitSet; use itertools::Itertools as _; pub use optimizer_context::*; +use plan_expr_rewriter::ConstEvalRewriter; use plan_rewriter::ShareSourceRewriter; use property::Order; use risingwave_common::catalog::{Field, Schema}; diff --git a/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs index 4f85bccf6086e..9804cc15d71c8 100644 --- a/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs +++ b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs @@ -1,6 +1,6 @@ use risingwave_common::error::RwError; -use crate::expr::{ExprRewriter, ExprImpl, Literal, Expr}; +use crate::expr::{Expr, ExprImpl, ExprRewriter, Literal}; pub(crate) struct ConstEvalRewriter { pub(crate) error: Option, @@ -10,7 +10,8 @@ impl ExprRewriter for ConstEvalRewriter { if self.error.is_some() { return expr; } - if expr.is_const() { //} && expr.count_nows() == 0 { + if expr.is_const() { + //} && expr.count_nows() == 0 { let data_type = expr.return_type(); match expr.eval_row_const() { Ok(datum) => Literal::new(datum, data_type).into(), @@ -26,14 +27,10 @@ impl ExprRewriter for ConstEvalRewriter { ExprImpl::FunctionCall(inner) => self.rewrite_function_call(*inner), ExprImpl::AggCall(inner) => self.rewrite_agg_call(*inner), ExprImpl::Subquery(inner) => self.rewrite_subquery(*inner), - ExprImpl::CorrelatedInputRef(inner) => { - self.rewrite_correlated_input_ref(*inner) - } + ExprImpl::CorrelatedInputRef(inner) => self.rewrite_correlated_input_ref(*inner), ExprImpl::TableFunction(inner) => self.rewrite_table_function(*inner), ExprImpl::WindowFunction(inner) => self.rewrite_window_function(*inner), - ExprImpl::UserDefinedFunction(inner) => { - self.rewrite_user_defined_function(*inner) - } + ExprImpl::UserDefinedFunction(inner) => self.rewrite_user_defined_function(*inner), } } } diff --git a/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs b/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs index f2187c55ea780..02a715ff5a06e 100644 --- a/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs +++ b/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs @@ -1,3 +1,2 @@ - mod const_eval_rewriter; -pub(crate) use const_eval_rewriter::ConstEvalRewriter; \ No newline at end of file +pub(crate) use const_eval_rewriter::ConstEvalRewriter; From 592bb4faa4ca52a98431a4c9106051f1fe520008 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 26 Jan 2023 18:20:15 +0800 Subject: [PATCH 04/22] license --- .../plan_expr_rewriter/const_eval_rewriter.rs | 14 ++++++++++++++ .../src/optimizer/plan_expr_rewriter/mod.rs | 14 ++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs index 9804cc15d71c8..a3f94cc59bbdb 100644 --- a/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs +++ b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs @@ -1,3 +1,17 @@ +// Copyright 2023 Singularity Data +// +// 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_common::error::RwError; use crate::expr::{Expr, ExprImpl, ExprRewriter, Literal}; diff --git a/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs b/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs index 02a715ff5a06e..4dcb60ca5e729 100644 --- a/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs +++ b/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs @@ -1,2 +1,16 @@ +// Copyright 2023 Singularity Data +// +// 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 const_eval_rewriter; pub(crate) use const_eval_rewriter::ConstEvalRewriter; From 93556199859862ed453b3ee6b5c59087889fd685 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 26 Jan 2023 18:25:01 +0800 Subject: [PATCH 05/22] license --- .../src/optimizer/plan_node/expr_rewritable.rs | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs index 5efc91640ae53..5358f348afd2b 100644 --- a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs +++ b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs @@ -1,3 +1,17 @@ +// Copyright 2023 Singularity Data +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + use std::ops::Deref; use paste::paste; From a688ed9b66c524f8a560901426e9855ff2da2bf9 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Sat, 28 Jan 2023 17:03:18 +0800 Subject: [PATCH 06/22] fix recursive --- src/frontend/src/optimizer/mod.rs | 2 +- .../src/optimizer/plan_node/expr_rewritable.rs | 12 ++++-------- .../src/optimizer/plan_node/logical_share.rs | 4 +--- src/frontend/src/optimizer/plan_node/mod.rs | 12 +++++++++++- 4 files changed, 17 insertions(+), 13 deletions(-) diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index e7d4c82edb934..e96d72ca2e8a4 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -722,7 +722,7 @@ impl PlanRoot { fn const_eval_exprs(plan: PlanRef) -> Result { let mut const_eval_rewriter = ConstEvalRewriter { error: None }; - let plan = plan.rewrite_exprs(&mut const_eval_rewriter); + let plan = plan.rewrite_exprs_recursive(&mut const_eval_rewriter); if let Some(error) = const_eval_rewriter.error { return Err(error); } diff --git a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs index 5358f348afd2b..1ff3920d5dcba 100644 --- a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs +++ b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs @@ -20,20 +20,16 @@ use super::*; use crate::expr::ExprRewriter; use crate::{for_batch_plan_nodes, for_stream_plan_nodes}; +/// Rewrites expressions in a `PlanRef`. Due to `Share` operator, +/// the `ExprRewriter` needs to be idempotent i.e. applying it more than once +/// to the same `ExprImpl` will be a noop on subsequent applications. pub trait ExprRewritable { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef; } impl ExprRewritable for PlanRef { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - let inputs: Vec = self - .inputs() - .iter() - .map(|plan_ref| plan_ref.rewrite_exprs(r)) - .collect(); - let new = self.clone_with_inputs(&inputs[..]); - let dyn_t = new.deref(); - dyn_t.rewrite_exprs(r) + self.deref().rewrite_exprs(r) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs index 1e5be1df066c0..d91e2efe69aa0 100644 --- a/src/frontend/src/optimizer/plan_node/logical_share.rs +++ b/src/frontend/src/optimizer/plan_node/logical_share.rs @@ -122,9 +122,7 @@ impl ColPrunable for LogicalShare { impl ExprRewritable for LogicalShare { fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - unimplemented!( - "call rewrite_exprs of the PlanRef instead of calling directly on LogicalShare" - ) + self.clone().into() } } diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 9943ad6621aa9..3af2437dc64d0 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -373,6 +373,16 @@ impl dyn PlanNode { } } + pub fn rewrite_exprs_recursive(&self, r: &mut impl ExprRewriter) -> PlanRef { + let new = self.rewrite_exprs(r); + let inputs: Vec = new + .inputs() + .iter() + .map(|plan_ref| plan_ref.rewrite_exprs_recursive(r)) + .collect(); + new.clone_with_inputs(&inputs[..]) + } + /// Serialize the plan node and its children to a batch plan proto. pub fn to_batch_prost(&self) -> BatchPlanProst { self.to_batch_prost_identity(true) @@ -569,7 +579,7 @@ pub use stream_table_scan::StreamTableScan; pub use stream_topn::StreamTopN; pub use stream_union::StreamUnion; -use crate::expr::{ExprImpl, InputRef, Literal}; +use crate::expr::{ExprImpl, ExprRewriter, InputRef, Literal}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::utils::{ColIndexMapping, Condition}; From 1e6f001604c95a216c6cedef18568563a960c197 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Sat, 28 Jan 2023 17:03:34 +0800 Subject: [PATCH 07/22] planner-test --- .../planner_test/tests/testdata/explain.yaml | 59 ++- .../planner_test/tests/testdata/expr.yaml | 12 +- .../planner_test/tests/testdata/insert.yaml | 6 +- .../tests/testdata/predicate_pushdown.yaml | 4 +- .../planner_test/tests/testdata/share.yaml | 394 +++++++++++++++++- .../planner_test/tests/testdata/tpch.yaml | 140 +++---- 6 files changed, 528 insertions(+), 87 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index ea082c6c69bd8..365ecbf79256e 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -36,6 +36,28 @@ LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + WTF2 + + + Node: Ok("LogicalProject { exprs: [1:Int32] }\n└─LogicalValues { rows: [[]], schema: Schema { fields: [] } }\n") + + Rewritten Node: Ok("LogicalProject { exprs: [1:Int32] }\n└─LogicalValues { rows: [[]], schema: Schema { fields: [] } }\n") + + inputs before: [Ok("LogicalValues { rows: [[]], schema: Schema { fields: [] } }\n")] + + WTF2 + + + Node: Ok("LogicalValues { rows: [[]], schema: Schema { fields: [] } }\n") + + Rewritten Node: Ok("LogicalValues { rows: [[]], schema: Schema { fields: [] } }\n") + + inputs before: [] + + inputs after: [] + + inputs after: [Ok("LogicalValues { rows: [[]], schema: Schema { fields: [] } }\n")] + Const eval exprs: LogicalProject { exprs: [1:Int32] } @@ -56,7 +78,7 @@ "stages": { "0": { "root": { - "plan_node_id": 28, + "plan_node_id": 29, "plan_node_type": "BatchProject", "schema": [ { @@ -69,7 +91,7 @@ ], "children": [ { - "plan_node_id": 26, + "plan_node_id": 27, "plan_node_type": "BatchValues", "schema": [], "children": [], @@ -170,6 +192,39 @@ ├─LogicalScan { table: t1, columns: [v1] } └─LogicalScan { table: t2, columns: [v2] } + WTF2 + + + Node: Ok("LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) }\n├─LogicalScan { table: t1, columns: [v1] }\n└─LogicalScan { table: t2, columns: [v2] }\n") + + Rewritten Node: Ok("LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) }\n├─LogicalScan { table: t1, columns: [v1] }\n└─LogicalScan { table: t2, columns: [v2] }\n") + + inputs before: [Ok("LogicalScan { table: t1, columns: [v1] }\n"), Ok("LogicalScan { table: t2, columns: [v2] }\n")] + + WTF2 + + + Node: Ok("LogicalScan { table: t1, columns: [v1] }\n") + + Rewritten Node: Ok("LogicalScan { table: t1, columns: [v1] }\n") + + inputs before: [] + + inputs after: [] + + WTF2 + + + Node: Ok("LogicalScan { table: t2, columns: [v2] }\n") + + Rewritten Node: Ok("LogicalScan { table: t2, columns: [v2] }\n") + + inputs before: [] + + inputs after: [] + + inputs after: [Ok("LogicalScan { table: t1, columns: [v1] }\n"), Ok("LogicalScan { table: t2, columns: [v2] }\n")] + Const eval exprs: LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 286ae7e5c65b0..4e8b8ac2e8f7c 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -94,7 +94,7 @@ SELECT b2 from b where 1 in (3, 1.0, (select min(v1) from t)); batch_plan: | BatchProject { exprs: [b.b2] } - └─BatchFilter { predicate: (In(1:Int32::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (1:Int32 = min(min(t.v1)))) } + └─BatchFilter { predicate: (true:Boolean OR (1:Int32 = min(min(t.v1)))) } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b2], distribution: SomeShard } @@ -171,7 +171,7 @@ select position(replace('1','1','2'),'123') where '12' like '%1'; batch_plan: | BatchProject { exprs: [0:Int32] } - └─BatchFilter { predicate: Like('12':Varchar, '%1':Varchar) } + └─BatchFilter { predicate: false:Boolean } └─BatchValues { rows: [[]] } - name: case searched form with else sql: | @@ -420,10 +420,10 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [Some((1:Int32 < ArrayCat(Array(1:Int32), ARRAY[2]:List { datatype: Int32 })))] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat(ARRAY[1]:List { datatype: Int32 }, ARRAY[2]:List { datatype: Int32 })))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchProject { exprs: [Array(1:Int32)] } + └─BatchProject { exprs: [ARRAY[1]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - sql: | select 1 < ALL(array[null]::integer[]); @@ -444,10 +444,10 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [All((1:Int32 < ArrayCat(Array(1:Int32), ARRAY[2]:List { datatype: Int32 })))] } + BatchProject { exprs: [All((1:Int32 < ArrayCat(ARRAY[1]:List { datatype: Int32 }, ARRAY[2]:List { datatype: Int32 })))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchProject { exprs: [Array(1:Int32)] } + └─BatchProject { exprs: [ARRAY[1]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - name: now expression sql: | diff --git a/src/frontend/planner_test/tests/testdata/insert.yaml b/src/frontend/planner_test/tests/testdata/insert.yaml index 40a528fcfae51..985277c9273f7 100644 --- a/src/frontend/planner_test/tests/testdata/insert.yaml +++ b/src/frontend/planner_test/tests/testdata/insert.yaml @@ -14,7 +14,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [[22.33:Decimal::Float32, 33:Int32], [44:Int32::Float32, 55.0:Decimal::Int32]] } + └─BatchValues { rows: [[22.33:Float32, 33:Int32], [44:Float32, 55:Int32]] } - name: insert values on non-assign-castable types sql: | create table t (v1 real, v2 int); @@ -73,7 +73,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [['2020-01-01 01:02:03':Timestamp::Time], ['03:04:05':Time]] } + └─BatchValues { rows: [['01:02:03':Time], ['03:04:05':Time]] } - name: a `VALUES` without insert context may be invalid on its own (compare with above) sql: | create table t (v1 time); @@ -115,7 +115,7 @@ BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } └─BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: ['2020-01-01 01:02:03':Timestamp::Time, 11:Int32, 4.5:Decimal::Float32] } + └─BatchProject { exprs: ['01:02:03':Time, 11:Int32, 4.5:Float32] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: insert into select with cast error sql: | diff --git a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml index f8b2fda0b1111..b7d24e08fceb9 100644 --- a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml @@ -257,7 +257,7 @@ select * from t1 join t2 where v1 = v2 and v1 > now() + '1 hr'; optimized_logical_plan: | LogicalJoin { type: Inner, on: (t1.v1 = t2.v2), output: all } - ├─LogicalFilter { predicate: (t1.v1 > (Now + '01:00:00':Interval)) } + ├─LogicalFilter { predicate: (t1.v1 > '2021-04-01 01:00:00+00:00':Timestamptz) } | └─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | @@ -343,7 +343,7 @@ select * from t1 join t2 where v1 = v2 and v1 > now(); optimized_logical_plan: | LogicalJoin { type: Inner, on: (t1.v1 = t2.v2), output: all } - ├─LogicalFilter { predicate: (t1.v1 > Now) } + ├─LogicalFilter { predicate: (t1.v1 > '2021-04-01 00:00:00+00:00':Timestamptz) } | └─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index d150ff23dcd82..328f922dc4268 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -20,6 +20,21 @@ - create_sources sql: | select count(*) cnt from auction A join auction B on A.id = B.id where A.initial_bid = 1 and B.initial_bid = 2; + optimized_logical_plan: | + LogicalAgg { aggs: [count] } + └─LogicalJoin { type: Inner, on: (id = id), output: [] } + ├─LogicalProject { exprs: [id] } + | └─LogicalFilter { predicate: (initial_bid = 1:Int32) } + | └─LogicalShare { id = 91 } + | └─LogicalProject { exprs: [id, initial_bid] } + | └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } + | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + └─LogicalProject { exprs: [id] } + └─LogicalFilter { predicate: (initial_bid = 2:Int32) } + └─LogicalShare { id = 91 } + └─LogicalProject { exprs: [id, initial_bid] } + └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } + └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } batch_plan: | BatchSimpleAgg { aggs: [sum0(count)] } └─BatchExchange { order: [], dist: Single } @@ -48,7 +63,7 @@ | └─StreamProject { exprs: [id, _row_id] } | └─StreamFilter { predicate: (initial_bid = 1:Int32) } | └─StreamProject { exprs: [id, initial_bid, _row_id] } - | └─StreamShare { id = 519 } + | └─StreamShare { id = 757 } | └─StreamProject { exprs: [id, initial_bid, _row_id] } | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } | └─StreamRowIdGen { row_id_index: 9 } @@ -57,7 +72,7 @@ └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } └─StreamProject { exprs: [id, initial_bid, _row_id] } - └─StreamShare { id = 519 } + └─StreamShare { id = 757 } └─StreamProject { exprs: [id, initial_bid, _row_id] } └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 9 } @@ -125,7 +140,7 @@ | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } | └─StreamProject { exprs: [auction, date_time, _row_id] } - | └─StreamShare { id = 744 } + | └─StreamShare { id = 764 } | └─StreamProject { exprs: [auction, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 4 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } @@ -137,7 +152,378 @@ └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } └─StreamProject { exprs: [auction, date_time, _row_id] } - └─StreamShare { id = 744 } + └─StreamShare { id = 764 } └─StreamProject { exprs: [auction, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 4 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } +- id: self_join_13 + before: + - create_sources + sql: | + explain (trace, logical) select count(*) = (1 + 2) cnt from auction A join auction B on A.id = B.id where A.initial_bid = 2-1 and B.initial_bid = 5-3; + explain_output: |+ + Begin: + + LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } + └─LogicalAgg { aggs: [count] } + └─LogicalProject { exprs: [] } + └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) AND (initial_bid = (5:Int32 - 3:Int32)) } + └─LogicalJoin { type: Inner, on: (id = id) } + ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + + Share Source: + + LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } + └─LogicalAgg { aggs: [count] } + └─LogicalProject { exprs: [] } + └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) AND (initial_bid = (5:Int32 - 3:Int32)) } + └─LogicalJoin { type: Inner, on: (id = id) } + ├─LogicalShare { id = 8 } + | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + └─LogicalShare { id = 8 } + └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + + Predicate Push Down: + + LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } + └─LogicalAgg { aggs: [count] } + └─LogicalProject { exprs: [] } + └─LogicalJoin { type: Inner, on: (id = id) } + ├─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } + | └─LogicalShare { id = 8 } + | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } + └─LogicalShare { id = 8 } + └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + + Predicate Push Down: + + LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } + └─LogicalAgg { aggs: [count] } + └─LogicalProject { exprs: [] } + └─LogicalJoin { type: Inner, on: (id = id) } + ├─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } + | └─LogicalShare { id = 8 } + | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } + └─LogicalShare { id = 8 } + └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + + Predicate Push Down: + + LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } + └─LogicalAgg { aggs: [count] } + └─LogicalProject { exprs: [] } + └─LogicalJoin { type: Inner, on: (id = id) } + ├─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } + | └─LogicalShare { id = 8 } + | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } + └─LogicalShare { id = 8 } + └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + + Push Down the Calculation of Inputs of Join's Condition: + + apply PushCalculationOfJoinRule 1 time(s) + + LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } + └─LogicalAgg { aggs: [count] } + └─LogicalProject { exprs: [] } + └─LogicalJoin { type: Inner, on: (id = id) } + ├─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } + | └─LogicalShare { id = 8 } + | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } + └─LogicalShare { id = 8 } + └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + + Prune Columns: + + LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } + └─LogicalAgg { aggs: [count] } + └─LogicalProject { exprs: [] } + └─LogicalJoin { type: Inner, on: (id = id) } + ├─LogicalProject { exprs: [id] } + | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } + | └─LogicalProject { exprs: [id, initial_bid] } + | └─LogicalShare { id = 8 } + | └─LogicalProject { exprs: [id, null:Varchar, null:Varchar, initial_bid, null:Int32, null:Timestamp, null:Timestamp, null:Int32, null:Int32, null:Int64] } + | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + | └─LogicalProject { exprs: [id, initial_bid] } + | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + └─LogicalProject { exprs: [id] } + └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } + └─LogicalProject { exprs: [id, initial_bid] } + └─LogicalShare { id = 8 } + └─LogicalProject { exprs: [id, null:Varchar, null:Varchar, initial_bid, null:Int32, null:Timestamp, null:Timestamp, null:Int32, null:Int32, null:Int64] } + └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + └─LogicalProject { exprs: [id, initial_bid] } + └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + + Prune Columns (For DAG): + + LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } + └─LogicalAgg { aggs: [count] } + └─LogicalProject { exprs: [] } + └─LogicalJoin { type: Inner, on: (id = id) } + ├─LogicalProject { exprs: [id] } + | └─LogicalProject { exprs: [id] } + | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } + | └─LogicalProject { exprs: [id, initial_bid] } + | └─LogicalProject { exprs: [id, initial_bid] } + | └─LogicalShare { id = 91 } + | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + | └─LogicalProject { exprs: [id, initial_bid] } + | └─LogicalProject { exprs: [id, initial_bid] } + | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + └─LogicalProject { exprs: [id] } + └─LogicalProject { exprs: [id] } + └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } + └─LogicalProject { exprs: [id, initial_bid] } + └─LogicalProject { exprs: [id, initial_bid] } + └─LogicalShare { id = 91 } + └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + └─LogicalProject { exprs: [id, initial_bid] } + └─LogicalProject { exprs: [id, initial_bid] } + └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + + Predicate Push Down: + + LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } + └─LogicalAgg { aggs: [count] } + └─LogicalProject { exprs: [] } + └─LogicalJoin { type: Inner, on: (id = id) } + ├─LogicalProject { exprs: [id] } + | └─LogicalProject { exprs: [id] } + | └─LogicalProject { exprs: [id, initial_bid] } + | └─LogicalProject { exprs: [id, initial_bid] } + | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } + | └─LogicalShare { id = 91 } + | └─LogicalProject { exprs: [id, initial_bid] } + | └─LogicalProject { exprs: [id, initial_bid] } + | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + └─LogicalProject { exprs: [id] } + └─LogicalProject { exprs: [id] } + └─LogicalProject { exprs: [id, initial_bid] } + └─LogicalProject { exprs: [id, initial_bid] } + └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } + └─LogicalShare { id = 91 } + └─LogicalProject { exprs: [id, initial_bid] } + └─LogicalProject { exprs: [id, initial_bid] } + └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + + Project Remove: + + apply ProjectMergeRule 3 time(s) + apply ProjectEliminateRule 5 time(s) + + LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } + └─LogicalAgg { aggs: [count] } + └─LogicalJoin { type: Inner, on: (id = id) } + ├─LogicalProject { exprs: [id] } + | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } + | └─LogicalShare { id = 91 } + | └─LogicalProject { exprs: [id, initial_bid] } + | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + └─LogicalProject { exprs: [id] } + └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } + └─LogicalShare { id = 91 } + └─LogicalProject { exprs: [id, initial_bid] } + └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } + └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + + WTF2 + + + Node: Ok("LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] }\n└─LogicalAgg { aggs: [count] }\n └─LogicalJoin { type: Inner, on: (id = id) }\n ├─LogicalProject { exprs: [id] }\n | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n | └─LogicalShare { id = 91 }\n | └─LogicalProject { exprs: [id, initial_bid] }\n | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n └─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalProject { exprs: [(count = 3:Int32)] }\n└─LogicalAgg { aggs: [count] }\n └─LogicalJoin { type: Inner, on: (id = id) }\n ├─LogicalProject { exprs: [id] }\n | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n | └─LogicalShare { id = 91 }\n | └─LogicalProject { exprs: [id, initial_bid] }\n | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n └─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [Ok("LogicalAgg { aggs: [count] }\n└─LogicalJoin { type: Inner, on: (id = id) }\n ├─LogicalProject { exprs: [id] }\n | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n | └─LogicalShare { id = 91 }\n | └─LogicalProject { exprs: [id, initial_bid] }\n | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n └─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalAgg { aggs: [count] }\n└─LogicalJoin { type: Inner, on: (id = id) }\n ├─LogicalProject { exprs: [id] }\n | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n | └─LogicalShare { id = 91 }\n | └─LogicalProject { exprs: [id, initial_bid] }\n | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n └─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalAgg { aggs: [count] }\n└─LogicalJoin { type: Inner, on: (id = id) }\n ├─LogicalProject { exprs: [id] }\n | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n | └─LogicalShare { id = 91 }\n | └─LogicalProject { exprs: [id, initial_bid] }\n | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n └─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [Ok("LogicalJoin { type: Inner, on: (id = id) }\n├─LogicalProject { exprs: [id] }\n| └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n| └─LogicalShare { id = 91 }\n| └─LogicalProject { exprs: [id, initial_bid] }\n| └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n| └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n└─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalJoin { type: Inner, on: (id = id) }\n├─LogicalProject { exprs: [id] }\n| └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n| └─LogicalShare { id = 91 }\n| └─LogicalProject { exprs: [id, initial_bid] }\n| └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n| └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n└─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalJoin { type: Inner, on: (id = id) }\n├─LogicalProject { exprs: [id] }\n| └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n| └─LogicalShare { id = 91 }\n| └─LogicalProject { exprs: [id, initial_bid] }\n| └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n| └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n└─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n"), Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [Ok("LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalFilter { predicate: (initial_bid = 1:Int32) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [Ok("LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [] + + inputs after: [] + + inputs after: [Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + inputs after: [Ok("LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + inputs after: [Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + inputs after: [Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + inputs after: [Ok("LogicalFilter { predicate: (initial_bid = 1:Int32) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [Ok("LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalFilter { predicate: (initial_bid = 2:Int32) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [Ok("LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + WTF2 + + + Node: Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + Rewritten Node: Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") + + inputs before: [] + + inputs after: [] + + inputs after: [Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + inputs after: [Ok("LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + inputs after: [Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + inputs after: [Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + inputs after: [Ok("LogicalFilter { predicate: (initial_bid = 2:Int32) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + inputs after: [Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = 1:Int32) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n"), Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = 2:Int32) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + inputs after: [Ok("LogicalJoin { type: Inner, on: (id = id) }\n├─LogicalProject { exprs: [id] }\n| └─LogicalFilter { predicate: (initial_bid = 1:Int32) }\n| └─LogicalShare { id = 91 }\n| └─LogicalProject { exprs: [id, initial_bid] }\n| └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n| └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n└─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = 2:Int32) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + inputs after: [Ok("LogicalAgg { aggs: [count] }\n└─LogicalJoin { type: Inner, on: (id = id) }\n ├─LogicalProject { exprs: [id] }\n | └─LogicalFilter { predicate: (initial_bid = 1:Int32) }\n | └─LogicalShare { id = 91 }\n | └─LogicalProject { exprs: [id, initial_bid] }\n | └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n └─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = 2:Int32) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] + + Const eval exprs: + + LogicalProject { exprs: [(count = 3:Int32)] } + └─LogicalAgg { aggs: [count] } + └─LogicalJoin { type: Inner, on: (id = id) } + ├─LogicalProject { exprs: [id] } + | └─LogicalFilter { predicate: (initial_bid = 1:Int32) } + | └─LogicalShare { id = 91 } + | └─LogicalProject { exprs: [id, initial_bid] } + | └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } + | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + └─LogicalProject { exprs: [id] } + └─LogicalFilter { predicate: (initial_bid = 2:Int32) } + └─LogicalShare { id = 91 } + └─LogicalProject { exprs: [id, initial_bid] } + └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } + └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } + diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index 0d9224930095e..9ac3af9bf4ef2 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -128,7 +128,7 @@ LogicalProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32), RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32), RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32), count] } └─LogicalAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─LogicalProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount] } - └─LogicalScan { table: lineitem, output_columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus], required_columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate], predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + └─LogicalScan { table: lineitem, output_columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus], required_columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate], predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } batch_plan: | BatchExchange { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], dist: Single } └─BatchProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32), RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32), RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32), count] } @@ -136,7 +136,7 @@ └─BatchHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } └─BatchProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount] } - └─BatchFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } └─BatchScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], pk_columns: [l_returnflag, l_linestatus] } @@ -144,7 +144,7 @@ └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─StreamExchange { dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -157,7 +157,7 @@ Fragment 1 StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -717,7 +717,7 @@ optimized_logical_plan: | LogicalAgg { group_key: [orders.o_orderpriority], aggs: [count] } └─LogicalJoin { type: LeftSemi, on: (lineitem.l_orderkey = orders.o_orderkey), output: [orders.o_orderpriority] } - ├─LogicalScan { table: orders, output_columns: [orders.o_orderkey, orders.o_orderpriority], required_columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } + ├─LogicalScan { table: orders, output_columns: [orders.o_orderkey, orders.o_orderpriority], required_columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } └─LogicalScan { table: lineitem, output_columns: [lineitem.l_orderkey], required_columns: [lineitem.l_orderkey, lineitem.l_commitdate, lineitem.l_receiptdate], predicate: (lineitem.l_commitdate < lineitem.l_receiptdate) } batch_plan: | BatchExchange { order: [orders.o_orderpriority ASC], dist: Single } @@ -727,7 +727,7 @@ └─BatchHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority] } ├─BatchExchange { order: [], dist: HashShard(orders.o_orderkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey] } @@ -741,7 +741,7 @@ └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } ├─StreamExchange { dist: HashShard(orders.o_orderkey) } | └─StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } + | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } @@ -764,7 +764,7 @@ Fragment 2 StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } + StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -835,7 +835,7 @@ | | ├─LogicalJoin { type: Inner, on: (customer.c_nationkey = supplier.s_nationkey), output: [orders.o_orderkey, supplier.s_suppkey, supplier.s_nationkey] } | | | ├─LogicalJoin { type: Inner, on: (customer.c_custkey = orders.o_custkey), output: [customer.c_nationkey, orders.o_orderkey] } | | | | ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey] } - | | | | └─LogicalScan { table: orders, output_columns: [orders.o_orderkey, orders.o_custkey], required_columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } + | | | | └─LogicalScan { table: orders, output_columns: [orders.o_orderkey, orders.o_custkey], required_columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } | | | └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey] } | | └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount] } | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey] } @@ -859,7 +859,7 @@ | | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], distribution: UpstreamHashShard(customer.c_custkey) } | | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } + | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } | | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } | └─BatchExchange { order: [], dist: HashShard(supplier.s_nationkey) } | └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } @@ -884,7 +884,7 @@ | | | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } + | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } | | | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | | | └─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) } @@ -943,7 +943,7 @@ Fragment 7 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } + StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -1014,13 +1014,13 @@ optimized_logical_plan: | LogicalAgg { aggs: [sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─LogicalProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount)] } - └─LogicalScan { table: lineitem, output_columns: [lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_quantity, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } + └─LogicalScan { table: lineitem, output_columns: [lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_quantity, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } batch_plan: | BatchSimpleAgg { aggs: [sum(sum((lineitem.l_extendedprice * lineitem.l_discount)))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─BatchProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount)] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } └─BatchScan { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [revenue], pk_columns: [] } @@ -1029,7 +1029,7 @@ └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount), lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -1043,7 +1043,7 @@ Fragment 1 StreamStatelessLocalSimpleAgg { aggs: [count, sum((lineitem.l_extendedprice * lineitem.l_discount))] } StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount), lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } Chain { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -1330,9 +1330,9 @@ | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } optimized_logical_plan: | - LogicalProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - └─LogicalAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─LogicalProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + LogicalProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─LogicalAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─LogicalProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─LogicalJoin { type: Inner, on: (nation.n_regionkey = region.r_regionkey), output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name] } ├─LogicalJoin { type: Inner, on: (customer.c_nationkey = nation.n_nationkey), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey] } | ├─LogicalJoin { type: Inner, on: (orders.o_custkey = customer.c_custkey), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, customer.c_nationkey] } @@ -1350,11 +1350,11 @@ └─LogicalScan { table: region, output_columns: [region.r_regionkey], required_columns: [region.r_regionkey, region.r_name], predicate: (region.r_name = 'ASIA':Varchar) } batch_plan: | BatchExchange { order: [Extract('YEAR':Varchar, orders.o_orderdate) ASC], dist: Single } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } └─BatchSort { order: [Extract('YEAR':Varchar, orders.o_orderdate) ASC] } - └─BatchHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─BatchHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─BatchExchange { order: [], dist: HashShard(Extract('YEAR':Varchar, orders.o_orderdate)) } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'ASIA':Varchar), output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name] } └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } └─BatchLookupJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey] } @@ -1376,10 +1376,10 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year] } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - └─StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(Extract('YEAR':Varchar, orders.o_orderdate)) } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } ├─StreamExchange { dist: HashShard(nation.n_regionkey) } | └─StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, customer.c_nationkey, nation.n_nationkey] } @@ -1418,13 +1418,13 @@ Fragment 0 StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year] } materialized table: 4294967294 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([4]) from 2 @@ -1511,7 +1511,7 @@ Upstream BatchPlanNode - Table 0 { columns: [Extract('YEAR':Varchar, orders_o_orderdate), count, sum(Case((nation_n_name = 'IRAN':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Int32::Decimal)), sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)))], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [Extract('YEAR':Varchar, orders_o_orderdate), count, sum(Case((nation_n_name = 'IRAN':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Decimal)), sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)))], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_orderdate, nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, customer_c_nationkey, nation_n_nationkey_0], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 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: [4] } Table 2 { columns: [nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey_0, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [14], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC, $0 ASC], value indices: [0], distribution key: [0] } @@ -1812,7 +1812,7 @@ ├─LogicalJoin { type: Inner, on: (customer.c_nationkey = nation.n_nationkey), output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey, nation.n_name] } | ├─LogicalJoin { type: Inner, on: (customer.c_custkey = orders.o_custkey), output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey] } | | ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment] } - | | └─LogicalScan { table: orders, output_columns: [orders.o_orderkey, orders.o_custkey], required_columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } + | | └─LogicalScan { table: orders, output_columns: [orders.o_orderkey, orders.o_custkey], required_columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name] } └─LogicalScan { table: lineitem, output_columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_returnflag], predicate: (lineitem.l_returnflag = 'R':Varchar) } batch_plan: | @@ -1832,7 +1832,7 @@ | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], distribution: UpstreamHashShard(customer.c_custkey) } | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount] } @@ -1858,7 +1858,7 @@ | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } + | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | └─StreamExchange { dist: HashShard(nation.n_nationkey) } | └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } @@ -1910,7 +1910,7 @@ Fragment 6 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } + StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -2217,7 +2217,7 @@ └─LogicalProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32), Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32)] } └─LogicalJoin { type: Inner, on: (orders.o_orderkey = lineitem.l_orderkey), output: [orders.o_orderpriority, lineitem.l_shipmode] } ├─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority] } - └─LogicalScan { table: lineitem, output_columns: [lineitem.l_orderkey, lineitem.l_shipmode], required_columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } + └─LogicalScan { table: lineitem, output_columns: [lineitem.l_orderkey, lineitem.l_shipmode], required_columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } batch_plan: | BatchExchange { order: [lineitem.l_shipmode ASC], dist: Single } └─BatchSort { order: [lineitem.l_shipmode ASC] } @@ -2229,7 +2229,7 @@ | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode] } - └─BatchFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } + └─BatchFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], pk_columns: [l_shipmode] } @@ -2242,7 +2242,7 @@ | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } + └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -2267,7 +2267,7 @@ Fragment 3 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } + StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -2408,34 +2408,34 @@ ├─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } optimized_logical_plan: | - LogicalProject { exprs: [((100.00:Decimal * sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } - └─LogicalAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─LogicalProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + LogicalProject { exprs: [((100.00:Decimal * sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + └─LogicalAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─LogicalProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─LogicalJoin { type: Inner, on: (lineitem.l_partkey = part.p_partkey), output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type] } - ├─LogicalScan { table: lineitem, output_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } + ├─LogicalScan { table: lineitem, output_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } └─LogicalScan { table: part, columns: [part.p_partkey, part.p_type] } batch_plan: | - BatchProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - └─BatchSimpleAgg { aggs: [sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + BatchProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + └─BatchSimpleAgg { aggs: [sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + └─BatchSimpleAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type] } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_partkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [promo_revenue], pk_columns: [] } - └─StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + └─StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } └─StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } ├─StreamExchange { dist: HashShard(lineitem.l_partkey) } | └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } └─StreamExchange { dist: HashShard(part.p_partkey) } └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } @@ -2443,14 +2443,14 @@ Fragment 0 StreamMaterialize { columns: [promo_revenue], pk_columns: [] } materialized table: 4294967294 - StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -2458,7 +2458,7 @@ Fragment 2 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -2468,7 +2468,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum(Case(Like(part_p_type, 'PROMO%':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Int32::Decimal))), sum(sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount))))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [count, sum(sum(Case(Like(part_p_type, 'PROMO%':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Decimal))), sum(sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount))))], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 1 { columns: [lineitem_l_partkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 2 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 3 { columns: [part_p_partkey, part_p_type], primary key: [$0 ASC, $0 ASC], value indices: [0, 1], distribution key: [0] } @@ -2537,12 +2537,12 @@ | └─LogicalShare { id = 165 } | └─LogicalAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - | └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + | └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └─LogicalAgg { aggs: [max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─LogicalShare { id = 165 } └─LogicalAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } batch_plan: | BatchExchange { order: [supplier.s_suppkey ASC], dist: Single } └─BatchSort { order: [supplier.s_suppkey ASC] } @@ -2554,7 +2554,7 @@ | └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } | └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } | └─BatchScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } └─BatchExchange { order: [], dist: HashShard(max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))) } └─BatchSimpleAgg { aggs: [max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2563,7 +2563,7 @@ └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └─BatchScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2573,12 +2573,12 @@ | ├─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) } | └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - | └─StreamShare { id = 900 } + | └─StreamShare { id = 921 } | └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber] } - | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } └─StreamExchange { dist: HashShard(max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))) } └─StreamProject { exprs: [max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2587,12 +2587,12 @@ └─StreamHashAgg { group_key: [Vnode(lineitem.l_suppkey)], aggs: [count, max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), Vnode(lineitem.l_suppkey)] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamShare { id = 900 } + └─StreamShare { id = 921 } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -2623,7 +2623,7 @@ Fragment 4 StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -3343,7 +3343,7 @@ └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(partsupp.ps_partkey, lineitem.l_partkey) AND IsNotDistinctFrom(partsupp.ps_suppkey, lineitem.l_suppkey), output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity] } ├─LogicalAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [] } | └─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey] } - └─LogicalScan { table: lineitem, output_columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity], required_columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } + └─LogicalScan { table: lineitem, output_columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity], required_columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } batch_plan: | BatchExchange { order: [supplier.s_name ASC], dist: Single } └─BatchSort { order: [supplier.s_name ASC] } @@ -3369,7 +3369,7 @@ | └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity] } - └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } + └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], pk_columns: [supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], order_descs: [s_name, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } @@ -3404,7 +3404,7 @@ | └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─StreamExchange { dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } + └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -3476,7 +3476,7 @@ Fragment 9 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } + StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode From da3e2076409bbdab886d9db87b176e81cd4f7d6c Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Sat, 28 Jan 2023 17:12:51 +0800 Subject: [PATCH 08/22] minor --- .../planner_test/tests/testdata/share.yaml | 386 ------------------ 1 file changed, 386 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index 328f922dc4268..77a167e2ba390 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -20,21 +20,6 @@ - create_sources sql: | select count(*) cnt from auction A join auction B on A.id = B.id where A.initial_bid = 1 and B.initial_bid = 2; - optimized_logical_plan: | - LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: (id = id), output: [] } - ├─LogicalProject { exprs: [id] } - | └─LogicalFilter { predicate: (initial_bid = 1:Int32) } - | └─LogicalShare { id = 91 } - | └─LogicalProject { exprs: [id, initial_bid] } - | └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } - | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - └─LogicalProject { exprs: [id] } - └─LogicalFilter { predicate: (initial_bid = 2:Int32) } - └─LogicalShare { id = 91 } - └─LogicalProject { exprs: [id, initial_bid] } - └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } - └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } batch_plan: | BatchSimpleAgg { aggs: [sum0(count)] } └─BatchExchange { order: [], dist: Single } @@ -156,374 +141,3 @@ └─StreamProject { exprs: [auction, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 4 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } -- id: self_join_13 - before: - - create_sources - sql: | - explain (trace, logical) select count(*) = (1 + 2) cnt from auction A join auction B on A.id = B.id where A.initial_bid = 2-1 and B.initial_bid = 5-3; - explain_output: |+ - Begin: - - LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } - └─LogicalAgg { aggs: [count] } - └─LogicalProject { exprs: [] } - └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) AND (initial_bid = (5:Int32 - 3:Int32)) } - └─LogicalJoin { type: Inner, on: (id = id) } - ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - - Share Source: - - LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } - └─LogicalAgg { aggs: [count] } - └─LogicalProject { exprs: [] } - └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) AND (initial_bid = (5:Int32 - 3:Int32)) } - └─LogicalJoin { type: Inner, on: (id = id) } - ├─LogicalShare { id = 8 } - | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - └─LogicalShare { id = 8 } - └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - - Predicate Push Down: - - LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } - └─LogicalAgg { aggs: [count] } - └─LogicalProject { exprs: [] } - └─LogicalJoin { type: Inner, on: (id = id) } - ├─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } - | └─LogicalShare { id = 8 } - | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } - └─LogicalShare { id = 8 } - └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - - Predicate Push Down: - - LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } - └─LogicalAgg { aggs: [count] } - └─LogicalProject { exprs: [] } - └─LogicalJoin { type: Inner, on: (id = id) } - ├─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } - | └─LogicalShare { id = 8 } - | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } - └─LogicalShare { id = 8 } - └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - - Predicate Push Down: - - LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } - └─LogicalAgg { aggs: [count] } - └─LogicalProject { exprs: [] } - └─LogicalJoin { type: Inner, on: (id = id) } - ├─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } - | └─LogicalShare { id = 8 } - | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } - └─LogicalShare { id = 8 } - └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - - Push Down the Calculation of Inputs of Join's Condition: - - apply PushCalculationOfJoinRule 1 time(s) - - LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } - └─LogicalAgg { aggs: [count] } - └─LogicalProject { exprs: [] } - └─LogicalJoin { type: Inner, on: (id = id) } - ├─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } - | └─LogicalShare { id = 8 } - | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } - └─LogicalShare { id = 8 } - └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - - Prune Columns: - - LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } - └─LogicalAgg { aggs: [count] } - └─LogicalProject { exprs: [] } - └─LogicalJoin { type: Inner, on: (id = id) } - ├─LogicalProject { exprs: [id] } - | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } - | └─LogicalProject { exprs: [id, initial_bid] } - | └─LogicalShare { id = 8 } - | └─LogicalProject { exprs: [id, null:Varchar, null:Varchar, initial_bid, null:Int32, null:Timestamp, null:Timestamp, null:Int32, null:Int32, null:Int64] } - | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - | └─LogicalProject { exprs: [id, initial_bid] } - | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - └─LogicalProject { exprs: [id] } - └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } - └─LogicalProject { exprs: [id, initial_bid] } - └─LogicalShare { id = 8 } - └─LogicalProject { exprs: [id, null:Varchar, null:Varchar, initial_bid, null:Int32, null:Timestamp, null:Timestamp, null:Int32, null:Int32, null:Int64] } - └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - └─LogicalProject { exprs: [id, initial_bid] } - └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - - Prune Columns (For DAG): - - LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } - └─LogicalAgg { aggs: [count] } - └─LogicalProject { exprs: [] } - └─LogicalJoin { type: Inner, on: (id = id) } - ├─LogicalProject { exprs: [id] } - | └─LogicalProject { exprs: [id] } - | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } - | └─LogicalProject { exprs: [id, initial_bid] } - | └─LogicalProject { exprs: [id, initial_bid] } - | └─LogicalShare { id = 91 } - | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - | └─LogicalProject { exprs: [id, initial_bid] } - | └─LogicalProject { exprs: [id, initial_bid] } - | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - └─LogicalProject { exprs: [id] } - └─LogicalProject { exprs: [id] } - └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } - └─LogicalProject { exprs: [id, initial_bid] } - └─LogicalProject { exprs: [id, initial_bid] } - └─LogicalShare { id = 91 } - └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - └─LogicalProject { exprs: [id, initial_bid] } - └─LogicalProject { exprs: [id, initial_bid] } - └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - - Predicate Push Down: - - LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } - └─LogicalAgg { aggs: [count] } - └─LogicalProject { exprs: [] } - └─LogicalJoin { type: Inner, on: (id = id) } - ├─LogicalProject { exprs: [id] } - | └─LogicalProject { exprs: [id] } - | └─LogicalProject { exprs: [id, initial_bid] } - | └─LogicalProject { exprs: [id, initial_bid] } - | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } - | └─LogicalShare { id = 91 } - | └─LogicalProject { exprs: [id, initial_bid] } - | └─LogicalProject { exprs: [id, initial_bid] } - | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - └─LogicalProject { exprs: [id] } - └─LogicalProject { exprs: [id] } - └─LogicalProject { exprs: [id, initial_bid] } - └─LogicalProject { exprs: [id, initial_bid] } - └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } - └─LogicalShare { id = 91 } - └─LogicalProject { exprs: [id, initial_bid] } - └─LogicalProject { exprs: [id, initial_bid] } - └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - - Project Remove: - - apply ProjectMergeRule 3 time(s) - apply ProjectEliminateRule 5 time(s) - - LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] } - └─LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: (id = id) } - ├─LogicalProject { exprs: [id] } - | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) } - | └─LogicalShare { id = 91 } - | └─LogicalProject { exprs: [id, initial_bid] } - | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - └─LogicalProject { exprs: [id] } - └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) } - └─LogicalShare { id = 91 } - └─LogicalProject { exprs: [id, initial_bid] } - └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) } - └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - - WTF2 - - - Node: Ok("LogicalProject { exprs: [(count = (1:Int32 + 2:Int32))] }\n└─LogicalAgg { aggs: [count] }\n └─LogicalJoin { type: Inner, on: (id = id) }\n ├─LogicalProject { exprs: [id] }\n | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n | └─LogicalShare { id = 91 }\n | └─LogicalProject { exprs: [id, initial_bid] }\n | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n └─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalProject { exprs: [(count = 3:Int32)] }\n└─LogicalAgg { aggs: [count] }\n └─LogicalJoin { type: Inner, on: (id = id) }\n ├─LogicalProject { exprs: [id] }\n | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n | └─LogicalShare { id = 91 }\n | └─LogicalProject { exprs: [id, initial_bid] }\n | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n └─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [Ok("LogicalAgg { aggs: [count] }\n└─LogicalJoin { type: Inner, on: (id = id) }\n ├─LogicalProject { exprs: [id] }\n | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n | └─LogicalShare { id = 91 }\n | └─LogicalProject { exprs: [id, initial_bid] }\n | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n └─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalAgg { aggs: [count] }\n└─LogicalJoin { type: Inner, on: (id = id) }\n ├─LogicalProject { exprs: [id] }\n | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n | └─LogicalShare { id = 91 }\n | └─LogicalProject { exprs: [id, initial_bid] }\n | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n └─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalAgg { aggs: [count] }\n└─LogicalJoin { type: Inner, on: (id = id) }\n ├─LogicalProject { exprs: [id] }\n | └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n | └─LogicalShare { id = 91 }\n | └─LogicalProject { exprs: [id, initial_bid] }\n | └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n └─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [Ok("LogicalJoin { type: Inner, on: (id = id) }\n├─LogicalProject { exprs: [id] }\n| └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n| └─LogicalShare { id = 91 }\n| └─LogicalProject { exprs: [id, initial_bid] }\n| └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n| └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n└─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalJoin { type: Inner, on: (id = id) }\n├─LogicalProject { exprs: [id] }\n| └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n| └─LogicalShare { id = 91 }\n| └─LogicalProject { exprs: [id, initial_bid] }\n| └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n| └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n└─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalJoin { type: Inner, on: (id = id) }\n├─LogicalProject { exprs: [id] }\n| └─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n| └─LogicalShare { id = 91 }\n| └─LogicalProject { exprs: [id, initial_bid] }\n| └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n| └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n└─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n"), Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [Ok("LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalFilter { predicate: (initial_bid = (2:Int32 - 1:Int32)) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalFilter { predicate: (initial_bid = 1:Int32) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [Ok("LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [] - - inputs after: [] - - inputs after: [Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - inputs after: [Ok("LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - inputs after: [Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - inputs after: [Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - inputs after: [Ok("LogicalFilter { predicate: (initial_bid = 1:Int32) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [Ok("LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalFilter { predicate: (initial_bid = (5:Int32 - 3:Int32)) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalFilter { predicate: (initial_bid = 2:Int32) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [Ok("LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalFilter { predicate: ((initial_bid = (2:Int32 - 1:Int32)) OR (initial_bid = (5:Int32 - 3:Int32))) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - WTF2 - - - Node: Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - Rewritten Node: Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n") - - inputs before: [] - - inputs after: [] - - inputs after: [Ok("LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - inputs after: [Ok("LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n└─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - inputs after: [Ok("LogicalProject { exprs: [id, initial_bid] }\n└─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - inputs after: [Ok("LogicalShare { id = 91 }\n└─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - inputs after: [Ok("LogicalFilter { predicate: (initial_bid = 2:Int32) }\n└─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - inputs after: [Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = 1:Int32) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n"), Ok("LogicalProject { exprs: [id] }\n└─LogicalFilter { predicate: (initial_bid = 2:Int32) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - inputs after: [Ok("LogicalJoin { type: Inner, on: (id = id) }\n├─LogicalProject { exprs: [id] }\n| └─LogicalFilter { predicate: (initial_bid = 1:Int32) }\n| └─LogicalShare { id = 91 }\n| └─LogicalProject { exprs: [id, initial_bid] }\n| └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n| └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n└─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = 2:Int32) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - inputs after: [Ok("LogicalAgg { aggs: [count] }\n└─LogicalJoin { type: Inner, on: (id = id) }\n ├─LogicalProject { exprs: [id] }\n | └─LogicalFilter { predicate: (initial_bid = 1:Int32) }\n | └─LogicalShare { id = 91 }\n | └─LogicalProject { exprs: [id, initial_bid] }\n | └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n └─LogicalProject { exprs: [id] }\n └─LogicalFilter { predicate: (initial_bid = 2:Int32) }\n └─LogicalShare { id = 91 }\n └─LogicalProject { exprs: [id, initial_bid] }\n └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }\n └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] }\n")] - - Const eval exprs: - - LogicalProject { exprs: [(count = 3:Int32)] } - └─LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: (id = id) } - ├─LogicalProject { exprs: [id] } - | └─LogicalFilter { predicate: (initial_bid = 1:Int32) } - | └─LogicalShare { id = 91 } - | └─LogicalProject { exprs: [id, initial_bid] } - | └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } - | └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - └─LogicalProject { exprs: [id] } - └─LogicalFilter { predicate: (initial_bid = 2:Int32) } - └─LogicalShare { id = 91 } - └─LogicalProject { exprs: [id, initial_bid] } - └─LogicalFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } - └─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id], time_range: [(Unbounded, Unbounded)] } - From 4e27702e349f8bd854deb2d5dc53d605b660689d Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Sat, 28 Jan 2023 17:14:16 +0800 Subject: [PATCH 09/22] minor --- .../planner_test/tests/testdata/explain.yaml | 55 ------------------- .../planner_test/tests/testdata/share.yaml | 4 +- 2 files changed, 2 insertions(+), 57 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index 365ecbf79256e..eaab5e455a22f 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -36,28 +36,6 @@ LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - WTF2 - - - Node: Ok("LogicalProject { exprs: [1:Int32] }\n└─LogicalValues { rows: [[]], schema: Schema { fields: [] } }\n") - - Rewritten Node: Ok("LogicalProject { exprs: [1:Int32] }\n└─LogicalValues { rows: [[]], schema: Schema { fields: [] } }\n") - - inputs before: [Ok("LogicalValues { rows: [[]], schema: Schema { fields: [] } }\n")] - - WTF2 - - - Node: Ok("LogicalValues { rows: [[]], schema: Schema { fields: [] } }\n") - - Rewritten Node: Ok("LogicalValues { rows: [[]], schema: Schema { fields: [] } }\n") - - inputs before: [] - - inputs after: [] - - inputs after: [Ok("LogicalValues { rows: [[]], schema: Schema { fields: [] } }\n")] - Const eval exprs: LogicalProject { exprs: [1:Int32] } @@ -192,39 +170,6 @@ ├─LogicalScan { table: t1, columns: [v1] } └─LogicalScan { table: t2, columns: [v2] } - WTF2 - - - Node: Ok("LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) }\n├─LogicalScan { table: t1, columns: [v1] }\n└─LogicalScan { table: t2, columns: [v2] }\n") - - Rewritten Node: Ok("LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) }\n├─LogicalScan { table: t1, columns: [v1] }\n└─LogicalScan { table: t2, columns: [v2] }\n") - - inputs before: [Ok("LogicalScan { table: t1, columns: [v1] }\n"), Ok("LogicalScan { table: t2, columns: [v2] }\n")] - - WTF2 - - - Node: Ok("LogicalScan { table: t1, columns: [v1] }\n") - - Rewritten Node: Ok("LogicalScan { table: t1, columns: [v1] }\n") - - inputs before: [] - - inputs after: [] - - WTF2 - - - Node: Ok("LogicalScan { table: t2, columns: [v2] }\n") - - Rewritten Node: Ok("LogicalScan { table: t2, columns: [v2] }\n") - - inputs before: [] - - inputs after: [] - - inputs after: [Ok("LogicalScan { table: t1, columns: [v1] }\n"), Ok("LogicalScan { table: t2, columns: [v2] }\n")] - Const eval exprs: LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index 77a167e2ba390..4c42b81e71c31 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -48,7 +48,7 @@ | └─StreamProject { exprs: [id, _row_id] } | └─StreamFilter { predicate: (initial_bid = 1:Int32) } | └─StreamProject { exprs: [id, initial_bid, _row_id] } - | └─StreamShare { id = 757 } + | └─StreamShare { id = 539 } | └─StreamProject { exprs: [id, initial_bid, _row_id] } | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } | └─StreamRowIdGen { row_id_index: 9 } @@ -57,7 +57,7 @@ └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } └─StreamProject { exprs: [id, initial_bid, _row_id] } - └─StreamShare { id = 757 } + └─StreamShare { id = 539 } └─StreamProject { exprs: [id, initial_bid, _row_id] } └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 9 } From c384e3e7a1fbf6e5ca1cc0ed70e992747e847199 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Sat, 28 Jan 2023 21:31:35 +0800 Subject: [PATCH 10/22] always rewrite with new planid --- src/frontend/planner_test/tests/testdata/explain.yaml | 4 ++-- src/frontend/planner_test/tests/testdata/share.yaml | 8 ++++---- src/frontend/planner_test/tests/testdata/tpch.yaml | 4 ++-- src/frontend/src/optimizer/plan_node/logical_agg.rs | 10 +++++++--- src/frontend/src/optimizer/plan_node/logical_apply.rs | 1 + src/frontend/src/optimizer/plan_node/logical_filter.rs | 10 +++++++--- src/frontend/src/optimizer/plan_node/logical_join.rs | 10 +++++++--- .../src/optimizer/plan_node/logical_project.rs | 10 +++++++--- .../src/optimizer/plan_node/logical_project_set.rs | 10 +++++++--- src/frontend/src/optimizer/plan_node/logical_scan.rs | 10 +++++++--- .../src/optimizer/plan_node/logical_table_function.rs | 1 + src/frontend/src/optimizer/plan_node/logical_update.rs | 1 + src/frontend/src/optimizer/plan_node/logical_values.rs | 1 + src/frontend/src/optimizer/plan_node/plan_base.rs | 6 ++++++ 14 files changed, 60 insertions(+), 26 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index eaab5e455a22f..b94fcc977dea4 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -56,7 +56,7 @@ "stages": { "0": { "root": { - "plan_node_id": 29, + "plan_node_id": 31, "plan_node_type": "BatchProject", "schema": [ { @@ -69,7 +69,7 @@ ], "children": [ { - "plan_node_id": 27, + "plan_node_id": 29, "plan_node_type": "BatchValues", "schema": [], "children": [], diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index 4c42b81e71c31..23e8555256d33 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -48,7 +48,7 @@ | └─StreamProject { exprs: [id, _row_id] } | └─StreamFilter { predicate: (initial_bid = 1:Int32) } | └─StreamProject { exprs: [id, initial_bid, _row_id] } - | └─StreamShare { id = 539 } + | └─StreamShare { id = 559 } | └─StreamProject { exprs: [id, initial_bid, _row_id] } | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } | └─StreamRowIdGen { row_id_index: 9 } @@ -57,7 +57,7 @@ └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } └─StreamProject { exprs: [id, initial_bid, _row_id] } - └─StreamShare { id = 539 } + └─StreamShare { id = 559 } └─StreamProject { exprs: [id, initial_bid, _row_id] } └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 9 } @@ -125,7 +125,7 @@ | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } | └─StreamProject { exprs: [auction, date_time, _row_id] } - | └─StreamShare { id = 764 } + | └─StreamShare { id = 780 } | └─StreamProject { exprs: [auction, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 4 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } @@ -137,7 +137,7 @@ └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } └─StreamProject { exprs: [auction, date_time, _row_id] } - └─StreamShare { id = 764 } + └─StreamShare { id = 780 } └─StreamProject { exprs: [auction, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 4 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index 9ac3af9bf4ef2..6525a6cce78bf 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -2573,7 +2573,7 @@ | ├─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) } | └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - | └─StreamShare { id = 921 } + | └─StreamShare { id = 951 } | └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } @@ -2587,7 +2587,7 @@ └─StreamHashAgg { group_key: [Vnode(lineitem.l_suppkey)], aggs: [count, max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), Vnode(lineitem.l_suppkey)] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamShare { id = 921 } + └─StreamShare { id = 951 } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index b814fbd2bde31..fadc1e155893a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -790,9 +790,13 @@ impl fmt::Display for LogicalAgg { impl ExprRewritable for LogicalAgg { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - let mut new = self.clone(); - new.core.rewrite_exprs(r); - new.into() + let mut core = self.core.clone(); + core.rewrite_exprs(r); + Self { + base: self.base.clone_with_new_plan_id(), + core, + } + .into() } } diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index 8cf261a935fb6..d37427bef9869 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -299,6 +299,7 @@ impl ExprRewritable for LogicalApply { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut new = self.clone(); new.on = new.on.rewrite_expr(r); + new.base = new.base.clone_with_new_plan_id(); new.into() } } diff --git a/src/frontend/src/optimizer/plan_node/logical_filter.rs b/src/frontend/src/optimizer/plan_node/logical_filter.rs index eabea1f499fbe..330afbe96c5c6 100644 --- a/src/frontend/src/optimizer/plan_node/logical_filter.rs +++ b/src/frontend/src/optimizer/plan_node/logical_filter.rs @@ -177,9 +177,13 @@ impl ColPrunable for LogicalFilter { impl ExprRewritable for LogicalFilter { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - let mut new = self.clone(); - new.core.rewrite_exprs(r); - new.into() + let mut core = self.core.clone(); + core.rewrite_exprs(r); + Self { + base: self.base.clone_with_new_plan_id(), + core, + } + .into() } } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index aca13e469bd9d..9cf84b5bc7861 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -787,9 +787,13 @@ impl ColPrunable for LogicalJoin { impl ExprRewritable for LogicalJoin { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - let mut new = self.clone(); - new.core.rewrite_exprs(r); - new.into() + let mut core = self.core.clone(); + core.rewrite_exprs(r); + Self { + base: self.base.clone_with_new_plan_id(), + core, + } + .into() } } diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index f71343fc6c225..c8bfe7ddbca54 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -206,9 +206,13 @@ impl ColPrunable for LogicalProject { impl ExprRewritable for LogicalProject { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - let mut new = self.clone(); - new.core.rewrite_exprs(r); - new.into() + let mut core = self.core.clone(); + core.rewrite_exprs(r); + Self { + base: self.base.clone_with_new_plan_id(), + core, + } + .into() } } 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 0c84b65c798e9..96ed5b08ff2e1 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project_set.rs @@ -259,9 +259,13 @@ impl ColPrunable for LogicalProjectSet { impl ExprRewritable for LogicalProjectSet { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - let mut new = self.clone(); - new.core.rewrite_exprs(r); - new.into() + let mut core = self.core.clone(); + core.rewrite_exprs(r); + Self { + base: self.base.clone_with_new_plan_id(), + core, + } + .into() } } diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index b35fcb3982572..506c46c97e457 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -452,9 +452,13 @@ impl ColPrunable for LogicalScan { impl ExprRewritable for LogicalScan { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - let mut new = self.clone(); - new.core.rewrite_exprs(r); - new.into() + let mut core = self.core.clone(); + core.rewrite_exprs(r); + Self { + base: self.base.clone_with_new_plan_id(), + core, + } + .into() } } 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 6235e31cc52f2..8ce1469e517cb 100644 --- a/src/frontend/src/optimizer/plan_node/logical_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/logical_table_function.rs @@ -80,6 +80,7 @@ impl ExprRewritable for LogicalTableFunction { .into_iter() .map(|e| r.rewrite_expr(e)) .collect(); + new.base = self.base.clone_with_new_plan_id(); new.into() } } diff --git a/src/frontend/src/optimizer/plan_node/logical_update.rs b/src/frontend/src/optimizer/plan_node/logical_update.rs index 3d0d1fcd2ef96..7badd3535bffd 100644 --- a/src/frontend/src/optimizer/plan_node/logical_update.rs +++ b/src/frontend/src/optimizer/plan_node/logical_update.rs @@ -139,6 +139,7 @@ impl ExprRewritable for LogicalUpdate { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut new = self.clone(); new.exprs = new.exprs.into_iter().map(|e| r.rewrite_expr(e)).collect(); + new.base = new.base.clone_with_new_plan_id(); new.into() } } diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index 2806f1c992962..da374c331dbda 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -90,6 +90,7 @@ impl ExprRewritable for LogicalValues { }) .collect::>() .into(); + new.base = new.base.clone_with_new_plan_id(); new.into() } } diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index 6bd99e9892f6b..07680818352f3 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -149,6 +149,12 @@ impl PlanBase { plan_node.watermark_columns().clone(), ) } + + pub fn clone_with_new_plan_id(&self) -> Self { + let mut new = self.clone(); + new.id = self.ctx.next_plan_node_id(); + new + } } macro_rules! impl_base_delegate { From d72b63e66364e429beb7b7d8ea0b13d7c47e56d5 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Mon, 30 Jan 2023 02:30:10 +0800 Subject: [PATCH 11/22] clone only if expr_rewritable --- .../planner_test/tests/testdata/expr.yaml | 2 +- .../tests/testdata/shared_views.yaml | 7 ++--- src/frontend/src/optimizer/mod.rs | 1 + .../optimizer/plan_node/expr_rewritable.rs | 24 ++++++++++++----- .../src/optimizer/plan_node/logical_agg.rs | 4 +++ .../src/optimizer/plan_node/logical_apply.rs | 4 +++ .../src/optimizer/plan_node/logical_delete.rs | 7 +---- .../src/optimizer/plan_node/logical_expand.rs | 7 +---- .../src/optimizer/plan_node/logical_filter.rs | 4 +++ .../optimizer/plan_node/logical_hop_window.rs | 8 ++---- .../src/optimizer/plan_node/logical_insert.rs | 7 +---- .../src/optimizer/plan_node/logical_join.rs | 4 +++ .../src/optimizer/plan_node/logical_limit.rs | 7 +---- .../src/optimizer/plan_node/logical_now.rs | 7 +---- .../optimizer/plan_node/logical_over_agg.rs | 10 ++----- .../optimizer/plan_node/logical_project.rs | 4 +++ .../plan_node/logical_project_set.rs | 4 +++ .../src/optimizer/plan_node/logical_scan.rs | 4 +++ .../src/optimizer/plan_node/logical_share.rs | 7 +---- .../src/optimizer/plan_node/logical_source.rs | 8 ++---- .../plan_node/logical_table_function.rs | 4 +++ .../src/optimizer/plan_node/logical_topn.rs | 8 ++---- .../src/optimizer/plan_node/logical_union.rs | 8 ++---- .../src/optimizer/plan_node/logical_update.rs | 4 +++ .../src/optimizer/plan_node/logical_values.rs | 4 +++ src/frontend/src/optimizer/plan_node/mod.rs | 26 ++++++++++++------- 26 files changed, 95 insertions(+), 89 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index bef3a8fc723f4..8b0ef58d40501 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -541,4 +541,4 @@ - name: const_eval of division by 0 error sql: | select 1 / 0 t1; - stream_error: 'Expr error: Division by zero' \ No newline at end of file + stream_error: 'Expr error: Division by zero' diff --git a/src/frontend/planner_test/tests/testdata/shared_views.yaml b/src/frontend/planner_test/tests/testdata/shared_views.yaml index 0191bb22385db..9e0f06e80b7e2 100644 --- a/src/frontend/planner_test/tests/testdata/shared_views.yaml +++ b/src/frontend/planner_test/tests/testdata/shared_views.yaml @@ -1,3 +1,4 @@ +# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - sql: | create table t1 (x int, y int); create view v1 as select x + y as z from t1 where y > 0; @@ -25,7 +26,7 @@ └─StreamHashJoin { type: Inner, predicate: (t1.x + t1.y) = (t1.x * (t1.x + t1.y)), output: [(t1.x + t1.y), (t1.x * (t1.x + t1.y)), (t1.y * (t1.x + t1.y)), t1._row_id, t1._row_id, t1._row_id, t1.x, (t1.x + t1.y)] } ├─StreamExchange { dist: HashShard((t1.x + t1.y)) } | └─StreamProject { exprs: [(t1.x + t1.y), t1._row_id] } - | └─StreamShare { id = 207 } + | └─StreamShare { id = 220 } | └─StreamProject { exprs: [(t1.x + t1.y), 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) } @@ -36,7 +37,7 @@ | └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard((t1.x + t1.y)) } └─StreamProject { exprs: [(t1.x + t1.y), t1._row_id] } - └─StreamShare { id = 207 } + └─StreamShare { id = 220 } └─StreamProject { exprs: [(t1.x + t1.y), 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) } \ No newline at end of file + └─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/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 418d97066bf75..e90962dfee811 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -51,6 +51,7 @@ use crate::catalog::column_catalog::ColumnCatalog; use crate::catalog::table_catalog::{TableType, TableVersion}; use crate::optimizer::plan_node::{ BatchExchange, ColumnPruningContext, PlanNodeType, PlanTreeNode, PredicatePushdownContext, + RewriteExprsRecursive, }; use crate::optimizer::property::Distribution; use crate::utils::Condition; diff --git a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs index 9d04c163a607c..e9c7e3eb64d79 100644 --- a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs +++ b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs @@ -24,23 +24,33 @@ use crate::{for_batch_plan_nodes, for_stream_plan_nodes}; /// the `ExprRewriter` needs to be idempotent i.e. applying it more than once /// to the same `ExprImpl` will be a noop on subsequent applications. pub trait ExprRewritable { - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef; + fn has_rewritable_expr(&self) -> bool { + false + } + + fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { + unimplemented!("{}", self.has_rewritable_expr()) + } } impl ExprRewritable for PlanRef { + fn has_rewritable_expr(&self) -> bool { + true + } + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - self.deref().rewrite_exprs(r) + if self.deref().has_rewritable_expr() { + self.deref().rewrite_exprs(r) + } else { + self.clone() + } } } macro_rules! ban_expr_rewritable { ($( { $convention:ident, $name:ident }),*) => { paste!{ - $(impl ExprRewritable for [<$convention $name>] { - fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - unimplemented!() - } - })* + $(impl ExprRewritable for [<$convention $name>] {} )* } } } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index eb9ba67d9bba2..c75070280eb3f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -789,6 +789,10 @@ impl fmt::Display for LogicalAgg { } impl ExprRewritable for LogicalAgg { + fn has_rewritable_expr(&self) -> bool { + true + } + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut core = self.core.clone(); core.rewrite_exprs(r); diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index 118487d5d6f14..4ec4aac229f99 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -296,6 +296,10 @@ impl ColPrunable for LogicalApply { } impl ExprRewritable for LogicalApply { + fn has_rewritable_expr(&self) -> bool { + true + } + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut new = self.clone(); new.on = new.on.rewrite_expr(r); diff --git a/src/frontend/src/optimizer/plan_node/logical_delete.rs b/src/frontend/src/optimizer/plan_node/logical_delete.rs index eedf2e2a3c900..d69cc6c867c04 100644 --- a/src/frontend/src/optimizer/plan_node/logical_delete.rs +++ b/src/frontend/src/optimizer/plan_node/logical_delete.rs @@ -23,7 +23,6 @@ use super::{ PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; -use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -127,11 +126,7 @@ impl ColPrunable for LogicalDelete { } } -impl ExprRewritable for LogicalDelete { - fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - self.clone().into() - } -} +impl ExprRewritable for LogicalDelete {} impl PredicatePushdown for LogicalDelete { fn predicate_pushdown( diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index 39fa5fd617e57..c314acdac6293 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -23,7 +23,6 @@ use super::{ gen_filter_and_pushdown, generic, BatchExpand, ColPrunable, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamExpand, ToBatch, ToStream, }; -use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -158,11 +157,7 @@ impl ColPrunable for LogicalExpand { } } -impl ExprRewritable for LogicalExpand { - fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - self.clone().into() - } -} +impl ExprRewritable for LogicalExpand {} impl PredicatePushdown for LogicalExpand { fn predicate_pushdown( diff --git a/src/frontend/src/optimizer/plan_node/logical_filter.rs b/src/frontend/src/optimizer/plan_node/logical_filter.rs index c9085af5b6341..aecb974010e17 100644 --- a/src/frontend/src/optimizer/plan_node/logical_filter.rs +++ b/src/frontend/src/optimizer/plan_node/logical_filter.rs @@ -176,6 +176,10 @@ impl ColPrunable for LogicalFilter { } impl ExprRewritable for LogicalFilter { + fn has_rewritable_expr(&self) -> bool { + true + } + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut core = self.core.clone(); core.rewrite_exprs(r); 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 b862c0933171b..7c7e838bb0451 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -25,7 +25,7 @@ use super::{ gen_filter_and_pushdown, generic, BatchHopWindow, ColPrunable, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, ToBatch, ToStream, }; -use crate::expr::{ExprRewriter, InputRef}; +use crate::expr::InputRef; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -327,11 +327,7 @@ impl ColPrunable for LogicalHopWindow { } } -impl ExprRewritable for LogicalHopWindow { - fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - self.clone().into() - } -} +impl ExprRewritable for LogicalHopWindow {} impl PredicatePushdown for LogicalHopWindow { /// Keep predicate on time window parameters (`window_start`, `window_end`), diff --git a/src/frontend/src/optimizer/plan_node/logical_insert.rs b/src/frontend/src/optimizer/plan_node/logical_insert.rs index e6d29a02950d3..e18f7b7518049 100644 --- a/src/frontend/src/optimizer/plan_node/logical_insert.rs +++ b/src/frontend/src/optimizer/plan_node/logical_insert.rs @@ -23,7 +23,6 @@ use super::{ PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; -use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -161,11 +160,7 @@ impl ColPrunable for LogicalInsert { } } -impl ExprRewritable for LogicalInsert { - fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - self.clone().into() - } -} +impl ExprRewritable for LogicalInsert {} impl PredicatePushdown for LogicalInsert { fn predicate_pushdown( diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index cf6d45d253f72..f55bc0f96d6b8 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -786,6 +786,10 @@ impl ColPrunable for LogicalJoin { } impl ExprRewritable for LogicalJoin { + fn has_rewritable_expr(&self) -> bool { + true + } + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut core = self.core.clone(); core.rewrite_exprs(r); diff --git a/src/frontend/src/optimizer/plan_node/logical_limit.rs b/src/frontend/src/optimizer/plan_node/logical_limit.rs index 74b6a124a9717..3be0c575d02bd 100644 --- a/src/frontend/src/optimizer/plan_node/logical_limit.rs +++ b/src/frontend/src/optimizer/plan_node/logical_limit.rs @@ -20,7 +20,6 @@ use super::{ gen_filter_and_pushdown, BatchLimit, ColPrunable, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; -use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -100,11 +99,7 @@ impl ColPrunable for LogicalLimit { } } -impl ExprRewritable for LogicalLimit { - fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - self.clone().into() - } -} +impl ExprRewritable for LogicalLimit {} impl PredicatePushdown for LogicalLimit { fn predicate_pushdown( diff --git a/src/frontend/src/optimizer/plan_node/logical_now.rs b/src/frontend/src/optimizer/plan_node/logical_now.rs index d91aa943a731c..89c2df4775ef0 100644 --- a/src/frontend/src/optimizer/plan_node/logical_now.rs +++ b/src/frontend/src/optimizer/plan_node/logical_now.rs @@ -25,7 +25,6 @@ use super::{ ColPrunable, ColumnPruningContext, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, RewriteStreamContext, StreamNow, ToBatch, ToStream, ToStreamContext, }; -use crate::expr::ExprRewriter; use crate::optimizer::property::FunctionalDependencySet; use crate::utils::ColIndexMapping; use crate::OptimizerContextRef; @@ -70,11 +69,7 @@ impl fmt::Display for LogicalNow { impl_plan_tree_node_for_leaf! { LogicalNow } -impl ExprRewritable for LogicalNow { - fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - self.clone().into() - } -} +impl ExprRewritable for LogicalNow {} impl PredicatePushdown for LogicalNow { fn predicate_pushdown( diff --git a/src/frontend/src/optimizer/plan_node/logical_over_agg.rs b/src/frontend/src/optimizer/plan_node/logical_over_agg.rs index bc92040eaafe0..75e88e37ec2a1 100644 --- a/src/frontend/src/optimizer/plan_node/logical_over_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_over_agg.rs @@ -25,9 +25,7 @@ use super::{ gen_filter_and_pushdown, ColPrunable, ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; -use crate::expr::{ - Expr, ExprImpl, ExprRewriter, InputRef, InputRefDisplay, WindowFunction, WindowFunctionType, -}; +use crate::expr::{Expr, ExprImpl, InputRef, InputRefDisplay, WindowFunction, WindowFunctionType}; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; @@ -266,11 +264,7 @@ impl ColPrunable for LogicalOverAgg { } } -impl ExprRewritable for LogicalOverAgg { - fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - self.clone().into() - } -} +impl ExprRewritable for LogicalOverAgg {} impl PredicatePushdown for LogicalOverAgg { fn predicate_pushdown( diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index 8bca6cb81b4c2..5be68d3a8daac 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -205,6 +205,10 @@ impl ColPrunable for LogicalProject { } impl ExprRewritable for LogicalProject { + fn has_rewritable_expr(&self) -> bool { + true + } + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut core = self.core.clone(); core.rewrite_exprs(r); 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 46fa700f457fb..2954922c77ed1 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project_set.rs @@ -258,6 +258,10 @@ impl ColPrunable for LogicalProjectSet { } impl ExprRewritable for LogicalProjectSet { + fn has_rewritable_expr(&self) -> bool { + true + } + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut core = self.core.clone(); core.rewrite_exprs(r); diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index e790ecebf0c33..9528e02a58685 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -451,6 +451,10 @@ impl ColPrunable for LogicalScan { } impl ExprRewritable for LogicalScan { + fn has_rewritable_expr(&self) -> bool { + true + } + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut core = self.core.clone(); core.rewrite_exprs(r); diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs index a6a01f60dc4c5..49caa4bd1a4db 100644 --- a/src/frontend/src/optimizer/plan_node/logical_share.rs +++ b/src/frontend/src/optimizer/plan_node/logical_share.rs @@ -23,7 +23,6 @@ use super::{ ColPrunable, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; -use crate::expr::ExprRewriter; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, @@ -120,11 +119,7 @@ impl ColPrunable for LogicalShare { } } -impl ExprRewritable for LogicalShare { - fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - self.clone().into() - } -} +impl ExprRewritable for LogicalShare {} impl PredicatePushdown for LogicalShare { fn predicate_pushdown( diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index cf603ed6d7fc2..789edaf9ced68 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -29,7 +29,7 @@ use super::{ }; use crate::catalog::source_catalog::SourceCatalog; use crate::catalog::ColumnId; -use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType}; +use crate::expr::{Expr, ExprImpl, ExprType}; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, @@ -161,11 +161,7 @@ impl ColPrunable for LogicalSource { } } -impl ExprRewritable for LogicalSource { - fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - self.clone().into() - } -} +impl ExprRewritable for LogicalSource {} /// A util function to extract kafka offset timestamp range. /// 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 019201dbd0ce4..8f763cdcecc00 100644 --- a/src/frontend/src/optimizer/plan_node/logical_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/logical_table_function.rs @@ -72,6 +72,10 @@ impl ColPrunable for LogicalTableFunction { } impl ExprRewritable for LogicalTableFunction { + fn has_rewritable_expr(&self) -> bool { + true + } + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut new = self.clone(); new.table_function.args = new diff --git a/src/frontend/src/optimizer/plan_node/logical_topn.rs b/src/frontend/src/optimizer/plan_node/logical_topn.rs index ddbe1b0aada01..3ddf04e476097 100644 --- a/src/frontend/src/optimizer/plan_node/logical_topn.rs +++ b/src/frontend/src/optimizer/plan_node/logical_topn.rs @@ -24,7 +24,7 @@ use super::{ PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamGroupTopN, StreamProject, ToBatch, ToStream, }; -use crate::expr::{ExprRewriter, ExprType, FunctionCall, InputRef}; +use crate::expr::{ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ BatchTopN, ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, StreamTopN, ToStreamContext, @@ -354,11 +354,7 @@ impl ColPrunable for LogicalTopN { } } -impl ExprRewritable for LogicalTopN { - fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - self.clone().into() - } -} +impl ExprRewritable for LogicalTopN {} impl PredicatePushdown for LogicalTopN { fn predicate_pushdown( diff --git a/src/frontend/src/optimizer/plan_node/logical_union.rs b/src/frontend/src/optimizer/plan_node/logical_union.rs index b92792b6267eb..17cb5575be68f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_union.rs +++ b/src/frontend/src/optimizer/plan_node/logical_union.rs @@ -19,7 +19,7 @@ use risingwave_common::error::Result; use risingwave_common::types::{DataType, Scalar}; use super::{ColPrunable, ExprRewritable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream}; -use crate::expr::{ExprImpl, ExprRewriter, InputRef, Literal}; +use crate::expr::{ExprImpl, InputRef, Literal}; use crate::optimizer::plan_node::generic::{GenericPlanNode, GenericPlanRef}; use crate::optimizer::plan_node::stream_union::StreamUnion; use crate::optimizer::plan_node::{ @@ -109,11 +109,7 @@ impl ColPrunable for LogicalUnion { } } -impl ExprRewritable for LogicalUnion { - fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - self.clone().into() - } -} +impl ExprRewritable for LogicalUnion {} impl PredicatePushdown for LogicalUnion { fn predicate_pushdown( diff --git a/src/frontend/src/optimizer/plan_node/logical_update.rs b/src/frontend/src/optimizer/plan_node/logical_update.rs index 9d630df961f45..932d9a02fea9d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_update.rs +++ b/src/frontend/src/optimizer/plan_node/logical_update.rs @@ -136,6 +136,10 @@ impl fmt::Display for LogicalUpdate { } impl ExprRewritable for LogicalUpdate { + fn has_rewritable_expr(&self) -> bool { + true + } + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut new = self.clone(); new.exprs = new.exprs.into_iter().map(|e| r.rewrite_expr(e)).collect(); diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index 1d685b87a93e5..8795cefda3ad8 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -77,6 +77,10 @@ impl fmt::Display for LogicalValues { } impl ExprRewritable for LogicalValues { + fn has_rewritable_expr(&self) -> bool { + true + } + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut new = self.clone(); new.rows = new diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 223e6bc98bf37..55e08e947111f 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -86,6 +86,22 @@ pub enum Convention { Stream, } +pub(crate) trait RewriteExprsRecursive { + fn rewrite_exprs_recursive(&self, r: &mut impl ExprRewriter) -> PlanRef; +} + +impl RewriteExprsRecursive for PlanRef { + fn rewrite_exprs_recursive(&self, r: &mut impl ExprRewriter) -> PlanRef { + let new = self.rewrite_exprs(r); + let inputs: Vec = new + .inputs() + .iter() + .map(|plan_ref| plan_ref.rewrite_exprs_recursive(r)) + .collect(); + new.clone_with_inputs(&inputs[..]) + } +} + impl ColPrunable for PlanRef { fn prune_col(&self, required_cols: &[usize], ctx: &mut ColumnPruningContext) -> PlanRef { if let Some(logical_share) = self.as_logical_share() { @@ -373,16 +389,6 @@ impl dyn PlanNode { } } - pub fn rewrite_exprs_recursive(&self, r: &mut impl ExprRewriter) -> PlanRef { - let new = self.rewrite_exprs(r); - let inputs: Vec = new - .inputs() - .iter() - .map(|plan_ref| plan_ref.rewrite_exprs_recursive(r)) - .collect(); - new.clone_with_inputs(&inputs[..]) - } - /// Serialize the plan node and its children to a batch plan proto. pub fn to_batch_prost(&self) -> BatchPlanProst { self.to_batch_prost_identity(true) From 5a94ee59e4bb2b0fc71e7cb4cca465261e7f3a9f Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 2 Feb 2023 13:37:59 +0800 Subject: [PATCH 12/22] fix license --- .../src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs | 2 +- src/frontend/src/optimizer/plan_expr_rewriter/mod.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs index a3f94cc59bbdb..28dc18ad3d10e 100644 --- a/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs +++ b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs @@ -1,4 +1,4 @@ -// Copyright 2023 Singularity Data +// 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. diff --git a/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs b/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs index 4dcb60ca5e729..4c2df9169315f 100644 --- a/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs +++ b/src/frontend/src/optimizer/plan_expr_rewriter/mod.rs @@ -1,4 +1,4 @@ -// Copyright 2023 Singularity Data +// 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. From 4024392361cb54330fd5adfee366d93eacae980b Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 2 Feb 2023 14:27:49 +0800 Subject: [PATCH 13/22] add batch --- .../src/optimizer/plan_node/batch_delete.rs | 4 +++- .../src/optimizer/plan_node/batch_exchange.rs | 4 +++- .../src/optimizer/plan_node/batch_expand.rs | 4 ++++ .../src/optimizer/plan_node/batch_filter.rs | 17 +++++++++++++++-- .../optimizer/plan_node/batch_group_topn.rs | 4 +++- .../src/optimizer/plan_node/batch_hash_agg.rs | 16 +++++++++++++++- .../optimizer/plan_node/batch_hash_join.rs | 18 ++++++++++++++++-- .../optimizer/plan_node/batch_hop_window.rs | 5 ++++- .../src/optimizer/plan_node/batch_insert.rs | 4 +++- .../src/optimizer/plan_node/batch_limit.rs | 4 +++- .../optimizer/plan_node/batch_lookup_join.rs | 19 ++++++++++++++++++- .../plan_node/batch_nested_loop_join.rs | 17 +++++++++++++++-- .../src/optimizer/plan_node/batch_project.rs | 17 +++++++++++++++-- .../optimizer/plan_node/batch_project_set.rs | 16 ++++++++++++++++ .../src/optimizer/plan_node/batch_seq_scan.rs | 17 ++++++++++++++++- .../optimizer/plan_node/batch_simple_agg.rs | 16 +++++++++++++++- .../src/optimizer/plan_node/batch_sort.rs | 4 +++- .../src/optimizer/plan_node/batch_sort_agg.rs | 18 ++++++++++++++++-- .../src/optimizer/plan_node/batch_source.rs | 3 ++- .../plan_node/batch_table_function.rs | 16 +++++++++++++++- .../src/optimizer/plan_node/batch_topn.rs | 4 +++- .../src/optimizer/plan_node/batch_union.rs | 4 +++- .../src/optimizer/plan_node/batch_update.rs | 17 +++++++++++++++-- .../src/optimizer/plan_node/batch_values.rs | 17 +++++++++++++++-- .../optimizer/plan_node/eq_join_predicate.rs | 8 +++++++- .../optimizer/plan_node/expr_rewritable.rs | 3 +-- 26 files changed, 244 insertions(+), 32 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index 84b88c3e598ab..b7d1b4f802ade 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -19,7 +19,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::DeleteNode; use super::{ - LogicalDelete, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, + LogicalDelete, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable, }; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -86,3 +86,5 @@ impl ToLocalBatch for BatchDelete { Ok(self.clone_with_input(new_input).into()) } } + +impl ExprRewritable for BatchDelete {} \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/batch_exchange.rs b/src/frontend/src/optimizer/plan_node/batch_exchange.rs index d611cd64fecde..bad55eb2f66aa 100644 --- a/src/frontend/src/optimizer/plan_node/batch_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/batch_exchange.rs @@ -18,7 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{ExchangeNode, MergeSortExchangeNode}; -use super::{PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; +use super::{PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, DistributionDisplay, Order, OrderDisplay}; @@ -98,3 +98,5 @@ impl ToLocalBatch for BatchExchange { unreachable!() } } + +impl ExprRewritable for BatchExchange {} \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/batch_expand.rs b/src/frontend/src/optimizer/plan_node/batch_expand.rs index c5099d190854a..416eb7652f68c 100644 --- a/src/frontend/src/optimizer/plan_node/batch_expand.rs +++ b/src/frontend/src/optimizer/plan_node/batch_expand.rs @@ -26,6 +26,8 @@ use crate::optimizer::plan_node::{ use crate::optimizer::property::{Distribution, Order}; use crate::optimizer::PlanRef; +use super::ExprRewritable; + #[derive(Debug, Clone)] pub struct BatchExpand { pub base: PlanBase, @@ -99,3 +101,5 @@ impl ToLocalBatch for BatchExpand { Ok(self.clone_with_input(new_input).into()) } } + +impl ExprRewritable for BatchExpand {} \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index bf78553d7ee7d..c31a6ffb484fe 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -19,8 +19,8 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::FilterNode; use super::generic::GenericPlanRef; -use super::{LogicalFilter, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; -use crate::expr::{Expr, ExprImpl}; +use super::{LogicalFilter, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::{PlanBase, ToLocalBatch}; use crate::utils::Condition; @@ -93,3 +93,16 @@ impl ToLocalBatch for BatchFilter { Ok(self.clone_with_input(new_input).into()) } } + +impl ExprRewritable for BatchFilter { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self { + base: self.base.clone_with_new_plan_id(), + logical: self.logical.rewrite_exprs(r).as_logical_filter().unwrap() + }.into() + } +} \ No newline at end of file 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 4cac43850cb00..d168ebab81b66 100644 --- a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs @@ -18,7 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::GroupTopNNode; -use super::{LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; +use super::{LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, RequiredDist}; @@ -95,3 +95,5 @@ impl ToLocalBatch for BatchGroupTopN { Ok(self.clone_with_input(input).into()) } } + +impl ExprRewritable for BatchGroupTopN {} \ No newline at end of file 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 fb03cb56a4885..907d8e4314596 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -19,7 +19,8 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashAggNode; use super::generic::{GenericPlanRef, PlanAggCall}; -use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; +use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -107,3 +108,16 @@ impl ToLocalBatch for BatchHashAgg { Ok(self.clone_with_input(new_input).into()) } } + +impl ExprRewritable for BatchHashAgg { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self { + base: self.base.clone_with_new_plan_id(), + logical: self.logical.rewrite_exprs(r).as_logical_agg().unwrap() + }.into() + } +} \ No newline at end of file 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 fad3c2521ded1..7a886939b8cdc 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -23,9 +23,9 @@ use risingwave_pb::plan_common::JoinType; use super::generic::GenericPlanRef; use super::{ EqJoinPredicate, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchProst, - ToDistributedBatch, + ToDistributedBatch, ExprRewritable, }; -use crate::expr::Expr; +use crate::expr::{Expr, ExprRewriter}; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::plan_node::{EqJoinPredicateDisplay, ToLocalBatch}; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -262,3 +262,17 @@ impl ToLocalBatch for BatchHashJoin { Ok(self.clone_with_left_right(left, right).into()) } } + +impl ExprRewritable for BatchHashJoin { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self { + base: self.base.clone_with_new_plan_id(), + logical: self.logical.rewrite_exprs(r).as_logical_join().unwrap(), + eq_join_predicate: self.eq_join_predicate.rewrite_exprs(r), + }.into() + } +} \ No newline at end of file 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 d112a47716139..1ace57e0d677b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -19,7 +19,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HopWindowNode; use super::{ - LogicalHopWindow, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, + LogicalHopWindow, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable, }; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, RequiredDist}; @@ -117,3 +117,6 @@ impl ToLocalBatch for BatchHopWindow { Ok(self.clone_with_input(new_input).into()) } } + + +impl ExprRewritable for BatchHopWindow {} \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index 5c2f105a883a7..6ae82c2493dd6 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -19,7 +19,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::InsertNode; use risingwave_pb::catalog::ColumnIndex; -use super::{LogicalInsert, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; +use super::{LogicalInsert, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; use crate::optimizer::plan_node::{PlanBase, ToLocalBatch}; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -97,3 +97,5 @@ impl ToLocalBatch for BatchInsert { Ok(self.clone_with_input(new_input).into()) } } + +impl ExprRewritable for BatchInsert {} \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index 75499fea09974..a6d7c99235bda 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -18,7 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::LimitNode; -use super::{LogicalLimit, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; +use super::{LogicalLimit, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, RequiredDist}; @@ -94,3 +94,5 @@ impl ToLocalBatch for BatchLimit { self.two_phase_limit(self.input().to_local()?) } } + +impl ExprRewritable for BatchLimit {} \ No newline at end of file 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 5b4011a51f129..65f5907bfbbfe 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -19,8 +19,9 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{DistributedLookupJoinNode, LocalLookupJoinNode}; +use super::ExprRewritable; use super::generic::GenericPlanRef; -use crate::expr::Expr; +use crate::expr::{Expr, ExprRewriter}; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::plan_node::{ EqJoinPredicate, EqJoinPredicateDisplay, LogicalJoin, PlanBase, PlanTreeNodeBinary, @@ -281,3 +282,19 @@ impl ToLocalBatch for BatchLookupJoin { Ok(self.clone_with_distributed_lookup(input, false).into()) } } + + +impl ExprRewritable for BatchLookupJoin { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self { + base: self.base.clone_with_new_plan_id(), + logical: self.logical.rewrite_exprs(r).as_logical_join().unwrap(), + eq_join_predicate: self.eq_join_predicate.rewrite_exprs(r), + ..Self::clone(self) + }.into() + } +} \ No newline at end of file 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 30c0110b46f91..6e7bf8f718c23 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 @@ -20,8 +20,8 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::NestedLoopJoinNode; use super::generic::GenericPlanRef; -use super::{LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchProst, ToDistributedBatch}; -use crate::expr::{Expr, ExprImpl}; +use super::{LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -155,3 +155,16 @@ impl ToLocalBatch for BatchNestedLoopJoin { Ok(self.clone_with_left_right(left, right).into()) } } + +impl ExprRewritable for BatchNestedLoopJoin { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self { + base: self.base.clone_with_new_plan_id(), + logical: self.logical.rewrite_exprs(r).as_logical_join().unwrap(), + }.into() + } +} \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index 43256cc4f6a0c..0ff1b770d3f59 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -21,9 +21,9 @@ use risingwave_pb::expr::ExprNode; use super::generic::GenericPlanRef; use super::{ - LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, + LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable, }; -use crate::expr::{Expr, ExprImpl}; +use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::ToLocalBatch; /// `BatchProject` implements [`super::LogicalProject`] to evaluate specified expressions on input @@ -105,3 +105,16 @@ impl ToLocalBatch for BatchProject { Ok(self.clone_with_input(new_input).into()) } } + +impl ExprRewritable for BatchProject { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self { + base: self.base.clone_with_new_plan_id(), + logical: self.logical.rewrite_exprs(r).as_logical_project().unwrap() + }.into() + } +} 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 314a00d0198d1..3631e68669692 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -19,11 +19,14 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ProjectSetNode; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{ LogicalProjectSet, PlanBase, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ToLocalBatch, }; use crate::optimizer::PlanRef; +use super::ExprRewritable; + #[derive(Debug, Clone)] pub struct BatchProjectSet { pub base: PlanBase, @@ -93,3 +96,16 @@ impl ToLocalBatch for BatchProjectSet { Ok(self.clone_with_input(new_input).into()) } } + +impl ExprRewritable for BatchProjectSet { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self { + base: self.base.clone_with_new_plan_id(), + logical: self.logical.rewrite_exprs(r).as_logical_project_set().unwrap() + }.into() + } +} 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 181197b4adfb6..815c1b9afd869 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -23,8 +23,9 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{RowSeqScanNode, SysRowSeqScanNode}; use risingwave_pb::plan_common::ColumnDesc as ProstColumnDesc; -use super::{PlanBase, PlanRef, ToBatchProst, ToDistributedBatch}; +use super::{PlanBase, PlanRef, ToBatchProst, ToDistributedBatch, BatchProjectSet, ExprRewritable}; use crate::catalog::ColumnId; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{LogicalScan, ToLocalBatch}; use crate::optimizer::property::{Distribution, DistributionDisplay, Order}; @@ -262,3 +263,17 @@ impl ToLocalBatch for BatchSeqScan { Ok(Self::new_inner(self.logical.clone(), dist, self.scan_ranges.clone()).into()) } } + +impl ExprRewritable for BatchSeqScan { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self { + base: self.base.clone_with_new_plan_id(), + logical: self.logical.rewrite_exprs(r).as_logical_scan().unwrap(), + ..Self::clone(self) + }.into() + } +} 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 1d9846f524fad..1d180f6534550 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -19,7 +19,8 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; use super::generic::{GenericPlanRef, PlanAggCall}; -use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; +use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{BatchExchange, ToLocalBatch}; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -128,3 +129,16 @@ impl ToLocalBatch for BatchSimpleAgg { Ok(self.clone_with_input(new_input).into()) } } + +impl ExprRewritable for BatchSimpleAgg { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self { + base: self.base.clone_with_new_plan_id(), + logical: self.logical.rewrite_exprs(r).as_logical_agg().unwrap() + }.into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/batch_sort.rs b/src/frontend/src/optimizer/plan_node/batch_sort.rs index 5c8578c20313d..85659148aedd1 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort.rs @@ -18,7 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortNode; -use super::{PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; +use super::{PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, OrderDisplay}; @@ -84,3 +84,5 @@ impl ToLocalBatch for BatchSort { Ok(self.clone_with_input(new_input).into()) } } + +impl ExprRewritable for BatchSort {} \ No newline at end of file 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 dae5a2c1159fb..37dd5dd695092 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -21,8 +21,8 @@ use risingwave_pb::batch_plan::SortAggNode; use risingwave_pb::expr::ExprNode; use super::generic::{GenericPlanRef, PlanAggCall}; -use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; -use crate::expr::{Expr, ExprImpl, InputRef}; +use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use crate::expr::{Expr, ExprImpl, InputRef, ExprRewriter}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -143,3 +143,17 @@ impl ToLocalBatch for BatchSortAgg { Ok(self.clone_with_input(new_input).into()) } } + +impl ExprRewritable for BatchSortAgg { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self { + base: self.base.clone_with_new_plan_id(), + logical: self.logical.rewrite_exprs(r).as_logical_agg().unwrap(), + ..Clone::clone(self) + }.into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index e7307e5df5da6..bcab8bc2e299f 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::{LogicalSource, PlanBase, PlanRef, ToBatchProst, ToDistributedBatch, ToLocalBatch}; +use super::{LogicalSource, PlanBase, PlanRef, ToBatchProst, ToDistributedBatch, ToLocalBatch, ExprRewritable}; use crate::optimizer::property::{Distribution, Order}; /// [`BatchSource`] represents a table/connector source at the very beginning of the graph. @@ -105,3 +105,4 @@ impl ToBatchProst for BatchSource { }) } } +impl ExprRewritable for BatchSource {} \ No newline at end of file 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 d656eff516e00..54f3a65525578 100644 --- a/src/frontend/src/optimizer/plan_node/batch_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/batch_table_function.rs @@ -18,7 +18,8 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TableFunctionNode; -use super::{PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchProst, ToDistributedBatch}; +use super::{PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::logical_table_function::LogicalTableFunction; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order}; @@ -78,3 +79,16 @@ impl ToLocalBatch for BatchTableFunction { Ok(Self::with_dist(self.logical().clone(), Distribution::Single).into()) } } + +impl ExprRewritable for BatchTableFunction { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self { + base: self.base.clone_with_new_plan_id(), + logical: self.logical.rewrite_exprs(r).as_logical_table_function().unwrap() + }.into() + } +} \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/batch_topn.rs b/src/frontend/src/optimizer/plan_node/batch_topn.rs index 6091e7e944fae..f961920f595e7 100644 --- a/src/frontend/src/optimizer/plan_node/batch_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_topn.rs @@ -18,7 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TopNNode; -use super::{LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; +use super::{LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, RequiredDist}; @@ -102,3 +102,5 @@ impl ToLocalBatch for BatchTopN { self.two_phase_topn(self.input().to_local()?) } } + +impl ExprRewritable for BatchTopN {} \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/batch_union.rs b/src/frontend/src/optimizer/plan_node/batch_union.rs index 72babbc2ca695..8b7d1e902ae2a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_union.rs +++ b/src/frontend/src/optimizer/plan_node/batch_union.rs @@ -18,7 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UnionNode; -use super::{PlanRef, ToBatchProst, ToDistributedBatch}; +use super::{PlanRef, ToBatchProst, ToDistributedBatch, ExprRewritable}; use crate::optimizer::plan_node::{LogicalUnion, PlanBase, PlanTreeNode, ToLocalBatch}; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -100,3 +100,5 @@ impl ToLocalBatch for BatchUnion { Ok(self.clone_with_inputs(&new_inputs?)) } } + +impl ExprRewritable for BatchUnion {} diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index 57bc9d43e5d0d..a6b6837839f67 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -20,9 +20,9 @@ use risingwave_pb::batch_plan::UpdateNode; use super::generic::GenericPlanRef; use super::{ - LogicalUpdate, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, + LogicalUpdate, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable, }; -use crate::expr::Expr; +use crate::expr::{Expr, ExprRewriter}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -101,3 +101,16 @@ impl ToLocalBatch for BatchUpdate { Ok(self.clone_with_input(new_input).into()) } } + +impl ExprRewritable for BatchUpdate { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self { + base: self.base.clone_with_new_plan_id(), + logical: self.logical.rewrite_exprs(r).as_logical_update().unwrap() + }.into() + } +} \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index 3902895d5c858..51d3672e8543b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -20,8 +20,8 @@ use risingwave_pb::batch_plan::values_node::ExprTuple; use risingwave_pb::batch_plan::ValuesNode; use super::generic::GenericPlanRef; -use super::{LogicalValues, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchProst, ToDistributedBatch}; -use crate::expr::{Expr, ExprImpl}; +use super::{LogicalValues, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order}; @@ -104,3 +104,16 @@ impl ToLocalBatch for BatchValues { Ok(Self::with_dist(self.logical().clone(), Distribution::Single).into()) } } + +impl ExprRewritable for BatchValues { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self { + base: self.base.clone_with_new_plan_id(), + logical: self.logical.rewrite_exprs(r).as_logical_values().unwrap() + }.into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs index 974158bdef124..d343e1184c9e3 100644 --- a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs +++ b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs @@ -16,7 +16,7 @@ use std::fmt; use risingwave_common::catalog::Schema; -use crate::expr::{ExprType, FunctionCall, InputRef, InputRefDisplay}; +use crate::expr::{ExprType, FunctionCall, InputRef, InputRefDisplay, ExprRewriter}; use crate::utils::{ColIndexMapping, Condition, ConditionDisplay}; /// The join predicate used in optimizer @@ -242,6 +242,12 @@ impl EqJoinPredicate { Self::new(self.other_cond, new_eq_keys, self.left_cols_num) } + + pub fn rewrite_exprs(&self, rewriter: &mut impl ExprRewriter) -> Self { + let mut new = self.clone(); + new.other_cond = new.other_cond.rewrite_expr(rewriter); + new + } } pub struct EqJoinPredicateDisplay<'a> { diff --git a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs index e9c7e3eb64d79..2e852ffd8fe6b 100644 --- a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs +++ b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs @@ -18,7 +18,7 @@ use paste::paste; use super::*; use crate::expr::ExprRewriter; -use crate::{for_batch_plan_nodes, for_stream_plan_nodes}; +use crate::for_stream_plan_nodes; /// Rewrites expressions in a `PlanRef`. Due to `Share` operator, /// the `ExprRewriter` needs to be idempotent i.e. applying it more than once @@ -54,5 +54,4 @@ macro_rules! ban_expr_rewritable { } } } -for_batch_plan_nodes! {ban_expr_rewritable} for_stream_plan_nodes! {ban_expr_rewritable} From b8286805bfa7ee101e916b823a7524f706a77a58 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 2 Feb 2023 14:40:28 +0800 Subject: [PATCH 14/22] minor --- .../tests/testdata/basic_query.yaml | 2 +- .../planner_test/tests/testdata/explain.yaml | 4 +- .../planner_test/tests/testdata/expr.yaml | 16 +-- .../planner_test/tests/testdata/order_by.yaml | 4 +- .../tests/testdata/predicate_pushdown.yaml | 14 +-- .../planner_test/tests/testdata/share.yaml | 4 +- .../tests/testdata/shared_views.yaml | 4 +- .../tests/testdata/temporal_filter.yaml | 2 +- .../planner_test/tests/testdata/tpch.yaml | 106 +++++++++--------- src/frontend/src/optimizer/mod.rs | 6 +- .../src/optimizer/plan_node/batch_filter.rs | 2 +- .../src/optimizer/plan_node/batch_hash_agg.rs | 2 +- .../optimizer/plan_node/batch_hash_join.rs | 2 +- .../optimizer/plan_node/batch_lookup_join.rs | 2 +- .../plan_node/batch_nested_loop_join.rs | 2 +- .../src/optimizer/plan_node/batch_project.rs | 2 +- .../optimizer/plan_node/batch_project_set.rs | 2 +- .../src/optimizer/plan_node/batch_seq_scan.rs | 4 +- .../optimizer/plan_node/batch_simple_agg.rs | 2 +- .../src/optimizer/plan_node/batch_sort_agg.rs | 2 +- .../plan_node/batch_table_function.rs | 2 +- .../src/optimizer/plan_node/batch_update.rs | 2 +- .../src/optimizer/plan_node/batch_values.rs | 2 +- .../optimizer/plan_node/eq_join_predicate.rs | 2 +- 24 files changed, 97 insertions(+), 95 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/basic_query.yaml b/src/frontend/planner_test/tests/testdata/basic_query.yaml index 51ac43d99d7fd..63c3df67f6e47 100644 --- a/src/frontend/planner_test/tests/testdata/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/basic_query.yaml @@ -26,7 +26,7 @@ └─BatchScan { table: t, columns: [], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamFilter { predicate: true:Boolean AND true:Boolean } + └─StreamFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int); diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index b94fcc977dea4..7d52acb270b0a 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -56,7 +56,7 @@ "stages": { "0": { "root": { - "plan_node_id": 31, + "plan_node_id": 34, "plan_node_type": "BatchProject", "schema": [ { @@ -69,7 +69,7 @@ ], "children": [ { - "plan_node_id": 29, + "plan_node_id": 32, "plan_node_type": "BatchValues", "schema": [], "children": [], diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 7f1548ae71d41..4bcf31592e05c 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -183,7 +183,7 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal), t._row_id] } + └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal), t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: case searched form without else sql: | @@ -269,7 +269,7 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [ConcatWs(t.v1, '1':Varchar), t._row_id] } + └─StreamProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar), t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 varchar); @@ -295,7 +295,7 @@ └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar), t._row_id] } + └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar), t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 float); @@ -495,7 +495,7 @@ sql: | create table t (v1 timestamp with time zone, v2 timestamp with time zone); select * from t where v1 >= now() or v2 >= now(); - stream_error: 'Expr error: Expected epoch timestamp bound into Now' + stream_error: 'internal error: Conditions containing now must be of the form `input_expr cmp now() [+- const_expr]` or `now() [+- const_expr] cmp input_expr`, where `input_expr` references a column and contains no `now()`.' - name: now inside HAVING clause sql: | create table t (v1 timestamp with time zone, v2 int); @@ -541,10 +541,12 @@ sql: | create table t(v1 int); select 1 + 2 + v1 from t; - optimized_logical_plan: | - LogicalProject { exprs: [(3:Int32 + t.v1)] } + batch_plan: | + LogicalProject { exprs: [((1:Int32 + 2:Int32) + t.v1)] } └─LogicalScan { table: t, columns: [t.v1] } - name: const_eval of division by 0 error sql: | select 1 / 0 t1; - stream_error: 'Expr error: Division by zero' \ No newline at end of file + batch_plan: | + Feature is not yet implemented: Stream values executor is unimplemented! + No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml diff --git a/src/frontend/planner_test/tests/testdata/order_by.yaml b/src/frontend/planner_test/tests/testdata/order_by.yaml index 5a74f834a74f5..f81afee8ba0e8 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -64,8 +64,8 @@ └─BatchProject { exprs: [t.v1, t.v2, 2:Int32] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, 2:Int32(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [2:Int32, t._row_id] } - └─StreamProject { exprs: [t.v1, t.v2, 2:Int32, t._row_id] } + StreamMaterialize { columns: [v1, v2, (1:Int32 + 1:Int32)(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [(1:Int32 + 1:Int32), t._row_id] } + └─StreamProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32), t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 bigint, v2 double precision); diff --git a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml index b7d24e08fceb9..52ac449e4b5ae 100644 --- a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml @@ -257,7 +257,7 @@ select * from t1 join t2 where v1 = v2 and v1 > now() + '1 hr'; optimized_logical_plan: | LogicalJoin { type: Inner, on: (t1.v1 = t2.v2), output: all } - ├─LogicalFilter { predicate: (t1.v1 > '2021-04-01 01:00:00+00:00':Timestamptz) } + ├─LogicalFilter { predicate: (t1.v1 > (Now + '01:00:00':Interval)) } | └─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | @@ -277,28 +277,28 @@ create table t2(v2 timestamp with time zone, v3 interval); select * from t1, t2 where v1 = v2 and v1 > now() + v3; optimized_logical_plan: | - LogicalFilter { predicate: (t1.v1 > ('2021-04-01 00:00:00+00:00':Timestamptz + t2.v3)) } + LogicalFilter { predicate: (t1.v1 > (Now + t2.v3)) } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2), output: all } ├─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalScan { table: t2, columns: [t2.v2, t2.v3] } - stream_error: 'Expr error: Expected epoch timestamp bound into Now' + stream_error: 'internal error: Conditions containing now must be of the form `input_expr cmp now() [+- const_expr]` or `now() [+- const_expr] cmp input_expr`, where `input_expr` references a column and contains no `now()`.' - name: now() in complex cmp expr pushed onto join ON clause results in dynamic filter sql: | create table t1(v1 timestamp with time zone); create table t2(v2 timestamp with time zone, v3 interval); select * from t1 join t2 where v1 = v2 and v1 > now() + v3; optimized_logical_plan: | - LogicalFilter { predicate: (t1.v1 > ('2021-04-01 00:00:00+00:00':Timestamptz + t2.v3)) } + LogicalFilter { predicate: (t1.v1 > (Now + t2.v3)) } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2), output: all } ├─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalScan { table: t2, columns: [t2.v2, t2.v3] } - stream_error: 'Expr error: Expected epoch timestamp bound into Now' + stream_error: 'internal error: Conditions containing now must be of the form `input_expr cmp now() [+- const_expr]` or `now() [+- const_expr] cmp input_expr`, where `input_expr` references a column and contains no `now()`.' - name: now() does not get pushed to scan, but others do sql: | create table t1(v1 timestamp with time zone, v2 int); select * from t1 where v1 > now() + '30 min' and v2 > 5; optimized_logical_plan: | - LogicalFilter { predicate: (t1.v1 > '2021-04-01 00:30:00+00:00':Timestamptz) } + LogicalFilter { predicate: (t1.v1 > (Now + '00:30:00':Interval)) } └─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: (t1.v2 > 5:Int32) } stream_plan: | StreamMaterialize { columns: [v1, v2, t1._row_id(hidden)], pk_columns: [t1._row_id] } @@ -343,7 +343,7 @@ select * from t1 join t2 where v1 = v2 and v1 > now(); optimized_logical_plan: | LogicalJoin { type: Inner, on: (t1.v1 = t2.v2), output: all } - ├─LogicalFilter { predicate: (t1.v1 > '2021-04-01 00:00:00+00:00':Timestamptz) } + ├─LogicalFilter { predicate: (t1.v1 > Now) } | └─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index 23e8555256d33..9273d92b13642 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -125,7 +125,7 @@ | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } | └─StreamProject { exprs: [auction, date_time, _row_id] } - | └─StreamShare { id = 780 } + | └─StreamShare { id = 788 } | └─StreamProject { exprs: [auction, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 4 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } @@ -137,7 +137,7 @@ └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } └─StreamProject { exprs: [auction, date_time, _row_id] } - └─StreamShare { id = 780 } + └─StreamShare { id = 788 } └─StreamProject { exprs: [auction, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 4 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/shared_views.yaml b/src/frontend/planner_test/tests/testdata/shared_views.yaml index 9e0f06e80b7e2..fba887e52df4a 100644 --- a/src/frontend/planner_test/tests/testdata/shared_views.yaml +++ b/src/frontend/planner_test/tests/testdata/shared_views.yaml @@ -26,7 +26,7 @@ └─StreamHashJoin { type: Inner, predicate: (t1.x + t1.y) = (t1.x * (t1.x + t1.y)), output: [(t1.x + t1.y), (t1.x * (t1.x + t1.y)), (t1.y * (t1.x + t1.y)), t1._row_id, t1._row_id, t1._row_id, t1.x, (t1.x + t1.y)] } ├─StreamExchange { dist: HashShard((t1.x + t1.y)) } | └─StreamProject { exprs: [(t1.x + t1.y), t1._row_id] } - | └─StreamShare { id = 220 } + | └─StreamShare { id = 207 } | └─StreamProject { exprs: [(t1.x + t1.y), 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) } @@ -37,7 +37,7 @@ | └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard((t1.x + t1.y)) } └─StreamProject { exprs: [(t1.x + t1.y), t1._row_id] } - └─StreamShare { id = 220 } + └─StreamShare { id = 207 } └─StreamProject { exprs: [(t1.x + t1.y), 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/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/temporal_filter.yaml index 91dfe76e3d6c1..98a55a6044fa0 100644 --- a/src/frontend/planner_test/tests/testdata/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/temporal_filter.yaml @@ -40,7 +40,7 @@ sql: |- create table t1 (ts timestamp with time zone); select * from t1 where now() - interval '15 minutes' > ts; - stream_error: 'Expr error: Expected epoch timestamp bound into Now' + stream_error: 'internal error: All `now()` exprs were valid, but the condition must have at least one now expr as a lower bound.' - name: Temporal filter reorders now expressions correctly sql: | create table t1 (ts timestamp with time zone); diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index 91eb604ea0fd1..9ba275f340cf4 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -128,7 +128,7 @@ LogicalProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32), RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32), RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32), count] } └─LogicalAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─LogicalProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount] } - └─LogicalScan { table: lineitem, output_columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus], required_columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate], predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + └─LogicalScan { table: lineitem, output_columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus], required_columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate], predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } batch_plan: | BatchExchange { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], dist: Single } └─BatchProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32), RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32), RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32), count] } @@ -144,7 +144,7 @@ └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─StreamExchange { dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + └─StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -157,7 +157,7 @@ Fragment 1 StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } Chain { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -717,7 +717,7 @@ optimized_logical_plan: | LogicalAgg { group_key: [orders.o_orderpriority], aggs: [count] } └─LogicalJoin { type: LeftSemi, on: (lineitem.l_orderkey = orders.o_orderkey), output: [orders.o_orderpriority] } - ├─LogicalScan { table: orders, output_columns: [orders.o_orderkey, orders.o_orderpriority], required_columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } + ├─LogicalScan { table: orders, output_columns: [orders.o_orderkey, orders.o_orderpriority], required_columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } └─LogicalScan { table: lineitem, output_columns: [lineitem.l_orderkey], required_columns: [lineitem.l_orderkey, lineitem.l_commitdate, lineitem.l_receiptdate], predicate: (lineitem.l_commitdate < lineitem.l_receiptdate) } batch_plan: | BatchExchange { order: [orders.o_orderpriority ASC], dist: Single } @@ -741,7 +741,7 @@ └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } ├─StreamExchange { dist: HashShard(orders.o_orderkey) } | └─StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } + | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } @@ -764,7 +764,7 @@ Fragment 2 StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } + StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -835,7 +835,7 @@ | | ├─LogicalJoin { type: Inner, on: (customer.c_nationkey = supplier.s_nationkey), output: [orders.o_orderkey, supplier.s_suppkey, supplier.s_nationkey] } | | | ├─LogicalJoin { type: Inner, on: (customer.c_custkey = orders.o_custkey), output: [customer.c_nationkey, orders.o_orderkey] } | | | | ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey] } - | | | | └─LogicalScan { table: orders, output_columns: [orders.o_orderkey, orders.o_custkey], required_columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } + | | | | └─LogicalScan { table: orders, output_columns: [orders.o_orderkey, orders.o_custkey], required_columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } | | | └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey] } | | └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount] } | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey] } @@ -884,7 +884,7 @@ | | | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } + | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } | | | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | | | └─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) } @@ -943,7 +943,7 @@ Fragment 7 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } + StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -1014,7 +1014,7 @@ optimized_logical_plan: | LogicalAgg { aggs: [sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─LogicalProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount)] } - └─LogicalScan { table: lineitem, output_columns: [lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_quantity, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } + └─LogicalScan { table: lineitem, output_columns: [lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_quantity, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } batch_plan: | BatchSimpleAgg { aggs: [sum(sum((lineitem.l_extendedprice * lineitem.l_discount)))] } └─BatchExchange { order: [], dist: Single } @@ -1029,7 +1029,7 @@ └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount), lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -1043,7 +1043,7 @@ Fragment 1 StreamStatelessLocalSimpleAgg { aggs: [count, sum((lineitem.l_extendedprice * lineitem.l_discount))] } StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount), lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } Chain { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -1330,9 +1330,9 @@ | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } optimized_logical_plan: | - LogicalProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - └─LogicalAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─LogicalProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + LogicalProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─LogicalAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─LogicalProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─LogicalJoin { type: Inner, on: (nation.n_regionkey = region.r_regionkey), output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name] } ├─LogicalJoin { type: Inner, on: (customer.c_nationkey = nation.n_nationkey), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey] } | ├─LogicalJoin { type: Inner, on: (orders.o_custkey = customer.c_custkey), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, customer.c_nationkey] } @@ -1376,10 +1376,10 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year] } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - └─StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(Extract('YEAR':Varchar, orders.o_orderdate)) } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } ├─StreamExchange { dist: HashShard(nation.n_regionkey) } | └─StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, customer.c_nationkey, nation.n_nationkey] } @@ -1418,13 +1418,13 @@ Fragment 0 StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year] } materialized table: 4294967294 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([4]) from 2 @@ -1511,7 +1511,7 @@ Upstream BatchPlanNode - Table 0 { columns: [Extract('YEAR':Varchar, orders_o_orderdate), count, sum(Case((nation_n_name = 'IRAN':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Decimal)), sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)))], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [Extract('YEAR':Varchar, orders_o_orderdate), count, sum(Case((nation_n_name = 'IRAN':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Int32::Decimal)), sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)))], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_orderdate, nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, customer_c_nationkey, nation_n_nationkey_0], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 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: [4] } Table 2 { columns: [nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey_0, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [14], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1812,7 +1812,7 @@ ├─LogicalJoin { type: Inner, on: (customer.c_nationkey = nation.n_nationkey), output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey, nation.n_name] } | ├─LogicalJoin { type: Inner, on: (customer.c_custkey = orders.o_custkey), output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey] } | | ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment] } - | | └─LogicalScan { table: orders, output_columns: [orders.o_orderkey, orders.o_custkey], required_columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } + | | └─LogicalScan { table: orders, output_columns: [orders.o_orderkey, orders.o_custkey], required_columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name] } └─LogicalScan { table: lineitem, output_columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_returnflag], predicate: (lineitem.l_returnflag = 'R':Varchar) } batch_plan: | @@ -1858,7 +1858,7 @@ | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } + | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | └─StreamExchange { dist: HashShard(nation.n_nationkey) } | └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } @@ -1910,7 +1910,7 @@ Fragment 6 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } + StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -2217,7 +2217,7 @@ └─LogicalProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32), Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32)] } └─LogicalJoin { type: Inner, on: (orders.o_orderkey = lineitem.l_orderkey), output: [orders.o_orderpriority, lineitem.l_shipmode] } ├─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority] } - └─LogicalScan { table: lineitem, output_columns: [lineitem.l_orderkey, lineitem.l_shipmode], required_columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + └─LogicalScan { table: lineitem, output_columns: [lineitem.l_orderkey, lineitem.l_shipmode], required_columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } batch_plan: | BatchExchange { order: [lineitem.l_shipmode ASC], dist: Single } └─BatchSort { order: [lineitem.l_shipmode ASC] } @@ -2242,7 +2242,7 @@ | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -2267,7 +2267,7 @@ Fragment 3 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -2408,11 +2408,11 @@ ├─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } optimized_logical_plan: | - LogicalProject { exprs: [((100.00:Decimal * sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } - └─LogicalAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─LogicalProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + LogicalProject { exprs: [((100.00:Decimal * sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + └─LogicalAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─LogicalProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─LogicalJoin { type: Inner, on: (lineitem.l_partkey = part.p_partkey), output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type] } - ├─LogicalScan { table: lineitem, output_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } + ├─LogicalScan { table: lineitem, output_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } └─LogicalScan { table: part, columns: [part.p_partkey, part.p_type] } batch_plan: | BatchProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2427,15 +2427,15 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [promo_revenue], pk_columns: [] } - └─StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + └─StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } └─StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } ├─StreamExchange { dist: HashShard(lineitem.l_partkey) } | └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } └─StreamExchange { dist: HashShard(part.p_partkey) } └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } @@ -2443,14 +2443,14 @@ Fragment 0 StreamMaterialize { columns: [promo_revenue], pk_columns: [] } materialized table: 4294967294 - StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -2458,7 +2458,7 @@ Fragment 2 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -2468,7 +2468,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum(Case(Like(part_p_type, 'PROMO%':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Decimal))), sum(sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount))))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [count, sum(sum(Case(Like(part_p_type, 'PROMO%':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Int32::Decimal))), sum(sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount))))], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 1 { columns: [lineitem_l_partkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 2 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 3 { columns: [part_p_partkey, part_p_type], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -2537,12 +2537,12 @@ | └─LogicalShare { id = 165 } | └─LogicalAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - | └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + | └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } └─LogicalAgg { aggs: [max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─LogicalShare { id = 165 } └─LogicalAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } batch_plan: | BatchExchange { order: [supplier.s_suppkey ASC], dist: Single } └─BatchSort { order: [supplier.s_suppkey ASC] } @@ -2573,12 +2573,12 @@ | ├─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) } | └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - | └─StreamShare { id = 951 } + | └─StreamShare { id = 943 } | └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber] } - | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } └─StreamExchange { dist: HashShard(max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))) } └─StreamProject { exprs: [max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2587,12 +2587,12 @@ └─StreamHashAgg { group_key: [Vnode(lineitem.l_suppkey)], aggs: [count, max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), Vnode(lineitem.l_suppkey)] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamShare { id = 951 } + └─StreamShare { id = 943 } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -2623,7 +2623,7 @@ Fragment 4 StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } Chain { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -3343,7 +3343,7 @@ └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(partsupp.ps_partkey, lineitem.l_partkey) AND IsNotDistinctFrom(partsupp.ps_suppkey, lineitem.l_suppkey), output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity] } ├─LogicalAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [] } | └─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey] } - └─LogicalScan { table: lineitem, output_columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity], required_columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } + └─LogicalScan { table: lineitem, output_columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity], required_columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } batch_plan: | BatchExchange { order: [supplier.s_name ASC], dist: Single } └─BatchSort { order: [supplier.s_name ASC] } @@ -3404,7 +3404,7 @@ | └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─StreamExchange { dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } + └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -3476,7 +3476,7 @@ Fragment 9 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } + StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 11d8d8ef6c281..224d0c98deaa7 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -429,9 +429,6 @@ impl PlanRoot { ApplyOrder::TopDown, ); - // Const eval of exprs at the last minute - plan = const_eval_exprs(plan)?; - if explain_trace { ctx.trace("Const eval exprs:"); ctx.trace(plan.explain_to_string().unwrap()); @@ -461,6 +458,9 @@ impl PlanRoot { // Convert to physical plan node plan = plan.to_batch_with_order_required(&self.required_order)?; + // Const eval of exprs at the last minute + plan = const_eval_exprs(plan)?; + #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); assert!(*plan.distribution() == Distribution::Single, "{}", plan); diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index c31a6ffb484fe..cc29d4abe4948 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -102,7 +102,7 @@ impl ExprRewritable for BatchFilter { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_filter().unwrap() + logical: self.logical.rewrite_exprs(r).as_logical_filter().unwrap().clone() }.into() } } \ No newline at end of file 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 907d8e4314596..83d33320599c4 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -117,7 +117,7 @@ impl ExprRewritable for BatchHashAgg { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_agg().unwrap() + logical: self.logical.rewrite_exprs(r).as_logical_agg().unwrap().clone() }.into() } } \ No newline at end of file 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 7a886939b8cdc..96cde9659fc6a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -271,7 +271,7 @@ impl ExprRewritable for BatchHashJoin { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_join().unwrap(), + logical: self.logical.rewrite_exprs(r).as_logical_join().unwrap().clone(), eq_join_predicate: self.eq_join_predicate.rewrite_exprs(r), }.into() } 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 65f5907bfbbfe..54dcfb56c96c8 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -292,7 +292,7 @@ impl ExprRewritable for BatchLookupJoin { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_join().unwrap(), + logical: self.logical.rewrite_exprs(r).as_logical_join().unwrap().clone(), eq_join_predicate: self.eq_join_predicate.rewrite_exprs(r), ..Self::clone(self) }.into() 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 6e7bf8f718c23..f4395f3832de7 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 @@ -164,7 +164,7 @@ impl ExprRewritable for BatchNestedLoopJoin { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_join().unwrap(), + logical: self.logical.rewrite_exprs(r).as_logical_join().unwrap().clone(), }.into() } } \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index 0ff1b770d3f59..d35057b2c54ad 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -114,7 +114,7 @@ impl ExprRewritable for BatchProject { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_project().unwrap() + logical: self.logical.rewrite_exprs(r).as_logical_project().unwrap().clone() }.into() } } 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 3631e68669692..200306d019d5a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -105,7 +105,7 @@ impl ExprRewritable for BatchProjectSet { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_project_set().unwrap() + logical: self.logical.rewrite_exprs(r).as_logical_project_set().unwrap().clone() }.into() } } 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 815c1b9afd869..d0e76b63151f2 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -23,7 +23,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{RowSeqScanNode, SysRowSeqScanNode}; use risingwave_pb::plan_common::ColumnDesc as ProstColumnDesc; -use super::{PlanBase, PlanRef, ToBatchProst, ToDistributedBatch, BatchProjectSet, ExprRewritable}; +use super::{PlanBase, PlanRef, ToBatchProst, ToDistributedBatch, ExprRewritable}; use crate::catalog::ColumnId; use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{LogicalScan, ToLocalBatch}; @@ -272,7 +272,7 @@ impl ExprRewritable for BatchSeqScan { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_scan().unwrap(), + logical: self.logical.rewrite_exprs(r).as_logical_scan().unwrap().clone(), ..Self::clone(self) }.into() } 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 1d180f6534550..8a7b0ec93869c 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -138,7 +138,7 @@ impl ExprRewritable for BatchSimpleAgg { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_agg().unwrap() + logical: self.logical.rewrite_exprs(r).as_logical_agg().unwrap().clone() }.into() } } 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 37dd5dd695092..ace861a13912c 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -152,7 +152,7 @@ impl ExprRewritable for BatchSortAgg { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_agg().unwrap(), + logical: self.logical.rewrite_exprs(r).as_logical_agg().unwrap().clone(), ..Clone::clone(self) }.into() } 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 54f3a65525578..bf0a5c8225567 100644 --- a/src/frontend/src/optimizer/plan_node/batch_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/batch_table_function.rs @@ -88,7 +88,7 @@ impl ExprRewritable for BatchTableFunction { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_table_function().unwrap() + logical: self.logical.rewrite_exprs(r).as_logical_table_function().unwrap().clone() }.into() } } \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index a6b6837839f67..e173598d7f5ea 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -110,7 +110,7 @@ impl ExprRewritable for BatchUpdate { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_update().unwrap() + logical: self.logical.rewrite_exprs(r).as_logical_update().unwrap().clone() }.into() } } \ No newline at end of file diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index 51d3672e8543b..3a428276656a4 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -113,7 +113,7 @@ impl ExprRewritable for BatchValues { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_values().unwrap() + logical: self.logical.rewrite_exprs(r).as_logical_values().unwrap().clone() }.into() } } diff --git a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs index d343e1184c9e3..04b122e0cbad0 100644 --- a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs +++ b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs @@ -243,7 +243,7 @@ impl EqJoinPredicate { Self::new(self.other_cond, new_eq_keys, self.left_cols_num) } - pub fn rewrite_exprs(&self, rewriter: &mut impl ExprRewriter) -> Self { + pub fn rewrite_exprs(&self, rewriter: &mut (impl ExprRewriter + ?Sized)) -> Self { let mut new = self.clone(); new.other_cond = new.other_cond.rewrite_expr(rewriter); new From 9065b6b2e7dc5dd11ae13f0f49a07c9528d7f8ea Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 2 Feb 2023 14:40:43 +0800 Subject: [PATCH 15/22] minor --- src/frontend/planner_test/tests/testdata/expr.yaml | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 4bcf31592e05c..79c4b12e36e07 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -542,11 +542,10 @@ create table t(v1 int); select 1 + 2 + v1 from t; batch_plan: | - LogicalProject { exprs: [((1:Int32 + 2:Int32) + t.v1)] } - └─LogicalScan { table: t, columns: [t.v1] } + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [(3:Int32 + t.v1)] } + └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: const_eval of division by 0 error sql: | select 1 / 0 t1; - batch_plan: | - Feature is not yet implemented: Stream values executor is unimplemented! - No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml + batch_error: 'Expr error: Division by zero' From a7ccce9aeb75c8c12f72449455e023cdb47e47ed Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 2 Feb 2023 14:44:37 +0800 Subject: [PATCH 16/22] fmt --- .../src/optimizer/plan_node/batch_delete.rs | 5 +++-- .../src/optimizer/plan_node/batch_exchange.rs | 6 ++++-- .../src/optimizer/plan_node/batch_expand.rs | 5 ++--- .../src/optimizer/plan_node/batch_filter.rs | 18 +++++++++++++----- .../optimizer/plan_node/batch_group_topn.rs | 7 +++++-- .../src/optimizer/plan_node/batch_hash_agg.rs | 19 ++++++++++++++----- .../optimizer/plan_node/batch_hash_join.rs | 18 ++++++++++++------ .../optimizer/plan_node/batch_hop_window.rs | 6 +++--- .../src/optimizer/plan_node/batch_insert.rs | 6 ++++-- .../src/optimizer/plan_node/batch_limit.rs | 7 +++++-- .../optimizer/plan_node/batch_lookup_join.rs | 17 +++++++++++------ .../plan_node/batch_nested_loop_join.rs | 19 ++++++++++++++----- .../src/optimizer/plan_node/batch_project.rs | 15 +++++++++++---- .../optimizer/plan_node/batch_project_set.rs | 15 ++++++++++----- .../src/optimizer/plan_node/batch_seq_scan.rs | 14 ++++++++++---- .../optimizer/plan_node/batch_simple_agg.rs | 17 +++++++++++++---- .../src/optimizer/plan_node/batch_sort.rs | 6 ++++-- .../src/optimizer/plan_node/batch_sort_agg.rs | 19 ++++++++++++++----- .../src/optimizer/plan_node/batch_source.rs | 7 +++++-- .../plan_node/batch_table_function.rs | 18 +++++++++++++----- .../src/optimizer/plan_node/batch_topn.rs | 7 +++++-- .../src/optimizer/plan_node/batch_union.rs | 2 +- .../src/optimizer/plan_node/batch_update.rs | 17 ++++++++++++----- .../src/optimizer/plan_node/batch_values.rs | 17 +++++++++++++---- .../optimizer/plan_node/eq_join_predicate.rs | 2 +- 25 files changed, 202 insertions(+), 87 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index b7d1b4f802ade..8762d59834d8d 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -19,7 +19,8 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::DeleteNode; use super::{ - LogicalDelete, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable, + ExprRewritable, LogicalDelete, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ToDistributedBatch, }; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -87,4 +88,4 @@ impl ToLocalBatch for BatchDelete { } } -impl ExprRewritable for BatchDelete {} \ No newline at end of file +impl ExprRewritable for BatchDelete {} diff --git a/src/frontend/src/optimizer/plan_node/batch_exchange.rs b/src/frontend/src/optimizer/plan_node/batch_exchange.rs index bad55eb2f66aa..aabd57b898979 100644 --- a/src/frontend/src/optimizer/plan_node/batch_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/batch_exchange.rs @@ -18,7 +18,9 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{ExchangeNode, MergeSortExchangeNode}; -use super::{PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ + ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, +}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, DistributionDisplay, Order, OrderDisplay}; @@ -99,4 +101,4 @@ impl ToLocalBatch for BatchExchange { } } -impl ExprRewritable for BatchExchange {} \ No newline at end of file +impl ExprRewritable for BatchExchange {} diff --git a/src/frontend/src/optimizer/plan_node/batch_expand.rs b/src/frontend/src/optimizer/plan_node/batch_expand.rs index 416eb7652f68c..7f6232a0bf921 100644 --- a/src/frontend/src/optimizer/plan_node/batch_expand.rs +++ b/src/frontend/src/optimizer/plan_node/batch_expand.rs @@ -20,14 +20,13 @@ use risingwave_pb::batch_plan::expand_node::Subset; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ExpandNode; +use super::ExprRewritable; use crate::optimizer::plan_node::{ LogicalExpand, PlanBase, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ToLocalBatch, }; use crate::optimizer::property::{Distribution, Order}; use crate::optimizer::PlanRef; -use super::ExprRewritable; - #[derive(Debug, Clone)] pub struct BatchExpand { pub base: PlanBase, @@ -102,4 +101,4 @@ impl ToLocalBatch for BatchExpand { } } -impl ExprRewritable for BatchExpand {} \ No newline at end of file +impl ExprRewritable for BatchExpand {} diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index cc29d4abe4948..42e0494a8b370 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -19,7 +19,9 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::FilterNode; use super::generic::GenericPlanRef; -use super::{LogicalFilter, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ + ExprRewritable, LogicalFilter, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, +}; use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::{PlanBase, ToLocalBatch}; use crate::utils::Condition; @@ -100,9 +102,15 @@ impl ExprRewritable for BatchFilter { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { + Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_filter().unwrap().clone() - }.into() + logical: self + .logical + .rewrite_exprs(r) + .as_logical_filter() + .unwrap() + .clone(), + } + .into() } -} \ No newline at end of file +} 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 d168ebab81b66..ac9ad13c5c99a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs @@ -18,7 +18,10 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::GroupTopNNode; -use super::{LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ + ExprRewritable, LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ToDistributedBatch, +}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, RequiredDist}; @@ -96,4 +99,4 @@ impl ToLocalBatch for BatchGroupTopN { } } -impl ExprRewritable for BatchGroupTopN {} \ No newline at end of file +impl ExprRewritable for BatchGroupTopN {} 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 83d33320599c4..8f6cb42b3db59 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -19,7 +19,10 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashAggNode; use super::generic::{GenericPlanRef, PlanAggCall}; -use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ + ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ToDistributedBatch, +}; use crate::expr::ExprRewriter; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -115,9 +118,15 @@ impl ExprRewritable for BatchHashAgg { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { + Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_agg().unwrap().clone() - }.into() + logical: self + .logical + .rewrite_exprs(r) + .as_logical_agg() + .unwrap() + .clone(), + } + .into() } -} \ No newline at end of file +} 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 96cde9659fc6a..686d5e94df658 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -22,8 +22,8 @@ use risingwave_pb::plan_common::JoinType; use super::generic::GenericPlanRef; use super::{ - EqJoinPredicate, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchProst, - ToDistributedBatch, ExprRewritable, + EqJoinPredicate, ExprRewritable, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, + ToBatchProst, ToDistributedBatch, }; use crate::expr::{Expr, ExprRewriter}; use crate::optimizer::plan_node::utils::IndicesDisplay; @@ -269,10 +269,16 @@ impl ExprRewritable for BatchHashJoin { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { + Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_join().unwrap().clone(), + logical: self + .logical + .rewrite_exprs(r) + .as_logical_join() + .unwrap() + .clone(), eq_join_predicate: self.eq_join_predicate.rewrite_exprs(r), - }.into() + } + .into() } -} \ No newline at end of file +} 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 1ace57e0d677b..dfd460642fc02 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -19,7 +19,8 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HopWindowNode; use super::{ - LogicalHopWindow, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable, + ExprRewritable, LogicalHopWindow, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ToDistributedBatch, }; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, RequiredDist}; @@ -118,5 +119,4 @@ impl ToLocalBatch for BatchHopWindow { } } - -impl ExprRewritable for BatchHopWindow {} \ No newline at end of file +impl ExprRewritable for BatchHopWindow {} diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index 6ae82c2493dd6..f8aac43a7f797 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -19,7 +19,9 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::InsertNode; use risingwave_pb::catalog::ColumnIndex; -use super::{LogicalInsert, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ + ExprRewritable, LogicalInsert, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, +}; use crate::optimizer::plan_node::{PlanBase, ToLocalBatch}; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -98,4 +100,4 @@ impl ToLocalBatch for BatchInsert { } } -impl ExprRewritable for BatchInsert {} \ No newline at end of file +impl ExprRewritable for BatchInsert {} diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index a6d7c99235bda..2d6ad614ce4a6 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -18,7 +18,10 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::LimitNode; -use super::{LogicalLimit, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ + ExprRewritable, LogicalLimit, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ToDistributedBatch, +}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, RequiredDist}; @@ -95,4 +98,4 @@ impl ToLocalBatch for BatchLimit { } } -impl ExprRewritable for BatchLimit {} \ No newline at end of file +impl ExprRewritable for BatchLimit {} 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 54dcfb56c96c8..ea16eda2816c7 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -19,8 +19,8 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{DistributedLookupJoinNode, LocalLookupJoinNode}; -use super::ExprRewritable; use super::generic::GenericPlanRef; +use super::ExprRewritable; use crate::expr::{Expr, ExprRewriter}; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::plan_node::{ @@ -283,18 +283,23 @@ impl ToLocalBatch for BatchLookupJoin { } } - impl ExprRewritable for BatchLookupJoin { fn has_rewritable_expr(&self) -> bool { true } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { + Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_join().unwrap().clone(), + logical: self + .logical + .rewrite_exprs(r) + .as_logical_join() + .unwrap() + .clone(), eq_join_predicate: self.eq_join_predicate.rewrite_exprs(r), ..Self::clone(self) - }.into() + } + .into() } -} \ No newline at end of file +} 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 f4395f3832de7..711bee2993510 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 @@ -20,7 +20,10 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::NestedLoopJoinNode; use super::generic::GenericPlanRef; -use super::{LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ + ExprRewritable, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchProst, + ToDistributedBatch, +}; use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::plan_node::ToLocalBatch; @@ -162,9 +165,15 @@ impl ExprRewritable for BatchNestedLoopJoin { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { + Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_join().unwrap().clone(), - }.into() + logical: self + .logical + .rewrite_exprs(r) + .as_logical_join() + .unwrap() + .clone(), + } + .into() } -} \ No newline at end of file +} diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index d35057b2c54ad..b697da3d69e35 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -21,7 +21,8 @@ use risingwave_pb::expr::ExprNode; use super::generic::GenericPlanRef; use super::{ - LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable, + ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ToDistributedBatch, }; use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::ToLocalBatch; @@ -112,9 +113,15 @@ impl ExprRewritable for BatchProject { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { + Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_project().unwrap().clone() - }.into() + logical: self + .logical + .rewrite_exprs(r) + .as_logical_project() + .unwrap() + .clone(), + } + .into() } } 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 200306d019d5a..63627abab16d1 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -19,14 +19,13 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ProjectSetNode; +use super::ExprRewritable; use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{ LogicalProjectSet, PlanBase, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ToLocalBatch, }; use crate::optimizer::PlanRef; -use super::ExprRewritable; - #[derive(Debug, Clone)] pub struct BatchProjectSet { pub base: PlanBase, @@ -103,9 +102,15 @@ impl ExprRewritable for BatchProjectSet { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { + Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_project_set().unwrap().clone() - }.into() + logical: self + .logical + .rewrite_exprs(r) + .as_logical_project_set() + .unwrap() + .clone(), + } + .into() } } 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 d0e76b63151f2..25f0b972c6f59 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -23,7 +23,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{RowSeqScanNode, SysRowSeqScanNode}; use risingwave_pb::plan_common::ColumnDesc as ProstColumnDesc; -use super::{PlanBase, PlanRef, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ExprRewritable, PlanBase, PlanRef, ToBatchProst, ToDistributedBatch}; use crate::catalog::ColumnId; use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{LogicalScan, ToLocalBatch}; @@ -270,10 +270,16 @@ impl ExprRewritable for BatchSeqScan { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { + Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_scan().unwrap().clone(), + logical: self + .logical + .rewrite_exprs(r) + .as_logical_scan() + .unwrap() + .clone(), ..Self::clone(self) - }.into() + } + .into() } } 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 8a7b0ec93869c..013fe9298d961 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -19,7 +19,10 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; use super::generic::{GenericPlanRef, PlanAggCall}; -use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ + ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ToDistributedBatch, +}; use crate::expr::ExprRewriter; use crate::optimizer::plan_node::{BatchExchange, ToLocalBatch}; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -136,9 +139,15 @@ impl ExprRewritable for BatchSimpleAgg { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { + Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_agg().unwrap().clone() - }.into() + logical: self + .logical + .rewrite_exprs(r) + .as_logical_agg() + .unwrap() + .clone(), + } + .into() } } diff --git a/src/frontend/src/optimizer/plan_node/batch_sort.rs b/src/frontend/src/optimizer/plan_node/batch_sort.rs index 85659148aedd1..ca1d02c6f7d23 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort.rs @@ -18,7 +18,9 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortNode; -use super::{PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ + ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, +}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, OrderDisplay}; @@ -85,4 +87,4 @@ impl ToLocalBatch for BatchSort { } } -impl ExprRewritable for BatchSort {} \ No newline at end of file +impl ExprRewritable for BatchSort {} 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 ace861a13912c..a28819329aed6 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -21,8 +21,11 @@ use risingwave_pb::batch_plan::SortAggNode; use risingwave_pb::expr::ExprNode; use super::generic::{GenericPlanRef, PlanAggCall}; -use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; -use crate::expr::{Expr, ExprImpl, InputRef, ExprRewriter}; +use super::{ + ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ToDistributedBatch, +}; +use crate::expr::{Expr, ExprImpl, ExprRewriter, InputRef}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -150,10 +153,16 @@ impl ExprRewritable for BatchSortAgg { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { + Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_agg().unwrap().clone(), + logical: self + .logical + .rewrite_exprs(r) + .as_logical_agg() + .unwrap() + .clone(), ..Clone::clone(self) - }.into() + } + .into() } } diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index bcab8bc2e299f..b4f7780f274e7 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -19,7 +19,10 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SourceNode; -use super::{LogicalSource, PlanBase, PlanRef, ToBatchProst, ToDistributedBatch, ToLocalBatch, ExprRewritable}; +use super::{ + ExprRewritable, LogicalSource, PlanBase, PlanRef, ToBatchProst, ToDistributedBatch, + ToLocalBatch, +}; use crate::optimizer::property::{Distribution, Order}; /// [`BatchSource`] represents a table/connector source at the very beginning of the graph. @@ -105,4 +108,4 @@ impl ToBatchProst for BatchSource { }) } } -impl ExprRewritable for BatchSource {} \ No newline at end of file +impl ExprRewritable for BatchSource {} 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 bf0a5c8225567..19cf9808f8235 100644 --- a/src/frontend/src/optimizer/plan_node/batch_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/batch_table_function.rs @@ -18,7 +18,9 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TableFunctionNode; -use super::{PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ + ExprRewritable, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchProst, ToDistributedBatch, +}; use crate::expr::ExprRewriter; use crate::optimizer::plan_node::logical_table_function::LogicalTableFunction; use crate::optimizer::plan_node::ToLocalBatch; @@ -86,9 +88,15 @@ impl ExprRewritable for BatchTableFunction { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { + Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_table_function().unwrap().clone() - }.into() + logical: self + .logical + .rewrite_exprs(r) + .as_logical_table_function() + .unwrap() + .clone(), + } + .into() } -} \ No newline at end of file +} diff --git a/src/frontend/src/optimizer/plan_node/batch_topn.rs b/src/frontend/src/optimizer/plan_node/batch_topn.rs index f961920f595e7..2dc5645efbca0 100644 --- a/src/frontend/src/optimizer/plan_node/batch_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_topn.rs @@ -18,7 +18,10 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TopNNode; -use super::{LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ + ExprRewritable, LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ToDistributedBatch, +}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Order, RequiredDist}; @@ -103,4 +106,4 @@ impl ToLocalBatch for BatchTopN { } } -impl ExprRewritable for BatchTopN {} \ No newline at end of file +impl ExprRewritable for BatchTopN {} diff --git a/src/frontend/src/optimizer/plan_node/batch_union.rs b/src/frontend/src/optimizer/plan_node/batch_union.rs index 8b7d1e902ae2a..4ac6e87218ada 100644 --- a/src/frontend/src/optimizer/plan_node/batch_union.rs +++ b/src/frontend/src/optimizer/plan_node/batch_union.rs @@ -18,7 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UnionNode; -use super::{PlanRef, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ExprRewritable, PlanRef, ToBatchProst, ToDistributedBatch}; use crate::optimizer::plan_node::{LogicalUnion, PlanBase, PlanTreeNode, ToLocalBatch}; use crate::optimizer::property::{Distribution, Order, RequiredDist}; diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index e173598d7f5ea..f70a1b8c766f0 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -20,7 +20,8 @@ use risingwave_pb::batch_plan::UpdateNode; use super::generic::GenericPlanRef; use super::{ - LogicalUpdate, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, ExprRewritable, + ExprRewritable, LogicalUpdate, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, + ToDistributedBatch, }; use crate::expr::{Expr, ExprRewriter}; use crate::optimizer::plan_node::ToLocalBatch; @@ -108,9 +109,15 @@ impl ExprRewritable for BatchUpdate { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { + Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_update().unwrap().clone() - }.into() + logical: self + .logical + .rewrite_exprs(r) + .as_logical_update() + .unwrap() + .clone(), + } + .into() } -} \ No newline at end of file +} diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index 3a428276656a4..7edeb9f476388 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -20,7 +20,10 @@ use risingwave_pb::batch_plan::values_node::ExprTuple; use risingwave_pb::batch_plan::ValuesNode; use super::generic::GenericPlanRef; -use super::{LogicalValues, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchProst, ToDistributedBatch, ExprRewritable}; +use super::{ + ExprRewritable, LogicalValues, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchProst, + ToDistributedBatch, +}; use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order}; @@ -111,9 +114,15 @@ impl ExprRewritable for BatchValues { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { + Self { base: self.base.clone_with_new_plan_id(), - logical: self.logical.rewrite_exprs(r).as_logical_values().unwrap().clone() - }.into() + logical: self + .logical + .rewrite_exprs(r) + .as_logical_values() + .unwrap() + .clone(), + } + .into() } } diff --git a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs index 04b122e0cbad0..b684cc70c2024 100644 --- a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs +++ b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs @@ -16,7 +16,7 @@ use std::fmt; use risingwave_common::catalog::Schema; -use crate::expr::{ExprType, FunctionCall, InputRef, InputRefDisplay, ExprRewriter}; +use crate::expr::{ExprRewriter, ExprType, FunctionCall, InputRef, InputRefDisplay}; use crate::utils::{ColIndexMapping, Condition, ConditionDisplay}; /// The join predicate used in optimizer From c9a003171b18350a75431ae3333b790cf9f71762 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 2 Feb 2023 14:48:39 +0800 Subject: [PATCH 17/22] minor --- .../src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs | 1 - src/frontend/src/optimizer/plan_node/expr_rewritable.rs | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs index 28dc18ad3d10e..7fdfa1857713d 100644 --- a/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs +++ b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs @@ -25,7 +25,6 @@ impl ExprRewriter for ConstEvalRewriter { return expr; } if expr.is_const() { - //} && expr.count_nows() == 0 { let data_type = expr.return_type(); match expr.eval_row_const() { Ok(datum) => Literal::new(datum, data_type).into(), diff --git a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs index 2e852ffd8fe6b..b1d43bf8c6905 100644 --- a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs +++ b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs @@ -29,7 +29,7 @@ pub trait ExprRewritable { } fn rewrite_exprs(&self, _r: &mut dyn ExprRewriter) -> PlanRef { - unimplemented!("{}", self.has_rewritable_expr()) + unimplemented!() } } From cf6828e0bb495329e964a26d221a02dc6e0daf7e Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 2 Feb 2023 15:22:06 +0800 Subject: [PATCH 18/22] fix --- .../planner_test/tests/testdata/array.yaml | 14 ++-- .../tests/testdata/basic_query.yaml | 8 +-- .../planner_test/tests/testdata/cast.yaml | 4 +- .../planner_test/tests/testdata/explain.yaml | 15 +--- .../planner_test/tests/testdata/expr.yaml | 68 +++++++++---------- .../tests/testdata/index_selection.yaml | 6 +- .../planner_test/tests/testdata/insert.yaml | 8 +-- .../planner_test/tests/testdata/order_by.yaml | 6 +- .../tests/testdata/project_set.yaml | 12 ++-- .../tests/testdata/range_scan.yaml | 4 +- .../planner_test/tests/testdata/share.yaml | 8 +-- .../planner_test/tests/testdata/tpch.yaml | 38 +++++------ src/frontend/src/optimizer/mod.rs | 14 ++-- 13 files changed, 97 insertions(+), 108 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/array.yaml b/src/frontend/planner_test/tests/testdata/array.yaml index fd26d4a94ac4a..b436cbdff2d4e 100644 --- a/src/frontend/planner_test/tests/testdata/array.yaml +++ b/src/frontend/planner_test/tests/testdata/array.yaml @@ -4,13 +4,13 @@ logical_plan: | LogicalValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Varchar }] } } batch_plan: | - BatchValues { rows: [[ARRAY[foo, bar]:List { datatype: Varchar }]] } + BatchValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]] } - sql: | values (ARRAY[1, 2+3, 4*5+1]); logical_plan: | LogicalValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Int32 }] } } batch_plan: | - BatchValues { rows: [[ARRAY[1, 5, 21]:List { datatype: Int32 }]] } + BatchValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]] } - sql: | create table t (v1 int); select (ARRAY[1, v1]) from t; @@ -52,7 +52,7 @@ LogicalProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[66, 123]:List { datatype: Int32 }] } + BatchProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32))] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[array[66]], array[233]); @@ -60,7 +60,7 @@ LogicalProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[{66}, {233}]:List { datatype: List { datatype: Int32 } }] } + BatchProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32))] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[233], array[array[66]]); @@ -68,7 +68,7 @@ LogicalProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32)))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[{233}, {66}]:List { datatype: List { datatype: Int32 } }] } + BatchProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32)))] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[233], array[array[array[66]]]); @@ -85,7 +85,7 @@ LogicalProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[66, 123]:List { datatype: Int32 }] } + BatchProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32)] } └─BatchValues { rows: [[]] } - sql: | select array_append(123, 234); @@ -102,7 +102,7 @@ LogicalProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[123, 66]:List { datatype: Int32 }] } + BatchProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32))] } └─BatchValues { rows: [[]] } - sql: | select array_prepend(123, 234); diff --git a/src/frontend/planner_test/tests/testdata/basic_query.yaml b/src/frontend/planner_test/tests/testdata/basic_query.yaml index 63c3df67f6e47..5b141abcc0d1f 100644 --- a/src/frontend/planner_test/tests/testdata/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/basic_query.yaml @@ -1,7 +1,7 @@ # This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - sql: values (11, 22), (33+(1+2), 44); batch_plan: | - BatchValues { rows: [[11:Int32, 22:Int32], [36:Int32, 44:Int32]] } + BatchValues { rows: [[11:Int32, 22:Int32], [(33:Int32 + (1:Int32 + 2:Int32)), 44:Int32]] } - sql: select * from t binder_error: 'Catalog error: table or source not found: t' - sql: | @@ -22,7 +22,7 @@ select * from t where 1>2 and 1=1 and 3<1 and 4<>1 or 1=1 and 2>=1 and 1<=2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: true:Boolean AND true:Boolean } + └─BatchFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } └─BatchScan { table: t, columns: [], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [t._row_id(hidden)], pk_columns: [t._row_id] } @@ -130,11 +130,11 @@ - sql: | select * from unnest(Array[1,2,3]); batch_plan: | - BatchTableFunction { Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 }) } + BatchTableFunction { Unnest(Array(1:Int32, 2:Int32, 3:Int32)) } - sql: | select * from unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchTableFunction { Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } }) } + BatchTableFunction { Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32))) } - sql: | create table t1 (x int); select * from t1 where EXISTS(select * where t1.x=1); diff --git a/src/frontend/planner_test/tests/testdata/cast.yaml b/src/frontend/planner_test/tests/testdata/cast.yaml index b9385ce48355e..e122dd0ace778 100644 --- a/src/frontend/planner_test/tests/testdata/cast.yaml +++ b/src/frontend/planner_test/tests/testdata/cast.yaml @@ -18,7 +18,7 @@ sql: | select case when NULL then 1 end; batch_plan: | - BatchProject { exprs: [null:Int32] } + BatchProject { exprs: [Case(null:Boolean, 1:Int32)] } └─BatchValues { rows: [[]] } - name: implicit cast boolean (JOIN ON NULL) sql: | @@ -65,7 +65,7 @@ sql: | select case when 'y' then 1 end; batch_plan: | - BatchProject { exprs: [1:Int32] } + BatchProject { exprs: [Case(true:Boolean, 1:Int32)] } └─BatchValues { rows: [[]] } - name: implicit cast boolean (JOIN ON with literal 'y' of unknown type) sql: | diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index 7d52acb270b0a..c89502dce3ade 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -36,11 +36,6 @@ LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - Const eval exprs: - - LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - To Batch Physical Plan: BatchProject { exprs: [1:Int32] } @@ -56,7 +51,7 @@ "stages": { "0": { "root": { - "plan_node_id": 34, + "plan_node_id": 28, "plan_node_type": "BatchProject", "schema": [ { @@ -69,7 +64,7 @@ ], "children": [ { - "plan_node_id": 32, + "plan_node_id": 26, "plan_node_type": "BatchValues", "schema": [], "children": [], @@ -170,12 +165,6 @@ ├─LogicalScan { table: t1, columns: [v1] } └─LogicalScan { table: t2, columns: [v2] } - Const eval exprs: - - LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } - ├─LogicalScan { table: t1, columns: [v1] } - └─LogicalScan { table: t2, columns: [v2] } - - sql: | explain (logical) create table t1(v1 int); explain_output: | diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 79c4b12e36e07..0ad24241013a8 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -19,17 +19,17 @@ - sql: | values(cast(1 as bigint)); batch_plan: | - BatchValues { rows: [[1:Int64]] } + BatchValues { rows: [[1:Int32::Int64]] } - sql: | values(not true); batch_plan: | - BatchValues { rows: [[false:Boolean]] } + BatchValues { rows: [[Not(true:Boolean)]] } - sql: | create table t (); select (((((false is not true) is true) is not false) is false) is not null) is null from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [false:Boolean] } + └─BatchProject { exprs: [IsNull(IsNotNull(IsFalse(IsNotFalse(IsTrue(IsNotTrue(false:Boolean))))))] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: bind between sql: | @@ -65,13 +65,13 @@ sql: | SELECT 1::real in (3, 1.0, 2); batch_plan: | - BatchProject { exprs: [true:Boolean] } + BatchProject { exprs: [In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32)] } └─BatchValues { rows: [[]] } - name: not in-list with aligned types sql: | SELECT 1::real not in (3, 1.0, 2); batch_plan: | - BatchProject { exprs: [false:Boolean] } + BatchProject { exprs: [Not(In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32))] } └─BatchValues { rows: [[]] } - name: in-list with misaligned types sql: | @@ -82,7 +82,7 @@ create table t (v1 int); SELECT 1 in (3, 0.5*2, min(v1)) from t; batch_plan: | - BatchProject { exprs: [(true:Boolean OR (1:Int32 = min(min(t.v1))))] } + BatchProject { exprs: [(In(1:Int32::Decimal, 3:Int32::Decimal, (0.5:Decimal * 2:Int32)) OR (1:Int32 = min(min(t.v1))))] } └─BatchSimpleAgg { aggs: [min(min(t.v1))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [min(t.v1)] } @@ -94,7 +94,7 @@ SELECT b2 from b where 1 in (3, 1.0, (select min(v1) from t)); batch_plan: | BatchProject { exprs: [b.b2] } - └─BatchFilter { predicate: (true:Boolean OR (1:Int32 = min(min(t.v1)))) } + └─BatchFilter { predicate: (In(1:Int32::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (1:Int32 = min(min(t.v1)))) } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b2], distribution: SomeShard } @@ -108,7 +108,7 @@ create table b (b1 int, b2 int); SELECT b2 from b where exists (select 2 from t where v1 in (3, 1.0, b1)); batch_plan: | - BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } + BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b1, b.b2], distribution: SomeShard } └─BatchExchange { order: [], dist: Single } @@ -122,19 +122,19 @@ - sql: | values(round(42.4382, 2)); batch_plan: | - BatchValues { rows: [[42.44:Decimal]] } + BatchValues { rows: [[RoundDigit(42.4382:Decimal, 2:Int32)]] } - sql: | values(round(42.4382)); batch_plan: | - BatchValues { rows: [[42:Decimal]] } + BatchValues { rows: [[Round(42.4382:Decimal)]] } - sql: | values(round(42)); batch_plan: | - BatchValues { rows: [[42:Float64]] } + BatchValues { rows: [[Round(42:Int32::Float64)]] } - sql: | values(round(42, 2)); batch_plan: | - BatchValues { rows: [[42:Decimal]] } + BatchValues { rows: [[RoundDigit(42:Int32::Decimal, 2:Int32)]] } - sql: | values(round(true)); binder_error: |- @@ -144,19 +144,18 @@ -- Single quoted literal can be treated as number without error. values(round('123')); batch_plan: | - BatchValues { rows: [[123:Float64]] } + BatchValues { rows: [[Round(123:Float64)]] } - sql: | -- When it is invalid, PostgreSQL reports error during explain, but we have to wait until execution as of now. #4235 values(round('abc')); - batch_error: 'Expr error: Parse error: risingwave_common::types::ordered_float::OrderedFloat' - sql: | values(extract(hour from timestamp '2001-02-16 20:38:40')); batch_plan: | - BatchValues { rows: [[20:Decimal]] } + BatchValues { rows: [[Extract('HOUR':Varchar, '2001-02-16 20:38:40':Timestamp)]] } - sql: | values('Postgres' not like 'Post%'); batch_plan: | - BatchValues { rows: [[false:Boolean]] } + BatchValues { rows: [[Not(Like('Postgres':Varchar, 'Post%':Varchar))]] } - sql: | values(1 not like 1.23); binder_error: |- @@ -165,13 +164,13 @@ - sql: | select length(trim(trailing '1' from '12'))+length(trim(leading '2' from '23'))+length(trim(both '3' from '34')); batch_plan: | - BatchProject { exprs: [4:Int32] } + BatchProject { exprs: [((Length(Rtrim('12':Varchar, '1':Varchar)) + Length(Ltrim('23':Varchar, '2':Varchar))) + Length(Trim('34':Varchar, '3':Varchar)))] } └─BatchValues { rows: [[]] } - sql: | select position(replace('1','1','2'),'123') where '12' like '%1'; batch_plan: | - BatchProject { exprs: [0:Int32] } - └─BatchFilter { predicate: false:Boolean } + BatchProject { exprs: [Position(Replace('1':Varchar, '1':Varchar, '2':Varchar), '123':Varchar)] } + └─BatchFilter { predicate: Like('12':Varchar, '%1':Varchar) } └─BatchValues { rows: [[]] } - name: case searched form with else sql: | @@ -179,7 +178,7 @@ select (case when v1=1 then 1 when v1=2 then 2 else 0.0 end) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } @@ -191,7 +190,7 @@ select (case when v1=1 then 1 when v1=2 then 2.1 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2.1:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2.1:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case simple form sql: | @@ -199,7 +198,7 @@ select (case v1 when 1 then 1 when 2.0 then 2 else 0.0 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2.0:Decimal), 2:Decimal, 0.0:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2.0:Decimal), 2:Int32::Decimal, 0.0:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case misaligned result types sql: | @@ -265,7 +264,7 @@ select concat_ws(v1, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, '1':Varchar)] } + └─BatchProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } @@ -276,7 +275,7 @@ select concat_ws(v1, 1.2) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, '1.2':Varchar)] } + └─BatchProject { exprs: [ConcatWs(t.v1, 1.2:Decimal::Varchar)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - sql: | create table t (v1 int); @@ -291,7 +290,7 @@ select concat(v1, v2, v3, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar)] } + └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar)] } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } @@ -311,22 +310,22 @@ - sql: | select concat(':', true); batch_plan: | - BatchProject { exprs: [':t':Varchar] } + BatchProject { exprs: [ConcatWs('':Varchar, ':':Varchar, BoolOut(true:Boolean))] } └─BatchValues { rows: [[]] } - sql: | select ':' || true; batch_plan: | - BatchProject { exprs: [':true':Varchar] } + BatchProject { exprs: [ConcatOp(':':Varchar, true:Boolean::Varchar)] } └─BatchValues { rows: [[]] } - sql: | select substr('hello', NULL); batch_plan: | - BatchProject { exprs: [null:Varchar] } + BatchProject { exprs: [Substr('hello':Varchar, null:Int32)] } └─BatchValues { rows: [[]] } - sql: | select substr(NULL, 1); batch_plan: | - BatchProject { exprs: [null:Varchar] } + BatchProject { exprs: [Substr(null:Varchar, 1:Int32)] } └─BatchValues { rows: [[]] } - sql: | select pg_typeof('123'); @@ -420,10 +419,10 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [Some((1:Int32 < ArrayCat(ARRAY[1]:List { datatype: Int32 }, ARRAY[2]:List { datatype: Int32 })))] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat(Array(1:Int32), Array(2:Int32))))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchProject { exprs: [ARRAY[1]:List { datatype: Int32 }] } + └─BatchProject { exprs: [Array(1:Int32)] } └─BatchValues { rows: [[]] } - sql: | select 1 < ALL(array[null]::integer[]); @@ -444,10 +443,10 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [All((1:Int32 < ArrayCat(ARRAY[1]:List { datatype: Int32 }, ARRAY[2]:List { datatype: Int32 })))] } + BatchProject { exprs: [All((1:Int32 < ArrayCat(Array(1:Int32), Array(2:Int32))))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchProject { exprs: [ARRAY[1]:List { datatype: Int32 }] } + └─BatchProject { exprs: [Array(1:Int32)] } └─BatchValues { rows: [[]] } - name: now expression sql: | @@ -543,9 +542,8 @@ select 1 + 2 + v1 from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [(3:Int32 + t.v1)] } + └─BatchProject { exprs: [((1:Int32 + 2:Int32) + t.v1)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: const_eval of division by 0 error sql: | select 1 / 0 t1; - batch_error: 'Expr error: Division by zero' diff --git a/src/frontend/planner_test/tests/testdata/index_selection.yaml b/src/frontend/planner_test/tests/testdata/index_selection.yaml index c98da2bf8e0bc..af18731a327f8 100644 --- a/src/frontend/planner_test/tests/testdata/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/index_selection.yaml @@ -77,7 +77,7 @@ select a,b from t1 where a in (1,2) and b in (2,3) batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: In(idx1.b, 2:Decimal, 3:Decimal) } + └─BatchFilter { predicate: In(idx1.b, 2:Int32::Decimal, 3:Int32::Decimal) } └─BatchScan { table: idx1, columns: [idx1.a, idx1.b], scan_ranges: [idx1.a = Int32(1) , idx1.a = Int32(2)], distribution: UpstreamHashShard(idx1.a, idx1.b) } - sql: | create table t1 (a int, b numeric, c bigint); @@ -213,13 +213,13 @@ update t1 set c = 3 where a = 1 and b = 2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } + └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int32::Int64, $3] } └─BatchExchange { order: [], dist: Single } └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } batch_local_plan: | - BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } + BatchUpdate { table: t1, exprs: [$0, $1, 3:Int32::Int64, $3] } └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/insert.yaml b/src/frontend/planner_test/tests/testdata/insert.yaml index 985277c9273f7..aa67c803d90a2 100644 --- a/src/frontend/planner_test/tests/testdata/insert.yaml +++ b/src/frontend/planner_test/tests/testdata/insert.yaml @@ -14,7 +14,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [[22.33:Float32, 33:Int32], [44:Float32, 55:Int32]] } + └─BatchValues { rows: [[22.33:Decimal::Float32, 33:Int32], [44:Int32::Float32, 55.0:Decimal::Int32]] } - name: insert values on non-assign-castable types sql: | create table t (v1 real, v2 int); @@ -73,7 +73,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [['01:02:03':Time], ['03:04:05':Time]] } + └─BatchValues { rows: [['2020-01-01 01:02:03':Timestamp::Time], ['03:04:05':Time]] } - name: a `VALUES` without insert context may be invalid on its own (compare with above) sql: | create table t (v1 time); @@ -93,7 +93,7 @@ sql: | values (1), (null), (2.3); batch_plan: | - BatchValues { rows: [[1:Decimal], [null:Decimal], [2.3:Decimal]] } + BatchValues { rows: [[1:Int32::Decimal], [null:Decimal], [2.3:Decimal]] } - name: rows of different number of columns sql: | values (1), (2, 3); @@ -115,7 +115,7 @@ BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } └─BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: ['01:02:03':Time, 11:Int32, 4.5:Float32] } + └─BatchProject { exprs: ['2020-01-01 01:02:03':Timestamp::Time, 11:Int32, 4.5:Decimal::Float32] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: insert into select with cast error sql: | diff --git a/src/frontend/planner_test/tests/testdata/order_by.yaml b/src/frontend/planner_test/tests/testdata/order_by.yaml index f81afee8ba0e8..4f460213c6791 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -59,9 +59,9 @@ select * from t order by 1+1; batch_plan: | BatchProject { exprs: [t.v1, t.v2] } - └─BatchExchange { order: [2:Int32 ASC], dist: Single } - └─BatchSort { order: [2:Int32 ASC] } - └─BatchProject { exprs: [t.v1, t.v2, 2:Int32] } + └─BatchExchange { order: [(1:Int32 + 1:Int32) ASC], dist: Single } + └─BatchSort { order: [(1:Int32 + 1:Int32) ASC] } + └─BatchProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32)] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [v1, v2, (1:Int32 + 1:Int32)(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [(1:Int32 + 1:Int32), t._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/project_set.yaml b/src/frontend/planner_test/tests/testdata/project_set.yaml index fde593e397a9c..4adbeadc2bc68 100644 --- a/src/frontend/planner_test/tests/testdata/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/project_set.yaml @@ -8,14 +8,14 @@ - sql: | select unnest(Array[1,2,3]); batch_plan: | - BatchProject { exprs: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } - └─BatchProjectSet { select_list: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } + BatchProject { exprs: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } + └─BatchProjectSet { select_list: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } └─BatchValues { rows: [[]] } - sql: | select unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchProject { exprs: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } - └─BatchProjectSet { select_list: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } + BatchProject { exprs: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } + └─BatchProjectSet { select_list: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } └─BatchValues { rows: [[]] } - sql: | create table t(x int[]); @@ -43,9 +43,9 @@ create table t(x int[]); select unnest(x), unnest(Array[1,2]) from t; batch_plan: | - BatchProject { exprs: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } + BatchProject { exprs: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } └─BatchExchange { order: [], dist: Single } - └─BatchProjectSet { select_list: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } + └─BatchProjectSet { select_list: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } - name: table functions as parameters of usual functions sql: | diff --git a/src/frontend/planner_test/tests/testdata/range_scan.yaml b/src/frontend/planner_test/tests/testdata/range_scan.yaml index 4c6678d46ccbd..df8dbca94c355 100644 --- a/src/frontend/planner_test/tests/testdata/range_scan.yaml +++ b/src/frontend/planner_test/tests/testdata/range_scan.yaml @@ -87,7 +87,7 @@ SELECT * FROM orders_count_by_user WHERE user_id > 42 AND date = 1111 AND 2>1 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: true:Boolean AND (orders_count_by_user.date = 1111:Int32) } + └─BatchFilter { predicate: (2:Int32 > 1:Int32) AND (orders_count_by_user.date = 1111:Int32) } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id > Int64(42)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv @@ -95,7 +95,7 @@ SELECT * FROM orders_count_by_user WHERE date > 1111 AND user_id = 42 AND 5<6 AND date <= 6666 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: true:Boolean } + └─BatchFilter { predicate: (5:Int32 < 6:Int32) } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id = Int64(42) AND orders_count_by_user.date > Int32(1111) AND orders_count_by_user.date <= Int32(6666)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index 9273d92b13642..d150ff23dcd82 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -48,7 +48,7 @@ | └─StreamProject { exprs: [id, _row_id] } | └─StreamFilter { predicate: (initial_bid = 1:Int32) } | └─StreamProject { exprs: [id, initial_bid, _row_id] } - | └─StreamShare { id = 559 } + | └─StreamShare { id = 519 } | └─StreamProject { exprs: [id, initial_bid, _row_id] } | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } | └─StreamRowIdGen { row_id_index: 9 } @@ -57,7 +57,7 @@ └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } └─StreamProject { exprs: [id, initial_bid, _row_id] } - └─StreamShare { id = 559 } + └─StreamShare { id = 519 } └─StreamProject { exprs: [id, initial_bid, _row_id] } └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 9 } @@ -125,7 +125,7 @@ | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } | └─StreamProject { exprs: [auction, date_time, _row_id] } - | └─StreamShare { id = 788 } + | └─StreamShare { id = 744 } | └─StreamProject { exprs: [auction, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 4 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } @@ -137,7 +137,7 @@ └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } └─StreamProject { exprs: [auction, date_time, _row_id] } - └─StreamShare { id = 788 } + └─StreamShare { id = 744 } └─StreamProject { exprs: [auction, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 4 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index 9ba275f340cf4..953cdbb3bcfd3 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -136,7 +136,7 @@ └─BatchHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } └─BatchProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount] } - └─BatchFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + └─BatchFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } └─BatchScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], pk_columns: [l_returnflag, l_linestatus] } @@ -727,7 +727,7 @@ └─BatchHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority] } ├─BatchExchange { order: [], dist: HashShard(orders.o_orderkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey] } @@ -859,7 +859,7 @@ | | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], distribution: UpstreamHashShard(customer.c_custkey) } | | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } + | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } | | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } | └─BatchExchange { order: [], dist: HashShard(supplier.s_nationkey) } | └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } @@ -1020,7 +1020,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─BatchProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount)] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } └─BatchScan { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [revenue], pk_columns: [] } @@ -1350,11 +1350,11 @@ └─LogicalScan { table: region, output_columns: [region.r_regionkey], required_columns: [region.r_regionkey, region.r_name], predicate: (region.r_name = 'ASIA':Varchar) } batch_plan: | BatchExchange { order: [Extract('YEAR':Varchar, orders.o_orderdate) ASC], dist: Single } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } └─BatchSort { order: [Extract('YEAR':Varchar, orders.o_orderdate) ASC] } - └─BatchHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─BatchHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─BatchExchange { order: [], dist: HashShard(Extract('YEAR':Varchar, orders.o_orderdate)) } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'ASIA':Varchar), output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name] } └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } └─BatchLookupJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey] } @@ -1832,7 +1832,7 @@ | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], distribution: UpstreamHashShard(customer.c_custkey) } | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount] } @@ -2229,7 +2229,7 @@ | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode] } - └─BatchFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + └─BatchFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], pk_columns: [l_shipmode] } @@ -2415,15 +2415,15 @@ ├─LogicalScan { table: lineitem, output_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } └─LogicalScan { table: part, columns: [part.p_partkey, part.p_type] } batch_plan: | - BatchProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - └─BatchSimpleAgg { aggs: [sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + BatchProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + └─BatchSimpleAgg { aggs: [sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + └─BatchSimpleAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type] } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_partkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [promo_revenue], pk_columns: [] } @@ -2554,7 +2554,7 @@ | └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } | └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } | └─BatchScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } └─BatchExchange { order: [], dist: HashShard(max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))) } └─BatchSimpleAgg { aggs: [max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2563,7 +2563,7 @@ └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } └─BatchScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2573,7 +2573,7 @@ | ├─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) } | └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - | └─StreamShare { id = 943 } + | └─StreamShare { id = 900 } | └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } @@ -2587,7 +2587,7 @@ └─StreamHashAgg { group_key: [Vnode(lineitem.l_suppkey)], aggs: [count, max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), Vnode(lineitem.l_suppkey)] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamShare { id = 943 } + └─StreamShare { id = 900 } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } @@ -3369,7 +3369,7 @@ | └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity] } - └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } + └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], pk_columns: [supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], order_descs: [s_name, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 224d0c98deaa7..f53161351b2c2 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -429,11 +429,6 @@ impl PlanRoot { ApplyOrder::TopDown, ); - if explain_trace { - ctx.trace("Const eval exprs:"); - ctx.trace(plan.explain_to_string().unwrap()); - } - #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); @@ -458,8 +453,14 @@ impl PlanRoot { // Convert to physical plan node plan = plan.to_batch_with_order_required(&self.required_order)?; + // SessionTimezone substitution // Const eval of exprs at the last minute - plan = const_eval_exprs(plan)?; + // plan = const_eval_exprs(plan)?; + + // if explain_trace { + // ctx.trace("Const eval exprs:"); + // ctx.trace(plan.explain_to_string().unwrap()); + // } #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); @@ -722,6 +723,7 @@ impl PlanRoot { } } +#[allow(dead_code)] fn const_eval_exprs(plan: PlanRef) -> Result { let mut const_eval_rewriter = ConstEvalRewriter { error: None }; From 9d95a85ce30a29b242fcdb896f883cf655e2e9e7 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 2 Feb 2023 15:52:35 +0800 Subject: [PATCH 19/22] fix --- e2e_test/batch/functions/now.slt.part | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/e2e_test/batch/functions/now.slt.part b/e2e_test/batch/functions/now.slt.part index 5adb66c83db2a..a5472976df4ff 100644 --- a/e2e_test/batch/functions/now.slt.part +++ b/e2e_test/batch/functions/now.slt.part @@ -11,14 +11,14 @@ statement ok insert into t values(now()); # constant eval of now in batch plan -query T -explain select now() + interval '1 hour' = now() + interval '30 minutes' + interval '30 minutes' true; ----- -BatchProject { exprs: [true:Boolean] } - └─BatchValues { rows: [[]] } +# query T +# explain select now() + interval '1 hour' = now() + interval '30 minutes' + interval '30 minutes' true; +# ---- +# BatchProject { exprs: [true:Boolean] } +# └─BatchValues { rows: [[]] } -statement ok -drop table tz +# statement ok +# drop table tz -statement ok -drop table t +# statement ok +# drop table t From 1b397a2f6715da8bb7cf90849f715fab64b1016b Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Thu, 2 Feb 2023 16:25:44 +0800 Subject: [PATCH 20/22] fix --- e2e_test/batch/functions/now.slt.part | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/e2e_test/batch/functions/now.slt.part b/e2e_test/batch/functions/now.slt.part index a5472976df4ff..8d3ce7867b3d9 100644 --- a/e2e_test/batch/functions/now.slt.part +++ b/e2e_test/batch/functions/now.slt.part @@ -17,8 +17,8 @@ insert into t values(now()); # BatchProject { exprs: [true:Boolean] } # └─BatchValues { rows: [[]] } -# statement ok -# drop table tz +statement ok +drop table tz -# statement ok -# drop table t +statement ok +drop table t From b09128270ea45aa8fe8a6ba7f37c5ebd98f3b56b Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Fri, 3 Feb 2023 09:18:10 +0800 Subject: [PATCH 21/22] try to use new for batch --- src/frontend/src/optimizer/plan_node/batch_filter.rs | 8 +++----- src/frontend/src/optimizer/plan_node/batch_hash_agg.rs | 8 +++----- .../src/optimizer/plan_node/batch_hash_join.rs | 10 ++++------ .../src/optimizer/plan_node/batch_nested_loop_join.rs | 8 +++----- src/frontend/src/optimizer/plan_node/batch_project.rs | 8 +++----- .../src/optimizer/plan_node/batch_project_set.rs | 8 +++----- src/frontend/src/optimizer/plan_node/batch_seq_scan.rs | 10 ++++------ .../src/optimizer/plan_node/batch_simple_agg.rs | 8 +++----- src/frontend/src/optimizer/plan_node/batch_sort_agg.rs | 9 +++------ .../src/optimizer/plan_node/batch_table_function.rs | 8 +++----- src/frontend/src/optimizer/plan_node/batch_update.rs | 8 +++----- src/frontend/src/optimizer/plan_node/batch_values.rs | 8 +++----- 12 files changed, 38 insertions(+), 63 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index 42e0494a8b370..c92634c2713e1 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -102,15 +102,13 @@ impl ExprRewritable for BatchFilter { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { - base: self.base.clone_with_new_plan_id(), - logical: self - .logical + Self::new( + self.logical .rewrite_exprs(r) .as_logical_filter() .unwrap() .clone(), - } + ) .into() } } 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 8f6cb42b3db59..db1847f18cb7e 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -118,15 +118,13 @@ impl ExprRewritable for BatchHashAgg { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { - base: self.base.clone_with_new_plan_id(), - logical: self - .logical + Self::new( + self.logical .rewrite_exprs(r) .as_logical_agg() .unwrap() .clone(), - } + ) .into() } } 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 686d5e94df658..77fdef9ffefeb 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -269,16 +269,14 @@ impl ExprRewritable for BatchHashJoin { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { - base: self.base.clone_with_new_plan_id(), - logical: self - .logical + Self::new( + self.logical .rewrite_exprs(r) .as_logical_join() .unwrap() .clone(), - eq_join_predicate: self.eq_join_predicate.rewrite_exprs(r), - } + self.eq_join_predicate.rewrite_exprs(r), + ) .into() } } 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 711bee2993510..490cc65d733c8 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 @@ -165,15 +165,13 @@ impl ExprRewritable for BatchNestedLoopJoin { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { - base: self.base.clone_with_new_plan_id(), - logical: self - .logical + Self::new( + self.logical .rewrite_exprs(r) .as_logical_join() .unwrap() .clone(), - } + ) .into() } } diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index b697da3d69e35..538575bd2f903 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -113,15 +113,13 @@ impl ExprRewritable for BatchProject { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { - base: self.base.clone_with_new_plan_id(), - logical: self - .logical + Self::new( + self.logical .rewrite_exprs(r) .as_logical_project() .unwrap() .clone(), - } + ) .into() } } 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 63627abab16d1..9380ce37f7e25 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -102,15 +102,13 @@ impl ExprRewritable for BatchProjectSet { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { - base: self.base.clone_with_new_plan_id(), - logical: self - .logical + Self::new( + self.logical .rewrite_exprs(r) .as_logical_project_set() .unwrap() .clone(), - } + ) .into() } } 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 25f0b972c6f59..5ddddb8105a1b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -270,16 +270,14 @@ impl ExprRewritable for BatchSeqScan { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { - base: self.base.clone_with_new_plan_id(), - logical: self - .logical + Self::new( + self.logical .rewrite_exprs(r) .as_logical_scan() .unwrap() .clone(), - ..Self::clone(self) - } + self.scan_ranges.clone(), + ) .into() } } 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 013fe9298d961..5739b740145c4 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -139,15 +139,13 @@ impl ExprRewritable for BatchSimpleAgg { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { - base: self.base.clone_with_new_plan_id(), - logical: self - .logical + Self::new( + self.logical .rewrite_exprs(r) .as_logical_agg() .unwrap() .clone(), - } + ) .into() } } 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 a28819329aed6..1e3056dff4427 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -153,16 +153,13 @@ impl ExprRewritable for BatchSortAgg { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { - base: self.base.clone_with_new_plan_id(), - logical: self - .logical + Self::new( + self.logical .rewrite_exprs(r) .as_logical_agg() .unwrap() .clone(), - ..Clone::clone(self) - } + ) .into() } } 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 19cf9808f8235..355a6c60db0f6 100644 --- a/src/frontend/src/optimizer/plan_node/batch_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/batch_table_function.rs @@ -88,15 +88,13 @@ impl ExprRewritable for BatchTableFunction { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { - base: self.base.clone_with_new_plan_id(), - logical: self - .logical + Self::new( + self.logical .rewrite_exprs(r) .as_logical_table_function() .unwrap() .clone(), - } + ) .into() } } diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index f70a1b8c766f0..381072d592051 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -109,15 +109,13 @@ impl ExprRewritable for BatchUpdate { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { - base: self.base.clone_with_new_plan_id(), - logical: self - .logical + Self::new( + self.logical .rewrite_exprs(r) .as_logical_update() .unwrap() .clone(), - } + ) .into() } } diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index 7edeb9f476388..21b719d3821eb 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -114,15 +114,13 @@ impl ExprRewritable for BatchValues { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self { - base: self.base.clone_with_new_plan_id(), - logical: self - .logical + Self::new( + self.logical .rewrite_exprs(r) .as_logical_values() .unwrap() .clone(), - } + ) .into() } } From c7a48684c5a4ce52d3cc25ac626c865695e7df50 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Fri, 3 Feb 2023 09:22:26 +0800 Subject: [PATCH 22/22] minor --- src/frontend/src/optimizer/plan_node/expr_rewritable.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs index b1d43bf8c6905..baeb4caa1d61a 100644 --- a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs +++ b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs @@ -23,6 +23,8 @@ use crate::for_stream_plan_nodes; /// Rewrites expressions in a `PlanRef`. Due to `Share` operator, /// the `ExprRewriter` needs to be idempotent i.e. applying it more than once /// to the same `ExprImpl` will be a noop on subsequent applications. +/// `rewrite_exprs` should only return a plan with the given node modified. +/// To rewrite recursively, call `rewrite_exprs_recursive` on [`RewriteExprsRecursive`]. pub trait ExprRewritable { fn has_rewritable_expr(&self) -> bool { false