From edc6660ff069f389c7010d1e85a9dc8230e4e131 Mon Sep 17 00:00:00 2001 From: Dylan Date: Wed, 13 Jul 2022 20:38:23 +0800 Subject: [PATCH] feat(frontend): support explain verbose (#3798) * feat(frontend) support explain verbose * prefix each field name with table_name * do-apply-planner-test * minor improvement * fix batch two phase agg which should satisfy all agg-call contain no distinct and order by * fix stream two phase agg which should satisfy all agg-call contain no distinct and order by * move explain_verbose flag from session config to optimizer context * fmt * fix test case --- src/common/src/catalog/mod.rs | 2 +- src/common/src/catalog/schema.rs | 23 +++ src/frontend/src/expr/function_call.rs | 105 ++++++++++- src/frontend/src/expr/input_ref.rs | 35 ++++ src/frontend/src/expr/mod.rs | 41 ++++- src/frontend/src/handler/explain.rs | 6 +- .../src/optimizer/plan_node/batch_exchange.rs | 30 +++- .../src/optimizer/plan_node/batch_expand.rs | 24 ++- .../src/optimizer/plan_node/batch_filter.rs | 18 +- .../src/optimizer/plan_node/batch_hash_agg.rs | 39 ++-- .../optimizer/plan_node/batch_hash_join.rs | 19 +- .../plan_node/batch_nested_loop_join.rs | 18 +- .../src/optimizer/plan_node/batch_seq_scan.rs | 19 +- .../optimizer/plan_node/batch_simple_agg.rs | 23 ++- .../src/optimizer/plan_node/batch_sort.rs | 16 +- .../src/optimizer/plan_node/batch_topn.rs | 17 +- .../optimizer/plan_node/eq_join_predicate.rs | 70 +++++++- .../src/optimizer/plan_node/logical_agg.rs | 168 ++++++++++++++---- .../src/optimizer/plan_node/logical_expand.rs | 14 +- .../optimizer/plan_node/logical_hop_window.rs | 15 +- .../optimizer/plan_node/logical_project.rs | 29 ++- .../src/optimizer/plan_node/logical_scan.rs | 31 +++- src/frontend/src/optimizer/plan_node/mod.rs | 2 +- .../optimizer/plan_node/stream_delta_join.rs | 20 ++- .../plan_node/stream_dynamic_filter.rs | 21 ++- .../optimizer/plan_node/stream_exchange.rs | 24 ++- .../src/optimizer/plan_node/stream_expand.rs | 24 ++- .../src/optimizer/plan_node/stream_filter.rs | 18 +- .../plan_node/stream_global_simple_agg.rs | 12 +- .../optimizer/plan_node/stream_hash_agg.rs | 39 ++-- .../optimizer/plan_node/stream_hash_join.rs | 28 ++- .../optimizer/plan_node/stream_index_scan.rs | 8 +- .../plan_node/stream_local_simple_agg.rs | 12 +- .../src/optimizer/plan_node/stream_project.rs | 3 +- .../optimizer/plan_node/stream_table_scan.rs | 10 +- .../src/optimizer/plan_node/stream_topn.rs | 21 ++- .../src/optimizer/property/distribution.rs | 59 ++++++ src/frontend/src/optimizer/property/order.rs | 60 ++++++- src/frontend/src/session.rs | 11 +- src/frontend/src/utils/condition.rs | 53 +++++- .../test_runner/tests/testdata/agg.yaml | 70 ++++---- .../tests/testdata/append_only.yaml | 10 +- .../tests/testdata/basic_query.yaml | 8 +- .../tests/testdata/column_pruning.yaml | 2 +- .../testdata/common_table_expressions.yaml | 6 +- .../tests/testdata/distribution_derive.yaml | 4 +- .../test_runner/tests/testdata/expr.yaml | 10 +- .../test_runner/tests/testdata/index.yaml | 18 +- .../test_runner/tests/testdata/join.yaml | 12 +- .../test_runner/tests/testdata/mv_on_mv.yaml | 6 +- .../test_runner/tests/testdata/nexmark.yaml | 24 +-- .../test_runner/tests/testdata/order_by.yaml | 14 +- .../test_runner/tests/testdata/pk_derive.yaml | 8 +- .../tests/testdata/stream_proto.yaml | 104 +++++------ .../tests/testdata/struct_query.yaml | 2 +- .../tests/testdata/time_window.yaml | 14 +- .../test_runner/tests/testdata/tpch.yaml | 38 ++-- 57 files changed, 1223 insertions(+), 314 deletions(-) diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index 4a2943558de90..650724ca55a20 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -26,7 +26,7 @@ use async_trait::async_trait; pub use column::*; pub use internal_table::*; pub use physical_table::*; -pub use schema::{test_utils as schema_test_utils, Field, Schema}; +pub use schema::{test_utils as schema_test_utils, Field, FieldVerboseDisplay, Schema}; use crate::array::Row; pub use crate::config::constant::hummock; diff --git a/src/common/src/catalog/schema.rs b/src/common/src/catalog/schema.rs index 69199c8fde84f..4afd8847383d4 100644 --- a/src/common/src/catalog/schema.rs +++ b/src/common/src/catalog/schema.rs @@ -67,6 +67,20 @@ impl Field { } } +pub struct FieldVerboseDisplay<'a>(pub &'a Field); + +impl std::fmt::Debug for FieldVerboseDisplay<'_> { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0.name) + } +} + +impl std::fmt::Display for FieldVerboseDisplay<'_> { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0.name) + } +} + /// `schema_unnamed` builds a `Schema` with the given types, but without names. #[macro_export] macro_rules! schema_unnamed { @@ -176,6 +190,15 @@ impl Field { pub fn data_type(&self) -> DataType { self.data_type.clone() } + + pub fn from_with_table_name_prefix(desc: &ColumnDesc, table_name: &str) -> Self { + Self { + data_type: desc.data_type.clone(), + name: format!("{}.{}", table_name, desc.name), + sub_fields: desc.field_descs.iter().map(|d| d.into()).collect_vec(), + type_name: desc.type_name.clone(), + } + } } impl From<&ProstField> for Field { diff --git a/src/frontend/src/expr/function_call.rs b/src/frontend/src/expr/function_call.rs index 1b5404bca73e5..6fccb7bde639e 100644 --- a/src/frontend/src/expr/function_call.rs +++ b/src/frontend/src/expr/function_call.rs @@ -14,11 +14,12 @@ use itertools::Itertools; use num_integer::Integer as _; +use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; use super::{align_types, cast_ok, infer_type, CastContext, Expr, ExprImpl, Literal}; -use crate::expr::ExprType; +use crate::expr::{ExprType, ExprVerboseDisplay}; #[derive(Clone, Eq, PartialEq, Hash)] pub struct FunctionCall { @@ -259,3 +260,105 @@ impl Expr for FunctionCall { } } } + +pub struct FunctionCallVerboseDisplay<'a> { + pub function_call: &'a FunctionCall, + pub input_schema: &'a Schema, +} + +impl std::fmt::Debug for FunctionCallVerboseDisplay<'_> { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let that = self.function_call; + match &that.func_type { + ExprType::Cast => { + assert_eq!(that.inputs.len(), 1); + ExprVerboseDisplay { + expr: &that.inputs[0], + input_schema: self.input_schema, + } + .fmt(f)?; + write!(f, "::{:?}", that.return_type) + } + ExprType::Add => explain_verbose_binary_op(f, "+", &that.inputs, self.input_schema), + ExprType::Subtract => { + explain_verbose_binary_op(f, "-", &that.inputs, self.input_schema) + } + ExprType::Multiply => { + explain_verbose_binary_op(f, "*", &that.inputs, self.input_schema) + } + ExprType::Divide => explain_verbose_binary_op(f, "/", &that.inputs, self.input_schema), + ExprType::Modulus => explain_verbose_binary_op(f, "%", &that.inputs, self.input_schema), + ExprType::Equal => explain_verbose_binary_op(f, "=", &that.inputs, self.input_schema), + ExprType::NotEqual => { + explain_verbose_binary_op(f, "<>", &that.inputs, self.input_schema) + } + ExprType::LessThan => { + explain_verbose_binary_op(f, "<", &that.inputs, self.input_schema) + } + ExprType::LessThanOrEqual => { + explain_verbose_binary_op(f, "<=", &that.inputs, self.input_schema) + } + ExprType::GreaterThan => { + explain_verbose_binary_op(f, ">", &that.inputs, self.input_schema) + } + ExprType::GreaterThanOrEqual => { + explain_verbose_binary_op(f, ">=", &that.inputs, self.input_schema) + } + ExprType::And => explain_verbose_binary_op(f, "AND", &that.inputs, self.input_schema), + ExprType::Or => explain_verbose_binary_op(f, "OR", &that.inputs, self.input_schema), + ExprType::BitwiseShiftLeft => { + explain_verbose_binary_op(f, "<<", &that.inputs, self.input_schema) + } + ExprType::BitwiseShiftRight => { + explain_verbose_binary_op(f, ">>", &that.inputs, self.input_schema) + } + ExprType::BitwiseAnd => { + explain_verbose_binary_op(f, "&", &that.inputs, self.input_schema) + } + ExprType::BitwiseOr => { + explain_verbose_binary_op(f, "|", &that.inputs, self.input_schema) + } + ExprType::BitwiseXor => { + explain_verbose_binary_op(f, "#", &that.inputs, self.input_schema) + } + _ => { + let func_name = format!("{:?}", that.func_type); + let mut builder = f.debug_tuple(&func_name); + that.inputs.iter().for_each(|child| { + builder.field(&ExprVerboseDisplay { + expr: child, + input_schema: self.input_schema, + }); + }); + builder.finish() + } + } + } +} + +fn explain_verbose_binary_op( + f: &mut std::fmt::Formatter<'_>, + op: &str, + inputs: &[ExprImpl], + input_schema: &Schema, +) -> std::fmt::Result { + use std::fmt::Debug; + + assert_eq!(inputs.len(), 2); + + write!(f, "(")?; + ExprVerboseDisplay { + expr: &inputs[0], + input_schema, + } + .fmt(f)?; + write!(f, " {} ", op)?; + ExprVerboseDisplay { + expr: &inputs[1], + input_schema, + } + .fmt(f)?; + write!(f, ")")?; + + Ok(()) +} diff --git a/src/frontend/src/expr/input_ref.rs b/src/frontend/src/expr/input_ref.rs index 8a449bc1c9e38..c131cb2c57367 100644 --- a/src/frontend/src/expr/input_ref.rs +++ b/src/frontend/src/expr/input_ref.rs @@ -15,6 +15,7 @@ use std::fmt; use itertools::Itertools; +use risingwave_common::catalog::Schema; use risingwave_common::types::DataType; use risingwave_pb::expr::agg_call::Arg as ProstAggCallArg; use risingwave_pb::expr::InputRefExpr; @@ -53,6 +54,40 @@ impl fmt::Debug for InputRefDisplay { } } +#[derive(Clone, Copy)] +pub struct InputRefVerboseDisplay<'a> { + pub input_ref: &'a InputRef, + pub input_schema: &'a Schema, +} + +impl fmt::Display for InputRefVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!( + f, + "{}", + self.input_schema + .fields + .get(self.input_ref.index) + .unwrap() + .name + ) + } +} + +impl fmt::Debug for InputRefVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!( + f, + "{}", + self.input_schema + .fields + .get(self.input_ref.index) + .unwrap() + .name + ) + } +} + impl fmt::Display for InputRef { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { write!(f, "{:?}", InputRefDisplay(self.index)) diff --git a/src/frontend/src/expr/mod.rs b/src/frontend/src/expr/mod.rs index 0aaadba18eeca..7c4eb8759adf1 100644 --- a/src/frontend/src/expr/mod.rs +++ b/src/frontend/src/expr/mod.rs @@ -34,8 +34,11 @@ mod utils; pub use agg_call::{AggCall, AggOrderBy, AggOrderByExpr}; pub use correlated_input_ref::CorrelatedInputRef; -pub use function_call::FunctionCall; -pub use input_ref::{as_alias_display, input_ref_to_column_indices, InputRef, InputRefDisplay}; +pub use function_call::{FunctionCall, FunctionCallVerboseDisplay}; +pub use input_ref::{ + as_alias_display, input_ref_to_column_indices, InputRef, InputRefDisplay, + InputRefVerboseDisplay, +}; pub use literal::Literal; pub use subquery::{Subquery, SubqueryKind}; @@ -489,6 +492,39 @@ impl std::fmt::Debug for ExprImpl { } } +pub struct ExprVerboseDisplay<'a> { + pub expr: &'a ExprImpl, + pub input_schema: &'a Schema, +} + +impl std::fmt::Debug for ExprVerboseDisplay<'_> { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let that = self.expr; + match that { + ExprImpl::InputRef(x) => write!( + f, + "{:?}", + InputRefVerboseDisplay { + input_ref: x, + input_schema: self.input_schema + } + ), + ExprImpl::Literal(x) => write!(f, "{:?}", x), + ExprImpl::FunctionCall(x) => write!( + f, + "{:?}", + FunctionCallVerboseDisplay { + function_call: x, + input_schema: self.input_schema + } + ), + ExprImpl::AggCall(x) => write!(f, "{:?}", x), + ExprImpl::Subquery(x) => write!(f, "{:?}", x), + ExprImpl::CorrelatedInputRef(x) => write!(f, "{:?}", x), + } + } +} + #[cfg(test)] /// Asserts that the expression is an [`InputRef`] with the given index. macro_rules! assert_eq_input_ref { @@ -502,6 +538,7 @@ macro_rules! assert_eq_input_ref { #[cfg(test)] pub(crate) use assert_eq_input_ref; +use risingwave_common::catalog::Schema; use crate::utils::Condition; diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index 90bdd613ac44e..3289e63bc312f 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::atomic::Ordering; + use pgwire::pg_field_descriptor::{PgFieldDescriptor, TypeOid}; use pgwire::pg_response::{PgResponse, StatementType}; use pgwire::types::Row; @@ -29,12 +31,12 @@ use crate::session::OptimizerContext; pub(super) fn handle_explain( context: OptimizerContext, stmt: Statement, - _verbose: bool, + verbose: bool, ) -> Result { let session = context.session_ctx.clone(); + context.explain_verbose.store(verbose, Ordering::Release); // bind, plan, optimize, and serialize here let mut planner = Planner::new(context.into()); - let plan = match stmt { Statement::CreateView { or_replace: false, diff --git a/src/frontend/src/optimizer/plan_node/batch_exchange.rs b/src/frontend/src/optimizer/plan_node/batch_exchange.rs index c99a242444638..6dcbd95b7ae30 100644 --- a/src/frontend/src/optimizer/plan_node/batch_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/batch_exchange.rs @@ -20,7 +20,9 @@ use risingwave_pb::batch_plan::{ExchangeNode, MergeSortExchangeNode}; use super::{PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; use crate::optimizer::plan_node::ToLocalBatch; -use crate::optimizer::property::{Distribution, Order}; +use crate::optimizer::property::{ + Distribution, DistributionVerboseDisplay, Order, OrderVerboseDisplay, +}; /// `BatchExchange` imposes a particular distribution on its input /// without changing its content. @@ -42,11 +44,27 @@ impl BatchExchange { impl fmt::Display for BatchExchange { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!( - f, - "BatchExchange {{ order: {}, dist: {:?} }}", - self.base.order, self.base.dist - ) + let verbose = self.base.ctx.is_explain_verbose(); + if verbose { + write!( + f, + "BatchExchange {{ order: {}, dist: {} }}", + OrderVerboseDisplay { + order: &self.base.order, + input_schema: self.input.schema() + }, + DistributionVerboseDisplay { + distribution: &self.base.dist, + input_schema: self.input.schema() + } + ) + } else { + write!( + f, + "BatchExchange {{ order: {}, dist: {:?} }}", + self.base.order, self.base.dist + ) + } } } diff --git a/src/frontend/src/optimizer/plan_node/batch_expand.rs b/src/frontend/src/optimizer/plan_node/batch_expand.rs index ef719f48a1d2f..41fffe50bd750 100644 --- a/src/frontend/src/optimizer/plan_node/batch_expand.rs +++ b/src/frontend/src/optimizer/plan_node/batch_expand.rs @@ -15,6 +15,7 @@ use std::fmt; use itertools::Itertools; +use risingwave_common::catalog::FieldVerboseDisplay; use risingwave_common::error::Result; use risingwave_pb::batch_plan::expand_node::Subset; use risingwave_pb::batch_plan::plan_node::NodeBody; @@ -47,15 +48,28 @@ impl BatchExpand { pub fn column_subsets(&self) -> &Vec> { self.logical.column_subsets() } + + pub fn column_subsets_verbose_display(&self) -> Vec> { + self.logical.column_subsets_verbose_display() + } } impl fmt::Display for BatchExpand { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!( - f, - "BatchExpand {{ column_subsets: {:?} }}", - self.column_subsets() - ) + let verbose = self.base.ctx.is_explain_verbose(); + if verbose { + write!( + f, + "BatchExpand {{ column_subsets: {:?} }}", + self.column_subsets_verbose_display() + ) + } else { + write!( + f, + "BatchExpand {{ column_subsets: {:?} }}", + self.column_subsets() + ) + } } } diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index 800a7a9918338..3b401b79b338d 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -21,7 +21,7 @@ use risingwave_pb::batch_plan::FilterNode; use super::{LogicalFilter, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; use crate::expr::{Expr, ExprImpl}; use crate::optimizer::plan_node::{PlanBase, ToLocalBatch}; -use crate::utils::Condition; +use crate::utils::{Condition, ConditionVerboseDisplay}; /// `BatchFilter` implements [`super::LogicalFilter`] #[derive(Debug, Clone)] @@ -50,7 +50,21 @@ impl BatchFilter { impl fmt::Display for BatchFilter { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "BatchFilter {{ predicate: {} }}", self.predicate()) + let verbose = self.base.ctx.is_explain_verbose(); + if verbose { + let input = self.input(); + let input_schema = input.schema(); + write!( + f, + "BatchFilter {{ predicate: {} }}", + ConditionVerboseDisplay { + condition: self.logical.predicate(), + input_schema + } + ) + } else { + write!(f, "BatchFilter {{ predicate: {} }}", self.predicate()) + } } } 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 a256124c25c82..25388c9f27f66 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -14,7 +14,7 @@ use std::fmt; -use itertools::Itertools; +use risingwave_common::catalog::FieldVerboseDisplay; use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashAggNode; @@ -22,6 +22,7 @@ use risingwave_pb::batch_plan::HashAggNode; use super::logical_agg::PlanAggCall; use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; use crate::expr::InputRefDisplay; +use crate::optimizer::plan_node::logical_agg::PlanAggCallVerboseDisplay; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order, RequiredDist}; @@ -53,22 +54,34 @@ impl BatchHashAgg { pub fn group_key(&self) -> &[usize] { self.logical.group_key() } + + pub fn agg_calls_verbose_display(&self) -> Vec { + self.logical.agg_calls_verbose_display() + } + + pub fn group_key_display(&self) -> Vec { + self.logical.group_key_display() + } + + pub fn group_key_verbose_display(&self) -> Vec { + self.logical.group_key_verbose_display() + } } impl fmt::Display for BatchHashAgg { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_struct("BatchHashAgg") - .field( - "group_key", - &self - .group_key() - .iter() - .copied() - .map(InputRefDisplay) - .collect_vec(), - ) - .field("aggs", &self.agg_calls()) - .finish() + let verbose = self.base.ctx.is_explain_verbose(); + let mut builder = f.debug_struct("BatchHashAgg"); + if verbose { + builder + .field("group_key", &self.group_key_verbose_display()) + .field("aggs", &self.agg_calls_verbose_display()); + } else { + builder + .field("group_key", &self.group_key_display()) + .field("aggs", &self.agg_calls()); + } + builder.finish() } } 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 6a7a3e48216f0..a767fb05fe90b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -14,6 +14,7 @@ use std::fmt; +use risingwave_common::catalog::Schema; use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashJoinNode; @@ -23,7 +24,7 @@ use super::{ ToDistributedBatch, }; use crate::expr::Expr; -use crate::optimizer::plan_node::ToLocalBatch; +use crate::optimizer::plan_node::{EqJoinPredicateVerboseDisplay, ToLocalBatch}; use crate::optimizer::property::{Distribution, Order, RequiredDist}; use crate::utils::ColIndexMapping; @@ -81,11 +82,25 @@ impl BatchHashJoin { impl fmt::Display for BatchHashJoin { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + let verbose = self.base.ctx.is_explain_verbose(); write!( f, "BatchHashJoin {{ type: {:?}, predicate: {}, output_indices: {} }}", self.logical.join_type(), - self.eq_join_predicate(), + if verbose { + let mut concat_schema = self.left().schema().fields.clone(); + concat_schema.extend(self.right().schema().fields.clone()); + let concat_schema = Schema::new(concat_schema); + format!( + "{}", + EqJoinPredicateVerboseDisplay { + eq_join_predicate: self.eq_join_predicate(), + input_schema: &concat_schema + } + ) + } else { + format!("{}", self.eq_join_predicate()) + }, if self .logical .output_indices() 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 7a80bf1c3d85c..091145e21813c 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 @@ -14,6 +14,7 @@ use std::fmt; +use risingwave_common::catalog::Schema; use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::NestedLoopJoinNode; @@ -22,6 +23,7 @@ use super::{LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchProst, To use crate::expr::{Expr, ExprImpl}; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order, RequiredDist}; +use crate::utils::ConditionVerboseDisplay; /// `BatchNestedLoopJoin` implements [`super::LogicalJoin`] by checking the join condition /// against all pairs of rows from inner & outer side within 2 layers of loops. @@ -52,11 +54,25 @@ impl BatchNestedLoopJoin { impl fmt::Display for BatchNestedLoopJoin { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + let verbose = self.base.ctx.is_explain_verbose(); write!( f, "BatchNestedLoopJoin {{ type: {:?}, predicate: {}, output_indices: {} }}", self.logical.join_type(), - self.logical.on(), + if verbose { + let mut concat_schema = self.left().schema().fields.clone(); + concat_schema.extend(self.right().schema().fields.clone()); + let concat_schema = Schema::new(concat_schema); + format!( + "{}", + ConditionVerboseDisplay { + condition: self.logical.on(), + input_schema: &concat_schema + } + ) + } else { + format!("{}", self.logical.on()) + }, if self .logical .output_indices() 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 30dfdda1b96df..0899aa79c0782 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -121,15 +121,24 @@ impl fmt::Display for BatchSeqScan { } } + let verbose = self.base.ctx.is_explain_verbose(); + if self.scan_range.is_full_table_scan() { write!( f, "BatchScan {{ table: {}, columns: [{}] }}", self.logical.table_name(), - self.logical.column_names().join(", ") + match verbose { + true => self.logical.column_names_with_table_prefix(), + false => self.logical.column_names(), + } + .join(", ") ) } else { - let order_names = self.logical.order_names(); + let order_names = match verbose { + true => self.logical.order_names_with_table_prefix(), + false => self.logical.order_names(), + }; #[expect(clippy::disallowed_methods)] let mut range_str = self .scan_range @@ -147,7 +156,11 @@ impl fmt::Display for BatchSeqScan { f, "BatchScan {{ table: {}, columns: [{}], scan_range: [{}] }}", self.logical.table_name(), - self.logical.column_names().join(", "), + match verbose { + true => self.logical.column_names_with_table_prefix(), + false => self.logical.column_names(), + } + .join(", "), range_str.join(", ") ) } 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 fdcc0fad4e36b..62ce1201f7f37 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -20,7 +20,7 @@ use risingwave_pb::batch_plan::SortAggNode; use super::logical_agg::PlanAggCall; use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; -use crate::optimizer::plan_node::{BatchExchange, ToLocalBatch}; +use crate::optimizer::plan_node::{BatchExchange, PlanAggCallVerboseDisplay, ToLocalBatch}; use crate::optimizer::property::{Distribution, Order, RequiredDist}; #[derive(Debug, Clone)] @@ -46,13 +46,22 @@ impl BatchSimpleAgg { pub fn agg_calls(&self) -> &[PlanAggCall] { self.logical.agg_calls() } + + pub fn agg_calls_verbose_display(&self) -> Vec { + self.logical.agg_calls_verbose_display() + } } impl fmt::Display for BatchSimpleAgg { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_struct("BatchSimpleAgg") - .field("aggs", &self.agg_calls()) - .finish() + let verbose = self.base.ctx.is_explain_verbose(); + let mut builder = f.debug_struct("BatchSimpleAgg"); + if verbose { + builder.field("aggs", &self.agg_calls_verbose_display()); + } else { + builder.field("aggs", &self.agg_calls()); + } + builder.finish() } } @@ -75,7 +84,11 @@ impl ToDistributedBatch for BatchSimpleAgg { // TODO: distinct agg cannot use 2-phase agg yet. if dist_input.distribution().satisfies(&RequiredDist::AnyShard) - && self.logical.agg_calls().iter().any(|call| !call.distinct) + && self + .logical + .agg_calls() + .iter() + .all(|call| !call.distinct && call.order_by_fields.is_empty()) { // partial agg let partial_agg = self.clone_with_input(dist_input).into(); diff --git a/src/frontend/src/optimizer/plan_node/batch_sort.rs b/src/frontend/src/optimizer/plan_node/batch_sort.rs index 708dfa92383f1..23562305925ef 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort.rs @@ -20,7 +20,7 @@ use risingwave_pb::batch_plan::OrderByNode; use super::{PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; use crate::optimizer::plan_node::ToLocalBatch; -use crate::optimizer::property::Order; +use crate::optimizer::property::{Order, OrderVerboseDisplay}; /// `BatchSort` buffers all data from input and sort these rows by specified order, providing the /// collation required by user or parent plan node. @@ -42,7 +42,19 @@ impl BatchSort { impl fmt::Display for BatchSort { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "BatchSort {{ order: {} }}", self.order()) + let verbose = self.base.ctx.is_explain_verbose(); + if verbose { + write!( + f, + "BatchSort {{ order: {} }}", + OrderVerboseDisplay { + order: self.order(), + input_schema: self.input.schema() + } + ) + } else { + write!(f, "BatchSort {{ order: {} }}", self.order()) + } } } diff --git a/src/frontend/src/optimizer/plan_node/batch_topn.rs b/src/frontend/src/optimizer/plan_node/batch_topn.rs index 8085e90d69c48..101dfa3721b73 100644 --- a/src/frontend/src/optimizer/plan_node/batch_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_topn.rs @@ -20,7 +20,7 @@ use risingwave_pb::batch_plan::TopNNode; use super::{LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch}; use crate::optimizer::plan_node::ToLocalBatch; -use crate::optimizer::property::{Order, RequiredDist}; +use crate::optimizer::property::{Order, OrderVerboseDisplay, RequiredDist}; /// `BatchTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone)] @@ -45,10 +45,23 @@ impl BatchTopN { impl fmt::Display for BatchTopN { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + let verbose = self.base.ctx.is_explain_verbose(); write!( f, "BatchTopN {{ order: {}, limit: {}, offset: {} }}", - self.logical.topn_order(), + if verbose { + let input = self.input(); + let input_schema = input.schema(); + format!( + "{}", + OrderVerboseDisplay { + order: self.logical.topn_order(), + input_schema + } + ) + } else { + format!("{}", self.logical.topn_order()) + }, self.logical.limit(), self.logical.offset(), ) 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 bf14079085b85..b0b72ceadd1e8 100644 --- a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs +++ b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs @@ -14,8 +14,10 @@ use std::fmt; -use crate::expr::{ExprType, FunctionCall, InputRef}; -use crate::utils::{ColIndexMapping, Condition}; +use risingwave_common::catalog::Schema; + +use crate::expr::{ExprType, FunctionCall, InputRef, InputRefVerboseDisplay}; +use crate::utils::{ColIndexMapping, Condition, ConditionVerboseDisplay}; /// The join predicate used in optimizer #[derive(Debug, Clone)] @@ -157,3 +159,67 @@ impl EqJoinPredicate { ColIndexMapping::new(map) } } + +pub struct EqJoinPredicateVerboseDisplay<'a> { + pub eq_join_predicate: &'a EqJoinPredicate, + pub input_schema: &'a Schema, +} + +impl EqJoinPredicateVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter) -> std::fmt::Result { + let that = self.eq_join_predicate; + let mut eq_keys = that.eq_keys().iter(); + if let Some((k1, k2)) = eq_keys.next() { + write!( + f, + "{} = {}", + InputRefVerboseDisplay { + input_ref: k1, + input_schema: self.input_schema + }, + InputRefVerboseDisplay { + input_ref: k2, + input_schema: self.input_schema + } + )?; + } + for (k1, k2) in eq_keys { + write!( + f, + " AND {} = {}", + InputRefVerboseDisplay { + input_ref: k1, + input_schema: self.input_schema + }, + InputRefVerboseDisplay { + input_ref: k2, + input_schema: self.input_schema + } + )?; + } + if !that.other_cond.always_true() { + write!( + f, + " AND {}", + ConditionVerboseDisplay { + condition: &that.other_cond, + input_schema: self.input_schema + } + )?; + } + + Ok(()) + } +} + +impl fmt::Display for EqJoinPredicateVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter) -> std::fmt::Result { + self.fmt(f) + } +} + +impl fmt::Debug for EqJoinPredicateVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter) -> std::fmt::Result { + self.fmt(f) + } +} diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 456afefd9e645..78bbb2590f1e7 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -17,7 +17,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::catalog::{Field, FieldVerboseDisplay, Schema}; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; @@ -30,11 +30,14 @@ use super::{ StreamLocalSimpleAgg, ToBatch, ToStream, }; use crate::catalog::table_catalog::TableCatalog; -use crate::expr::{AggCall, Expr, ExprImpl, ExprRewriter, ExprType, FunctionCall, InputRef}; +use crate::expr::{ + AggCall, Expr, ExprImpl, ExprRewriter, ExprType, FunctionCall, InputRef, InputRefDisplay, + InputRefVerboseDisplay, +}; use crate::optimizer::plan_node::utils::TableCatalogBuilder; use crate::optimizer::plan_node::{gen_filter_and_pushdown, LogicalProject}; use crate::optimizer::property::{Direction, Order, RequiredDist}; -use crate::utils::{ColIndexMapping, Condition, Substitute}; +use crate::utils::{ColIndexMapping, Condition, ConditionVerboseDisplay, Substitute}; /// See also [`crate::expr::AggOrderByExpr`] /// TODO(yuchao): replace `PlanAggOrderByField` with enhanced `FieldOrder` @@ -62,6 +65,37 @@ impl fmt::Debug for PlanAggOrderByField { } } +#[derive(Clone)] +pub struct PlanAggOrderByFieldVerboseDisplay<'a> { + pub plan_agg_order_by_field: &'a PlanAggOrderByField, + pub input_schema: &'a Schema, +} + +impl fmt::Debug for PlanAggOrderByFieldVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let that = self.plan_agg_order_by_field; + write!( + f, + "{:?}", + InputRefVerboseDisplay { + input_ref: &that.input, + input_schema: self.input_schema + } + )?; + match that.direction { + Direction::Asc => write!(f, " ASC")?, + Direction::Desc => write!(f, " DESC")?, + _ => {} + } + write!( + f, + " NULLS {}", + if that.nulls_first { "FIRST" } else { "LAST" } + )?; + Ok(()) + } +} + /// Aggregation Call #[derive(Clone)] pub struct PlanAggCall { @@ -169,6 +203,63 @@ impl PlanAggCall { } } +pub struct PlanAggCallVerboseDisplay<'a> { + pub plan_agg_call: &'a PlanAggCall, + pub input_schema: &'a Schema, +} + +impl fmt::Debug for PlanAggCallVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let that = self.plan_agg_call; + write!(f, "{}", that.agg_kind)?; + if !that.inputs.is_empty() { + write!(f, "(")?; + for (idx, input) in that.inputs.iter().enumerate() { + if idx == 0 && that.distinct { + write!(f, "distinct ")?; + } + write!( + f, + "{}", + self.input_schema.fields.get(input.index).unwrap().name + )?; + if idx != (that.inputs.len() - 1) { + write!(f, ",")?; + } + } + if !that.order_by_fields.is_empty() { + let clause_text = that + .order_by_fields + .iter() + .map(|e| { + format!( + "{:?}", + PlanAggOrderByFieldVerboseDisplay { + plan_agg_order_by_field: e, + input_schema: self.input_schema, + } + ) + }) + .join(", "); + write!(f, " order_by({})", clause_text)?; + } + write!(f, ")")?; + } + + if !that.filter.always_true() { + write!( + f, + " filter({:?})", + ConditionVerboseDisplay { + condition: &that.filter, + input_schema: self.input_schema, + } + )?; + } + Ok(()) + } +} + /// `LogicalAgg` groups input data by their group key and computes aggregation functions. /// /// It corresponds to the `GROUP BY` operator in a SQL query statement together with the aggregate @@ -497,14 +588,7 @@ impl ExprRewriter for LogicalAggBuilder { impl LogicalAgg { pub fn new(agg_calls: Vec, group_key: Vec, input: PlanRef) -> Self { let ctx = input.ctx(); - let schema = Self::derive_schema( - input.schema(), - &group_key, - agg_calls - .iter() - .map(|agg_call| agg_call.return_type.clone()) - .collect(), - ); + let schema = Self::derive_schema(input.schema(), &group_key, &agg_calls); let pk_indices = match group_key.is_empty() { // simple agg true => vec![], @@ -538,24 +622,19 @@ impl LogicalAgg { self.o2i_col_mapping().inverse() } - fn derive_schema( - input: &Schema, - group_key: &[usize], - agg_call_data_types: Vec, - ) -> Schema { + fn derive_schema(input: &Schema, group_key: &[usize], agg_calls: &[PlanAggCall]) -> Schema { let fields = group_key .iter() .cloned() .map(|i| input.fields()[i].clone()) - .chain( - agg_call_data_types - .into_iter() - .enumerate() - .map(|(id, data_type)| { - let name = format!("agg#{}", id); - Field::with_name(data_type, name) - }), - ) + .chain(agg_calls.iter().map(|agg_call| { + let plan_agg_call_display = PlanAggCallVerboseDisplay { + plan_agg_call: agg_call, + input_schema: input, + }; + let name = format!("{:?}", plan_agg_call_display); + Field::with_name(agg_call.return_type.clone(), name) + })) .collect(); Schema { fields } } @@ -596,6 +675,32 @@ impl LogicalAgg { self.agg_calls.as_ref() } + pub fn agg_calls_verbose_display(&self) -> Vec { + self.agg_calls() + .iter() + .map(|plan_agg_call| PlanAggCallVerboseDisplay { + plan_agg_call, + input_schema: self.input.schema(), + }) + .collect_vec() + } + + pub fn group_key_display(&self) -> Vec { + self.group_key() + .iter() + .copied() + .map(InputRefDisplay) + .collect_vec() + } + + pub fn group_key_verbose_display(&self) -> Vec { + self.group_key() + .iter() + .copied() + .map(|i| FieldVerboseDisplay(self.input.schema().fields.get(i).unwrap())) + .collect_vec() + } + /// Get a reference to the logical agg's group key. pub fn group_key(&self) -> &[usize] { self.group_key.as_ref() @@ -798,10 +903,9 @@ impl ToStream for LogicalAgg { // simple-agg if self.group_key().is_empty() { // TODO: Other agg calls will be supported by stateful local agg eventually. - let agg_calls_can_use_two_phase = self - .agg_calls - .iter() - .all(|c| matches!(c.agg_kind, AggKind::Count | AggKind::Sum)); + let agg_calls_can_use_two_phase = self.agg_calls.iter().all(|c| { + matches!(c.agg_kind, AggKind::Count | AggKind::Sum) && c.order_by_fields.is_empty() + }); let input_stream = input.to_stream()?; let input_distribution = input_stream.distribution(); @@ -810,7 +914,7 @@ impl ToStream for LogicalAgg { if input_distribution.satisfies(&RequiredDist::AnyShard) && agg_calls_can_use_two_phase { self.gen_two_phase_streaming_agg_plan(input_stream) - // simple 1-phase-agg + // simple 1-phase-agg } else { Ok(StreamGlobalSimpleAgg::new(self.clone_with_input( input.to_stream_with_dist_required(&RequiredDist::single())?, @@ -818,7 +922,7 @@ impl ToStream for LogicalAgg { .into()) } - // hash-agg + // hash-agg } else { Ok( StreamHashAgg::new(self.clone_with_input(input.to_stream_with_dist_required( @@ -859,7 +963,6 @@ impl ToStream for LogicalAgg { #[cfg(test)] mod tests { - use std::rc::Rc; use risingwave_common::catalog::Field; @@ -1012,6 +1115,7 @@ mod tests { assert_eq!(group_key, vec![0]); } } + /// Generate a agg call node with given [`DataType`] and fields. /// For example, `generate_agg_call(Int32, [v1, v2, v3])` will result in: /// ```text diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index c04dc75c28616..c6e321ded0280 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -15,7 +15,7 @@ use std::fmt; use itertools::Itertools; -use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::catalog::{Field, FieldVerboseDisplay, Schema}; use risingwave_common::types::DataType; use super::{ @@ -71,6 +71,18 @@ impl LogicalExpand { pub fn column_subsets(&self) -> &Vec> { &self.column_subsets } + + pub fn column_subsets_verbose_display(&self) -> Vec> { + self.column_subsets() + .iter() + .map(|subset| { + subset + .iter() + .map(|&i| FieldVerboseDisplay(self.input.schema().fields.get(i).unwrap())) + .collect_vec() + }) + .collect_vec() + } } impl PlanTreeNodeUnary for LogicalExpand { 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 97188d93cfabf..a937a4cf22ccb 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -24,7 +24,7 @@ use super::{ gen_filter_and_pushdown, BatchHopWindow, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, ToBatch, ToStream, }; -use crate::expr::{InputRef, InputRefDisplay}; +use crate::expr::{InputRef, InputRefDisplay, InputRefVerboseDisplay}; use crate::utils::{ColIndexMapping, Condition}; /// `LogicalHopWindow` implements Hop Table Function. @@ -170,11 +170,22 @@ impl LogicalHopWindow { } pub fn fmt_with_name(&self, f: &mut fmt::Formatter, name: &str) -> fmt::Result { + let verbose = self.base.ctx.is_explain_verbose(); write!( f, "{} {{ time_col: {}, slide: {}, size: {}, output_indices: {} }}", name, - InputRefDisplay(self.time_col.index), + if verbose { + format!( + "{}", + InputRefVerboseDisplay { + input_ref: &self.time_col, + input_schema: self.input.schema() + } + ) + } else { + format!("{}", InputRefDisplay(self.time_col.index)) + }, self.window_slide, self.window_size, if self diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index 8e6c8218b589d..844cb6e4fc001 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -25,7 +25,9 @@ use super::{ gen_filter_and_pushdown, BatchProject, ColPrunable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProject, ToBatch, ToStream, }; -use crate::expr::{assert_input_ref, Expr, ExprImpl, ExprRewriter, ExprVisitor, InputRef}; +use crate::expr::{ + assert_input_ref, Expr, ExprImpl, ExprRewriter, ExprVerboseDisplay, ExprVisitor, InputRef, +}; use crate::optimizer::plan_node::CollectInputRef; use crate::optimizer::property::{Distribution, Order, RequiredDist}; use crate::utils::{ColIndexMapping, Condition, Substitute}; @@ -168,7 +170,11 @@ impl LogicalProject { let field = input_schema.fields()[input_idx].clone(); (field.name, field.sub_fields, field.type_name) } - None => (format!("expr#{}", id), vec![], String::new()), + None => ( + format!("{:?}", ExprVerboseDisplay { expr, input_schema }), + vec![], + String::new(), + ), }; Field::with_struct(expr.return_type(), name, sub_fields, type_name) }) @@ -190,7 +196,24 @@ impl LogicalProject { } pub(super) fn fmt_with_name(&self, f: &mut fmt::Formatter, name: &str) -> fmt::Result { - f.debug_struct(name).field("exprs", self.exprs()).finish() + let verbose = self.base.ctx.is_explain_verbose(); + let mut builder = f.debug_struct(name); + if verbose { + builder.field( + "exprs", + &self + .exprs() + .iter() + .map(|expr| ExprVerboseDisplay { + expr, + input_schema: self.input.schema(), + }) + .collect_vec(), + ); + } else { + builder.field("exprs", self.exprs()); + } + builder.finish() } pub fn is_identity(&self) -> bool { diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 691daab6aa11a..be456b0a6c1e1 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -18,7 +18,7 @@ use std::rc::Rc; use fixedbitset::FixedBitSet; use itertools::Itertools; -use risingwave_common::catalog::{ColumnDesc, Schema, TableDesc}; +use risingwave_common::catalog::{ColumnDesc, Field, Schema, TableDesc}; use risingwave_common::error::{ErrorCode, Result, RwError}; use super::{ @@ -75,7 +75,7 @@ impl LogicalScan { .map(|(op_idx, tb_idx)| { let col = &table_desc.columns[*tb_idx]; id_to_op_idx.insert(col.column_id, op_idx); - col.into() + Field::from_with_table_name_prefix(col, &table_name) }) .collect(); @@ -138,6 +138,19 @@ impl LogicalScan { .collect() } + pub(super) fn column_names_with_table_prefix(&self) -> Vec { + self.output_col_idx + .iter() + .map(|i| { + format!( + "{}.{}", + self.table_name.clone(), + self.table_desc.columns[*i].name + ) + }) + .collect() + } + pub(super) fn order_names(&self) -> Vec { self.table_desc .order_column_indices() @@ -146,6 +159,20 @@ impl LogicalScan { .collect() } + pub(super) fn order_names_with_table_prefix(&self) -> Vec { + self.table_desc + .order_column_indices() + .iter() + .map(|&i| { + format!( + "{}.{}", + self.table_name.clone(), + self.table_desc.columns[i].name + ) + }) + .collect() + } + pub fn table_name(&self) -> &str { &self.table_name } diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 69727c09aeb3e..cc3df2fcba0f5 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -279,7 +279,7 @@ pub use batch_table_function::BatchTableFunction; pub use batch_topn::BatchTopN; pub use batch_update::BatchUpdate; pub use batch_values::BatchValues; -pub use logical_agg::{LogicalAgg, PlanAggCall}; +pub use logical_agg::{LogicalAgg, PlanAggCall, PlanAggCallVerboseDisplay}; pub use logical_apply::LogicalApply; pub use logical_delete::LogicalDelete; pub use logical_expand::LogicalExpand; diff --git a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs index c632b758a636b..17f24151e7b5a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -14,14 +14,14 @@ use std::fmt; -use risingwave_common::catalog::ColumnDesc; +use risingwave_common::catalog::{ColumnDesc, Schema}; use risingwave_pb::plan_common::JoinType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ArrangementInfo, DeltaIndexJoinNode}; use super::{LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, StreamHashJoin, ToStreamProst}; use crate::expr::Expr; -use crate::optimizer::plan_node::EqJoinPredicate; +use crate::optimizer::plan_node::{EqJoinPredicate, EqJoinPredicateVerboseDisplay}; /// [`StreamDeltaJoin`] implements [`super::LogicalJoin`] with delta join. It requires its two /// inputs to be indexes. @@ -78,11 +78,25 @@ impl StreamDeltaJoin { impl fmt::Display for StreamDeltaJoin { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + let verbose = self.base.ctx.is_explain_verbose(); write!( f, "StreamDeltaJoin {{ type: {:?}, predicate: {}, output_indices: {} }}", self.logical.join_type(), - self.eq_join_predicate(), + if verbose { + let mut concat_schema = self.left().schema().fields.clone(); + concat_schema.extend(self.right().schema().fields.clone()); + let concat_schema = Schema::new(concat_schema); + format!( + "{}", + EqJoinPredicateVerboseDisplay { + eq_join_predicate: self.eq_join_predicate(), + input_schema: &concat_schema + } + ) + } else { + format!("{}", self.eq_join_predicate()) + }, if self .logical .output_indices() 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 5c3f2bde9f4f6..1286f1b3ae3cc 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -13,7 +13,7 @@ // limitations under the License. use std::fmt; -use risingwave_common::catalog::{DatabaseId, SchemaId}; +use risingwave_common::catalog::{DatabaseId, Schema, SchemaId}; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::DynamicFilterNode; @@ -23,7 +23,7 @@ use crate::catalog::TableCatalog; use crate::expr::Expr; use crate::optimizer::plan_node::{PlanBase, PlanTreeNodeBinary, ToStreamProst}; use crate::optimizer::PlanRef; -use crate::utils::Condition; +use crate::utils::{Condition, ConditionVerboseDisplay}; #[derive(Clone, Debug)] pub struct StreamDynamicFilter { @@ -59,7 +59,22 @@ impl StreamDynamicFilter { impl fmt::Display for StreamDynamicFilter { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "StreamDynamicFilter {{ predicate: {} }}", self.predicate) + let verbose = self.base.ctx.is_explain_verbose(); + if verbose { + let mut concat_schema = self.left().schema().fields.clone(); + concat_schema.extend(self.right().schema().fields.clone()); + let concat_schema = Schema::new(concat_schema); + write!( + f, + "StreamDynamicFilter {{ predicate: {} }}", + ConditionVerboseDisplay { + condition: &self.predicate, + input_schema: &concat_schema + } + ) + } else { + write!(f, "StreamDynamicFilter {{ predicate: {} }}", self.predicate) + } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs index 7e36809880510..cee94f63a5947 100644 --- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs @@ -18,7 +18,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{DispatchStrategy, DispatcherType, ExchangeNode}; use super::{PlanBase, PlanRef, PlanTreeNodeUnary, ToStreamProst}; -use crate::optimizer::property::Distribution; +use crate::optimizer::property::{Distribution, DistributionVerboseDisplay}; /// `StreamExchange` imposes a particular distribution on its input /// without changing its content. @@ -46,10 +46,26 @@ impl StreamExchange { impl fmt::Display for StreamExchange { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + let verbose = self.base.ctx.is_explain_verbose(); let mut builder = f.debug_struct("StreamExchange"); - builder - .field("dist", &format_args!("{:?}", self.base.dist)) - .finish() + if verbose { + builder + .field( + "dist", + &format_args!( + "{:?}", + DistributionVerboseDisplay { + distribution: &self.base.dist, + input_schema: self.input.schema() + } + ), + ) + .finish() + } else { + builder + .field("dist", &format_args!("{:?}", self.base.dist)) + .finish() + } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_expand.rs b/src/frontend/src/optimizer/plan_node/stream_expand.rs index 9bbafdf5ad493..3d4563dae5887 100644 --- a/src/frontend/src/optimizer/plan_node/stream_expand.rs +++ b/src/frontend/src/optimizer/plan_node/stream_expand.rs @@ -15,6 +15,7 @@ use std::fmt; use itertools::Itertools; +use risingwave_common::catalog::FieldVerboseDisplay; use risingwave_pb::stream_plan::expand_node::Subset; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::ExpandNode; @@ -43,15 +44,28 @@ impl StreamExpand { pub fn column_subsets(&self) -> &Vec> { self.logical.column_subsets() } + + pub fn column_subsets_verbose_display(&self) -> Vec> { + self.logical.column_subsets_verbose_display() + } } impl fmt::Display for StreamExpand { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!( - f, - "StreamExpand {{ column_subsets: {:?} }}", - self.logical.column_subsets() - ) + let verbose = self.base.ctx.is_explain_verbose(); + if verbose { + write!( + f, + "StreamExpand {{ column_subsets: {:?} }}", + self.column_subsets_verbose_display() + ) + } else { + write!( + f, + "StreamExpand {{ column_subsets: {:?} }}", + self.column_subsets() + ) + } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index 133a0fe95b64f..afc77b90efc0b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -20,7 +20,7 @@ use risingwave_pb::stream_plan::FilterNode; use super::{LogicalFilter, PlanRef, PlanTreeNodeUnary, ToStreamProst}; use crate::expr::{Expr, ExprImpl}; use crate::optimizer::plan_node::PlanBase; -use crate::utils::Condition; +use crate::utils::{Condition, ConditionVerboseDisplay}; /// `StreamFilter` implements [`super::LogicalFilter`] #[derive(Debug, Clone)] @@ -53,7 +53,21 @@ impl StreamFilter { impl fmt::Display for StreamFilter { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "StreamFilter {{ predicate: {} }}", self.predicate()) + let verbose = self.base.ctx.is_explain_verbose(); + if verbose { + let input = self.input(); + let input_schema = input.schema(); + write!( + f, + "StreamFilter {{ predicate: {} }}", + ConditionVerboseDisplay { + condition: self.logical.predicate(), + input_schema + } + ) + } else { + write!(f, "StreamFilter {{ predicate: {} }}", self.predicate()) + } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs index 5cf0a313fbd1e..e782f1884ecf5 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs @@ -20,6 +20,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::logical_agg::PlanAggCall; use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToStreamProst}; +use crate::optimizer::plan_node::PlanAggCallVerboseDisplay; use crate::optimizer::property::Distribution; #[derive(Debug, Clone)] @@ -47,6 +48,10 @@ impl StreamGlobalSimpleAgg { pub fn agg_calls(&self) -> &[PlanAggCall] { self.logical.agg_calls() } + + pub fn agg_calls_verbose_display(&self) -> Vec { + self.logical.agg_calls_verbose_display() + } } impl fmt::Display for StreamGlobalSimpleAgg { @@ -56,7 +61,12 @@ impl fmt::Display for StreamGlobalSimpleAgg { } else { f.debug_struct("StreamGlobalSimpleAgg") }; - builder.field("aggs", &self.agg_calls()); + let verbose = self.base.ctx.is_explain_verbose(); + if verbose { + builder.field("aggs", &self.agg_calls_verbose_display()); + } else { + builder.field("aggs", &self.agg_calls()); + } builder.finish() } } diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index c1eecd86b44c6..f6d25e60daf38 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -15,12 +15,13 @@ use std::fmt; use itertools::Itertools; -use risingwave_common::catalog::{DatabaseId, SchemaId}; +use risingwave_common::catalog::{DatabaseId, FieldVerboseDisplay, SchemaId}; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::logical_agg::PlanAggCall; use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToStreamProst}; use crate::expr::InputRefDisplay; +use crate::optimizer::plan_node::PlanAggCallVerboseDisplay; use crate::optimizer::property::Distribution; #[derive(Debug, Clone)] @@ -53,6 +54,18 @@ impl StreamHashAgg { pub fn group_key(&self) -> &[usize] { self.logical.group_key() } + + pub fn agg_calls_verbose_display(&self) -> Vec { + self.logical.agg_calls_verbose_display() + } + + pub fn group_key_display(&self) -> Vec { + self.logical.group_key_display() + } + + pub fn group_key_verbose_display(&self) -> Vec { + self.logical.group_key_verbose_display() + } } impl fmt::Display for StreamHashAgg { @@ -62,18 +75,18 @@ impl fmt::Display for StreamHashAgg { } else { f.debug_struct("StreamHashAgg") }; - builder - .field( - "group_key", - &self - .group_key() - .iter() - .copied() - .map(InputRefDisplay) - .collect_vec(), - ) - .field("aggs", &self.agg_calls()) - .finish() + + let verbose = self.base.ctx.is_explain_verbose(); + if verbose { + builder + .field("group_key", &self.group_key_verbose_display()) + .field("aggs", &self.agg_calls_verbose_display()); + } else { + builder + .field("group_key", &self.group_key_display()) + .field("aggs", &self.agg_calls()); + } + builder.finish() } } diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs index 1cf18db4b9d55..554d7899ca13a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -15,7 +15,7 @@ use std::fmt; use itertools::Itertools; -use risingwave_common::catalog::{DatabaseId, Field, SchemaId}; +use risingwave_common::catalog::{DatabaseId, Field, Schema, SchemaId}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::plan_common::JoinType; @@ -26,7 +26,7 @@ use super::utils::TableCatalogBuilder; use super::{LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, StreamDeltaJoin, ToStreamProst}; use crate::catalog::table_catalog::TableCatalog; use crate::expr::Expr; -use crate::optimizer::plan_node::EqJoinPredicate; +use crate::optimizer::plan_node::{EqJoinPredicate, EqJoinPredicateVerboseDisplay}; use crate::optimizer::property::Distribution; use crate::utils::ColIndexMapping; @@ -128,9 +128,27 @@ impl fmt::Display for StreamHashJoin { } else { f.debug_struct("StreamHashJoin") }; - builder - .field("type", &format_args!("{:?}", self.logical.join_type())) - .field("predicate", &format_args!("{}", self.eq_join_predicate())); + + let verbose = self.base.ctx.is_explain_verbose(); + builder.field("type", &format_args!("{:?}", self.logical.join_type())); + + if verbose { + let mut concat_schema = self.left().schema().fields.clone(); + concat_schema.extend(self.right().schema().fields.clone()); + let concat_schema = Schema::new(concat_schema); + builder.field( + "predicate", + &format_args!( + "{}", + EqJoinPredicateVerboseDisplay { + eq_join_predicate: self.eq_join_predicate(), + input_schema: &concat_schema + } + ), + ); + } else { + builder.field("predicate", &format_args!("{}", self.eq_join_predicate())); + } if self.append_only() { builder.field("append_only", &format_args!("{}", true)); diff --git a/src/frontend/src/optimizer/plan_node/stream_index_scan.rs b/src/frontend/src/optimizer/plan_node/stream_index_scan.rs index b882680b5c957..b9dea9be30ad5 100644 --- a/src/frontend/src/optimizer/plan_node/stream_index_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_index_scan.rs @@ -66,11 +66,17 @@ impl_plan_tree_node_for_leaf! { StreamIndexScan } impl fmt::Display for StreamIndexScan { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + let verbose = self.base.ctx.is_explain_verbose(); write!( f, "StreamIndexScan {{ index: {}, columns: [{}], pk_indices: {:?} }}", self.logical.table_name(), - self.logical.column_names().join(", "), + if verbose { + self.logical.column_names_with_table_prefix() + } else { + self.logical.column_names() + } + .join(", "), self.base.pk_indices ) } diff --git a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs index 9a3ed1dd1bd34..9d66975c52c47 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs @@ -20,6 +20,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::logical_agg::PlanAggCall; use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToStreamProst}; +use crate::optimizer::plan_node::PlanAggCallVerboseDisplay; use crate::optimizer::property::RequiredDist; #[derive(Debug, Clone)] @@ -53,12 +54,21 @@ impl StreamLocalSimpleAgg { pub fn agg_calls(&self) -> &[PlanAggCall] { self.logical.agg_calls() } + + pub fn agg_calls_verbose_display(&self) -> Vec { + self.logical.agg_calls_verbose_display() + } } impl fmt::Display for StreamLocalSimpleAgg { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + let verbose = self.base.ctx.is_explain_verbose(); let mut builder = f.debug_struct("StreamLocalSimpleAgg"); - builder.field("aggs", &self.agg_calls()); + if verbose { + builder.field("aggs", &self.agg_calls_verbose_display()); + } else { + builder.field("aggs", &self.agg_calls()); + } builder.finish() } } diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index c44a2499f23a6..354c1ce3eae2a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -30,8 +30,7 @@ pub struct StreamProject { impl fmt::Display for StreamProject { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - let mut builder = f.debug_struct("StreamProject"); - builder.field("exprs", self.logical.exprs()).finish() + self.logical.fmt_with_name(f, "StreamProject") } } diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index 55f232d2a51bb..977b9515ee737 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -71,12 +71,20 @@ impl_plan_tree_node_for_leaf! { StreamTableScan } impl fmt::Display for StreamTableScan { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + let verbose = self.base.ctx.is_explain_verbose(); let mut builder = f.debug_struct("StreamTableScan"); builder .field("table", &format_args!("{}", self.logical.table_name())) .field( "columns", - &format_args!("[{}]", self.logical.column_names().join(", ")), + &format_args!( + "[{}]", + match verbose { + false => self.logical.column_names(), + true => self.logical.column_names_with_table_prefix(), + } + .join(", ") + ), ) .field("pk_indices", &format_args!("{:?}", self.base.pk_indices)) .finish() diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs index de22a731c4eb1..4f2eed019d326 100644 --- a/src/frontend/src/optimizer/plan_node/stream_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs @@ -17,7 +17,7 @@ use std::fmt; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::{LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, ToStreamProst}; -use crate::optimizer::property::{Distribution, FieldOrder}; +use crate::optimizer::property::{Distribution, FieldOrder, OrderVerboseDisplay}; /// `StreamTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone)] @@ -53,8 +53,25 @@ impl fmt::Display for StreamTopN { f.debug_struct("StreamTopN") }; + let verbose = self.base.ctx.is_explain_verbose(); + if verbose { + let input = self.input(); + let input_schema = input.schema(); + builder.field( + "order", + &format!( + "{}", + OrderVerboseDisplay { + order: self.logical.topn_order(), + input_schema + } + ), + ); + } else { + builder.field("order", &format!("{}", self.logical.topn_order())); + } + builder - .field("order", &format_args!("{}", self.logical.topn_order())) .field("limit", &format_args!("{}", self.logical.limit())) .field("offset", &format_args!("{}", self.logical.offset())) .finish() diff --git a/src/frontend/src/optimizer/property/distribution.rs b/src/frontend/src/optimizer/property/distribution.rs index 13f072cc72394..e1ea33cd5c51e 100644 --- a/src/frontend/src/optimizer/property/distribution.rs +++ b/src/frontend/src/optimizer/property/distribution.rs @@ -42,7 +42,11 @@ //! └─────────────┘ x└────────────┘ //! x //! x +use std::fmt; +use std::fmt::Debug; + use fixedbitset::FixedBitSet; +use risingwave_common::catalog::{FieldVerboseDisplay, Schema}; use risingwave_common::error::Result; use risingwave_pb::batch_plan::exchange_info::{ Distribution as DistributionProst, DistributionMode, HashInfo, @@ -139,6 +143,61 @@ impl Distribution { } } +impl fmt::Display for Distribution { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.write_str("[")?; + match self { + Self::Single => f.write_str("Single")?, + Self::SomeShard => f.write_str("SomeShard")?, + Self::Broadcast => f.write_str("Broadcast")?, + Self::HashShard(vec) => { + for key in vec { + std::fmt::Debug::fmt(&key, f)?; + } + } + } + f.write_str("]") + } +} + +pub struct DistributionVerboseDisplay<'a> { + pub distribution: &'a Distribution, + pub input_schema: &'a Schema, +} + +impl DistributionVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let that = self.distribution; + f.write_str("[")?; + match that { + Distribution::Single => f.write_str("Single")?, + Distribution::SomeShard => f.write_str("SomeShard")?, + Distribution::Broadcast => f.write_str("Broadcast")?, + Distribution::HashShard(vec) => { + for &key in vec { + std::fmt::Debug::fmt( + &FieldVerboseDisplay(self.input_schema.fields.get(key).unwrap()), + f, + )?; + } + } + } + f.write_str("]") + } +} + +impl fmt::Debug for DistributionVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.fmt(f) + } +} + +impl fmt::Display for DistributionVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.fmt(f) + } +} + impl RequiredDist { pub fn single() -> Self { Self::PhysicalDist(Distribution::Single) diff --git a/src/frontend/src/optimizer/property/order.rs b/src/frontend/src/optimizer/property/order.rs index da5c8eb260eab..2622513488c85 100644 --- a/src/frontend/src/optimizer/property/order.rs +++ b/src/frontend/src/optimizer/property/order.rs @@ -15,7 +15,7 @@ use std::fmt; use itertools::Itertools; -use risingwave_common::catalog::Schema; +use risingwave_common::catalog::{FieldVerboseDisplay, Schema}; use risingwave_common::error::Result; use risingwave_common::util::sort_util::{OrderPair, OrderType}; use risingwave_pb::plan_common::{ColumnOrder, OrderType as ProstOrderType}; @@ -55,6 +55,41 @@ impl fmt::Display for Order { } } +pub struct OrderVerboseDisplay<'a> { + pub order: &'a Order, + pub input_schema: &'a Schema, +} + +impl OrderVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let that = self.order; + f.write_str("[")?; + for (i, field_order) in that.field_order.iter().enumerate() { + if i > 0 { + f.write_str(", ")?; + } + FieldOrderVerboseDisplay { + field_order, + input_schema: self.input_schema, + } + .fmt(f)?; + } + f.write_str("]") + } +} + +impl fmt::Display for OrderVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.fmt(f) + } +} + +impl fmt::Debug for OrderVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.fmt(f) + } +} + #[derive(Clone, PartialEq)] pub struct FieldOrder { pub index: usize, @@ -67,6 +102,29 @@ impl std::fmt::Debug for FieldOrder { } } +pub struct FieldOrderVerboseDisplay<'a> { + pub field_order: &'a FieldOrder, + pub input_schema: &'a Schema, +} + +impl FieldOrderVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let that = self.field_order; + write!( + f, + "{} {}", + FieldVerboseDisplay(self.input_schema.fields.get(that.index).unwrap()), + that.direct + ) + } +} + +impl fmt::Debug for FieldOrderVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.fmt(f) + } +} + impl FieldOrder { pub fn ascending(index: usize) -> Self { Self { diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 64b7ad43dfedd..cc4c2fd71b8f7 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -16,7 +16,7 @@ use std::fmt::Formatter; use std::io::{Error, ErrorKind}; use std::marker::Sync; use std::path::PathBuf; -use std::sync::atomic::{AtomicI32, Ordering}; +use std::sync::atomic::{AtomicBool, AtomicI32, Ordering}; use std::sync::Arc; use std::time::Duration; @@ -64,6 +64,9 @@ pub struct OptimizerContext { pub next_id: AtomicI32, /// For debugging purposes, store the SQL string in Context pub sql: Arc, + + /// it indicates whether the explain mode is verbose for explain statement + pub explain_verbose: AtomicBool, } #[derive(Clone, Debug)] @@ -92,6 +95,10 @@ impl OptimizerContextRef { let next_id = self.inner.next_id.fetch_add(1, Ordering::Relaxed); PlanNodeId(next_id) } + + pub fn is_explain_verbose(&self) -> bool { + self.inner.explain_verbose.load(Ordering::Acquire) + } } impl OptimizerContext { @@ -100,6 +107,7 @@ impl OptimizerContext { session_ctx, next_id: AtomicI32::new(0), sql, + explain_verbose: AtomicBool::new(false), } } @@ -110,6 +118,7 @@ impl OptimizerContext { session_ctx: Arc::new(SessionImpl::mock()), next_id: AtomicI32::new(0), sql: Arc::from(""), + explain_verbose: AtomicBool::new(false), } .into() } diff --git a/src/frontend/src/utils/condition.rs b/src/frontend/src/utils/condition.rs index d39401f822d7c..4082b00525919 100644 --- a/src/frontend/src/utils/condition.rs +++ b/src/frontend/src/utils/condition.rs @@ -18,11 +18,13 @@ use std::ops::Bound; use fixedbitset::FixedBitSet; use itertools::Itertools; +use risingwave_common::catalog::Schema; use super::ScanRange; use crate::expr::{ factorization_expr, fold_boolean_constant, push_down_not, to_conjunctions, - try_get_bool_constant, ExprImpl, ExprRewriter, ExprType, ExprVisitor, InputRef, + try_get_bool_constant, ExprImpl, ExprRewriter, ExprType, ExprVerboseDisplay, ExprVisitor, + InputRef, }; #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -446,6 +448,55 @@ impl Condition { } } +pub struct ConditionVerboseDisplay<'a> { + pub condition: &'a Condition, + pub input_schema: &'a Schema, +} + +impl ConditionVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let that = self.condition; + let mut conjunctions = that.conjunctions.iter(); + if let Some(expr) = conjunctions.next() { + write!( + f, + "{:?}", + ExprVerboseDisplay { + expr, + input_schema: self.input_schema + } + )?; + } + if that.always_true() { + write!(f, "true")?; + } else { + for expr in conjunctions { + write!( + f, + " AND {:?}", + ExprVerboseDisplay { + expr, + input_schema: self.input_schema + } + )?; + } + } + Ok(()) + } +} + +impl fmt::Display for ConditionVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.fmt(f) + } +} + +impl fmt::Debug for ConditionVerboseDisplay<'_> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.fmt(f) + } +} + #[cfg(test)] mod tests { use rand::Rng; diff --git a/src/frontend/test_runner/tests/testdata/agg.yaml b/src/frontend/test_runner/tests/testdata/agg.yaml index f011665c9652d..9109e264a7cb9 100644 --- a/src/frontend/test_runner/tests/testdata/agg.yaml +++ b/src/frontend/test_runner/tests/testdata/agg.yaml @@ -147,7 +147,7 @@ BatchProject { exprs: [($0 + $1)] } BatchScan { table: t, columns: [v1, v2] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), cnt, sum], pk_columns: [] } + StreamMaterialize { columns: [sum(count)(hidden), cnt, sum], pk_columns: [] } StreamGlobalSimpleAgg { aggs: [sum($0), sum($1), sum($2)] } StreamExchange { dist: Single } StreamLocalSimpleAgg { aggs: [count, count($0), sum($0)] } @@ -269,7 +269,7 @@ BatchSimpleAgg { aggs: [sum($0)] } BatchScan { table: t, columns: [v1] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), s1], pk_columns: [] } + StreamMaterialize { columns: [count(hidden), s1], pk_columns: [] } StreamGlobalSimpleAgg { aggs: [count, sum($1)] } StreamGlobalSimpleAgg { aggs: [sum($0), sum($1)] } StreamExchange { dist: Single } @@ -285,7 +285,7 @@ BatchSimpleAgg { aggs: [sum($0)] } BatchScan { table: t, columns: [v1] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), s1], pk_columns: [] } + StreamMaterialize { columns: [sum(count)(hidden), s1], pk_columns: [] } StreamGlobalSimpleAgg { aggs: [sum($0), sum($1)] } StreamExchange { dist: Single } StreamLocalSimpleAgg { aggs: [count, sum($0)] } @@ -303,7 +303,7 @@ BatchExchange { order: [], dist: HashShard([0]) } BatchScan { table: t, columns: [v1, v2] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), s1], pk_columns: [] } + StreamMaterialize { columns: [sum(count)(hidden), s1], pk_columns: [] } StreamGlobalSimpleAgg { aggs: [sum($0), sum($1)] } StreamExchange { dist: Single } StreamLocalSimpleAgg { aggs: [count, sum($0)] } @@ -316,7 +316,7 @@ create table t(v1 int, v2 int) with ('appendonly' = true); select min(v1) as a1, max(v2) as a2 from t; stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), a1, a2], pk_columns: [] } + StreamMaterialize { columns: [count(hidden), a1, a2], pk_columns: [] } StreamAppendOnlyGlobalSimpleAgg { aggs: [count, min($0), max($1)] } StreamExchange { dist: Single } StreamTableScan { table: t, columns: [v1, v2, _row_id], pk_indices: [2] } @@ -325,7 +325,7 @@ create table t(v1 int, v2 int) with ('appendonly' = false); select sum(v1) as s1, min(v1) as a1, max(v2) as a2 from t; stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), s1, a1, a2], pk_columns: [] } + StreamMaterialize { columns: [count(hidden), s1, a1, a2], pk_columns: [] } StreamGlobalSimpleAgg { aggs: [count, sum($0), min($0), max($1)] } StreamExchange { dist: Single } StreamTableScan { table: t, columns: [v1, v2, _row_id], pk_indices: [2] } @@ -384,11 +384,10 @@ LogicalAgg { group_key: [], agg_calls: [sum($0 order_by($0 ASC NULLS LAST))] } LogicalScan { table: t, columns: [v1] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), s1], pk_columns: [] } - StreamGlobalSimpleAgg { aggs: [sum($0), sum($1 order_by($0 ASC NULLS LAST))] } + StreamMaterialize { columns: [count(hidden), s1], pk_columns: [] } + StreamGlobalSimpleAgg { aggs: [count, sum($0 order_by($0 ASC NULLS LAST))] } StreamExchange { dist: Single } - StreamLocalSimpleAgg { aggs: [count, sum($0 order_by($0 ASC NULLS LAST))] } - StreamTableScan { table: t, columns: [v1, _row_id], pk_indices: [1] } + StreamTableScan { table: t, columns: [v1, _row_id], pk_indices: [1] } - sql: | /* order by other columns */ create table t(v1 int, v2 varchar); @@ -402,11 +401,10 @@ LogicalAgg { group_key: [], agg_calls: [sum($0 order_by($1 ASC NULLS LAST))] } LogicalScan { table: t, columns: [v1, v2] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), s1], pk_columns: [] } - StreamGlobalSimpleAgg { aggs: [sum($0), sum($1 order_by($1 ASC NULLS LAST))] } + StreamMaterialize { columns: [count(hidden), s1], pk_columns: [] } + StreamGlobalSimpleAgg { aggs: [count, sum($0 order_by($1 ASC NULLS LAST))] } StreamExchange { dist: Single } - StreamLocalSimpleAgg { aggs: [count, sum($0 order_by($1 ASC NULLS LAST))] } - StreamTableScan { table: t, columns: [v1, v2, _row_id], pk_indices: [2] } + StreamTableScan { table: t, columns: [v1, v2, _row_id], pk_indices: [2] } - sql: | /* order by ASC/DESC and default */ create table t(v1 int, v2 varchar, v3 int); @@ -420,11 +418,10 @@ LogicalAgg { group_key: [], agg_calls: [sum($0 order_by($0 ASC NULLS LAST, $1 ASC NULLS LAST, $2 DESC NULLS FIRST))] } LogicalScan { table: t, columns: [v1, v2, v3] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), s1], pk_columns: [] } - StreamGlobalSimpleAgg { aggs: [sum($0), sum($1 order_by($0 ASC NULLS LAST, $1 ASC NULLS LAST, $2 DESC NULLS FIRST))] } + StreamMaterialize { columns: [count(hidden), s1], pk_columns: [] } + StreamGlobalSimpleAgg { aggs: [count, sum($0 order_by($0 ASC NULLS LAST, $1 ASC NULLS LAST, $2 DESC NULLS FIRST))] } StreamExchange { dist: Single } - StreamLocalSimpleAgg { aggs: [count, sum($0 order_by($0 ASC NULLS LAST, $1 ASC NULLS LAST, $2 DESC NULLS FIRST))] } - StreamTableScan { table: t, columns: [v1, v2, v3, _row_id], pk_indices: [3] } + StreamTableScan { table: t, columns: [v1, v2, v3, _row_id], pk_indices: [3] } - sql: | /* order by NULLS FIRST/LAST and default */ create table t(v1 int, v2 varchar, v3 int); @@ -438,11 +435,10 @@ LogicalAgg { group_key: [], agg_calls: [sum($0 order_by($0 ASC NULLS LAST, $1 ASC NULLS FIRST, $2 ASC NULLS LAST))] } LogicalScan { table: t, columns: [v1, v2, v3] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), s1], pk_columns: [] } - StreamGlobalSimpleAgg { aggs: [sum($0), sum($1 order_by($0 ASC NULLS LAST, $1 ASC NULLS FIRST, $2 ASC NULLS LAST))] } + StreamMaterialize { columns: [count(hidden), s1], pk_columns: [] } + StreamGlobalSimpleAgg { aggs: [count, sum($0 order_by($0 ASC NULLS LAST, $1 ASC NULLS FIRST, $2 ASC NULLS LAST))] } StreamExchange { dist: Single } - StreamLocalSimpleAgg { aggs: [count, sum($0 order_by($0 ASC NULLS LAST, $1 ASC NULLS FIRST, $2 ASC NULLS LAST))] } - StreamTableScan { table: t, columns: [v1, v2, v3, _row_id], pk_indices: [3] } + StreamTableScan { table: t, columns: [v1, v2, v3, _row_id], pk_indices: [3] } - sql: | /* order by complex expressions */ create table t(v1 int, v2 varchar, v3 int); @@ -457,12 +453,11 @@ LogicalProject { exprs: [$0, ($0 + $2), (Length($1) * $2)] } LogicalScan { table: t, columns: [v1, v2, v3] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), s1], pk_columns: [] } - StreamGlobalSimpleAgg { aggs: [sum($0), sum($1 order_by($1 ASC NULLS LAST, $2 DESC NULLS FIRST))] } + StreamMaterialize { columns: [count(hidden), s1], pk_columns: [] } + StreamGlobalSimpleAgg { aggs: [count, sum($0 order_by($1 ASC NULLS LAST, $2 DESC NULLS FIRST))] } StreamExchange { dist: Single } - StreamLocalSimpleAgg { aggs: [count, sum($0 order_by($1 ASC NULLS LAST, $2 DESC NULLS FIRST))] } - StreamProject { exprs: [$0, ($0 + $2), (Length($1) * $2), $3] } - StreamTableScan { table: t, columns: [v1, v2, v3, _row_id], pk_indices: [3] } + StreamProject { exprs: [$0, ($0 + $2), (Length($1) * $2), $3] } + StreamTableScan { table: t, columns: [v1, v2, v3, _row_id], pk_indices: [3] } - sql: | /* filter clause */ create table t(v1 int); @@ -476,7 +471,7 @@ LogicalAgg { group_key: [], agg_calls: [sum($0) filter(($0 > 0:Int32))] } LogicalScan { table: t, columns: [v1] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), sa], pk_columns: [] } + StreamMaterialize { columns: [sum(count)(hidden), sa], pk_columns: [] } StreamGlobalSimpleAgg { aggs: [sum($0), sum($1)] } StreamExchange { dist: Single } StreamLocalSimpleAgg { aggs: [count, sum($0) filter(($0 > 0:Int32))] } @@ -509,7 +504,7 @@ LogicalProject { exprs: [$0, $1, ($0 * $1)] } LogicalScan { table: t, columns: [a, b] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), sab], pk_columns: [] } + StreamMaterialize { columns: [count(hidden), sab], pk_columns: [] } StreamGlobalSimpleAgg { aggs: [count, max($2) filter(((($0 < $1) AND (($0 + $1) < 100:Int32)) AND (($0 * $1) <> (($0 + $1) - 1:Int32))))] } StreamExchange { dist: Single } StreamProject { exprs: [$0, $1, ($0 * $1), $2] } @@ -529,7 +524,7 @@ LogicalProject { exprs: [$1, $0] } LogicalScan { table: t, columns: [a, b] } stream_plan: | - StreamMaterialize { columns: [avga, b(hidden)], pk_columns: [b] } + StreamMaterialize { columns: [avga, t.b(hidden)], pk_columns: [t.b] } StreamProject { exprs: [($2::Decimal / $3), $0] } StreamHashAgg { group_key: [$0], aggs: [count, sum($1) filter(($1 > $0)), count($1) filter(($1 > $0))] } StreamExchange { dist: HashShard([0]) } @@ -548,7 +543,7 @@ LogicalAgg { group_key: [], agg_calls: [count filter(($0 > $1))] } LogicalScan { table: t, columns: [a, b] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), cnt_agb], pk_columns: [] } + StreamMaterialize { columns: [sum(count)(hidden), cnt_agb], pk_columns: [] } StreamGlobalSimpleAgg { aggs: [sum($0), sum($1)] } StreamExchange { dist: Single } StreamLocalSimpleAgg { aggs: [count, count filter(($0 > $1))] } @@ -583,7 +578,7 @@ BatchSimpleAgg { aggs: [sum($0) filter(($0 < 5:Int32))] } BatchScan { table: t, columns: [v2] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), b], pk_columns: [] } + StreamMaterialize { columns: [sum(count)(hidden), b], pk_columns: [] } StreamGlobalSimpleAgg { aggs: [sum($0), sum($1)] } StreamExchange { dist: Single } StreamLocalSimpleAgg { aggs: [count, sum($0) filter(($0 < 5:Int32))] } @@ -607,7 +602,7 @@ LogicalExpand { column_subsets: [[0, 2], [0, 1]] } LogicalScan { table: t, columns: [a, b, c] } stream_plan: | - StreamMaterialize { columns: [a, agg#0(hidden), distinct_b_num, sum_c], pk_columns: [a] } + StreamMaterialize { columns: [a, count(hidden), distinct_b_num, sum_c], pk_columns: [a] } StreamHashAgg { group_key: [$0], aggs: [count, count($1) filter(($2 = 1:Int64)), sum($4) filter(($2 = 0:Int64))] } StreamExchange { dist: HashShard([0]) } StreamHashAgg { group_key: [$0, $1, $4], aggs: [count, sum($2)] } @@ -646,9 +641,8 @@ LogicalProject { exprs: [$1, (Length($0) * $1), (Length($0) + $1)] } LogicalScan { table: t, columns: [a, b] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), s1], pk_columns: [] } - StreamGlobalSimpleAgg { aggs: [sum($0), sum($1 order_by($2 ASC NULLS LAST))] } + StreamMaterialize { columns: [count(hidden), s1], pk_columns: [] } + StreamGlobalSimpleAgg { aggs: [count, sum($1 order_by($2 ASC NULLS LAST)) filter((($0 < 100:Int32) AND (($0 * 2:Int32) > 10:Int32)))] } StreamExchange { dist: Single } - StreamLocalSimpleAgg { aggs: [count, sum($1 order_by($2 ASC NULLS LAST)) filter((($0 < 100:Int32) AND (($0 * 2:Int32) > 10:Int32)))] } - StreamProject { exprs: [$1, (Length($0) * $1), (Length($0) + $1), $2] } - StreamTableScan { table: t, columns: [a, b, _row_id], pk_indices: [2] } + StreamProject { exprs: [$1, (Length($0) * $1), (Length($0) + $1), $2] } + StreamTableScan { table: t, columns: [a, b, _row_id], pk_indices: [2] } diff --git a/src/frontend/test_runner/tests/testdata/append_only.yaml b/src/frontend/test_runner/tests/testdata/append_only.yaml index 3944b6f7e903a..b3a2109c89638 100644 --- a/src/frontend/test_runner/tests/testdata/append_only.yaml +++ b/src/frontend/test_runner/tests/testdata/append_only.yaml @@ -3,7 +3,7 @@ create table t1 (v1 int, v2 int) with ('appendonly' = true); select v1, max(v2) as mx2 from t1 group by v1; stream_plan: | - StreamMaterialize { columns: [v1, agg#0(hidden), mx2], pk_columns: [v1] } + StreamMaterialize { columns: [v1, count(hidden), mx2], pk_columns: [v1] } StreamAppendOnlyHashAgg { group_key: [$0], aggs: [count, max($1)] } StreamExchange { dist: HashShard([0]) } StreamTableScan { table: t1, columns: [v1, v2, _row_id], pk_indices: [2] } @@ -12,7 +12,7 @@ create table t2 (v1 int, v3 int) with ('appendonly' = true); select t1.v1 as id, v2, v3 from t1 join t2 on t1.v1=t2.v1; stream_plan: | - StreamMaterialize { columns: [id, v2, v3, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1] } + StreamMaterialize { columns: [id, v2, v3, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id] } StreamExchange { dist: HashShard([3, 4]) } StreamAppendOnlyHashJoin { type: Inner, predicate: $0 = $3, append_only: true, output_indices: [0, 1, 4, 2, 5] } StreamExchange { dist: HashShard([0]) } @@ -23,15 +23,15 @@ create table t1 (v1 int, v2 int) with ('appendonly' = true); select v1 from t1 order by v1 limit 3 offset 3; stream_plan: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id], order_descs: [v1, _row_id] } - StreamAppendOnlyTopN { order: [$0 ASC], limit: 3, offset: 3 } + StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id], order_descs: [v1, t1._row_id] } + StreamAppendOnlyTopN { order: "[$0 ASC]", limit: 3, offset: 3 } StreamExchange { dist: Single } StreamTableScan { table: t1, columns: [v1, _row_id], pk_indices: [1] } - sql: | create table t1 (v1 int, v2 int) with ('appendonly' = true); select max(v1) as max_v1 from t1; stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), max_v1], pk_columns: [] } + StreamMaterialize { columns: [count(hidden), max_v1], pk_columns: [] } StreamAppendOnlyGlobalSimpleAgg { aggs: [count, max($0)] } StreamExchange { dist: Single } StreamTableScan { table: t1, columns: [v1, _row_id], pk_indices: [1] } diff --git a/src/frontend/test_runner/tests/testdata/basic_query.yaml b/src/frontend/test_runner/tests/testdata/basic_query.yaml index 3b8db758ac30e..cbe9935d53603 100644 --- a/src/frontend/test_runner/tests/testdata/basic_query.yaml +++ b/src/frontend/test_runner/tests/testdata/basic_query.yaml @@ -11,7 +11,7 @@ BatchExchange { order: [], dist: Single } BatchScan { table: t, columns: [v1, v2] } stream_plan: | - StreamMaterialize { columns: [v1, v2, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id] } StreamTableScan { table: t, columns: [v1, v2, _row_id], pk_indices: [2] } - sql: | create table t (v1 bigint, v2 double precision); @@ -25,7 +25,7 @@ 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: [] } stream_plan: | - StreamMaterialize { columns: [_row_id(hidden)], pk_columns: [_row_id] } + 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))) } StreamTableScan { table: t, columns: [_row_id], pk_indices: [0] } - sql: | @@ -36,7 +36,7 @@ BatchFilter { predicate: ($0 < 1:Int32) } BatchScan { table: t, columns: [v1] } stream_plan: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id] } StreamFilter { predicate: ($0 < 1:Int32) } StreamTableScan { table: t, columns: [v1, _row_id], pk_indices: [1] } - sql: | @@ -94,7 +94,7 @@ BatchExchange { order: [], dist: Single } BatchScan { table: t, columns: [v1] } stream_plan: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id] } StreamTableScan { table: t, columns: [v1, _row_id], pk_indices: [1] } - sql: select 1 batch_plan: | diff --git a/src/frontend/test_runner/tests/testdata/column_pruning.yaml b/src/frontend/test_runner/tests/testdata/column_pruning.yaml index c0ab9603383b2..2f7fc0a28a6b3 100644 --- a/src/frontend/test_runner/tests/testdata/column_pruning.yaml +++ b/src/frontend/test_runner/tests/testdata/column_pruning.yaml @@ -137,6 +137,6 @@ BatchExchange { order: [], dist: Single } BatchScan { table: t1, columns: [a, created_at] } stream_plan: | - StreamMaterialize { columns: [a, window_end, _row_id(hidden)], pk_columns: [_row_id, window_end] } + StreamMaterialize { columns: [a, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_end] } StreamHopWindow { time_col: $1, slide: 00:15:00, size: 00:30:00, output_indices: [0, 4, 2] } StreamTableScan { table: t1, columns: [a, created_at, _row_id], pk_indices: [2] } diff --git a/src/frontend/test_runner/tests/testdata/common_table_expressions.yaml b/src/frontend/test_runner/tests/testdata/common_table_expressions.yaml index 05869a0adb02b..64d625c1dfebd 100644 --- a/src/frontend/test_runner/tests/testdata/common_table_expressions.yaml +++ b/src/frontend/test_runner/tests/testdata/common_table_expressions.yaml @@ -7,7 +7,7 @@ LogicalProject { exprs: [$1, $2] } LogicalScan { table: t1, columns: [_row_id, v1, v2] } stream_plan: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } StreamTableScan { table: t1, columns: [v1, _row_id], pk_indices: [1] } - sql: | create table t1 (v1 int, v2 int); @@ -20,7 +20,7 @@ LogicalProject { exprs: [$1] } LogicalScan { table: t1, columns: [_row_id, v1, v2] } stream_plan: | - StreamMaterialize { columns: [v3, v4, v1, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1] } + StreamMaterialize { columns: [v3, v4, v1, t2._row_id(hidden), t1._row_id(hidden)], pk_columns: [t2._row_id, t1._row_id] } StreamExchange { dist: HashShard([3, 4]) } StreamHashJoin { type: Inner, predicate: $0 = $3, output_indices: [0, 1, 3, 2, 4] } StreamExchange { dist: HashShard([0]) } @@ -37,5 +37,5 @@ LogicalProject { exprs: [$1, $2] } LogicalScan { table: t1, columns: [_row_id, v1, v2] } stream_plan: | - StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } StreamTableScan { table: t1, columns: [v1, _row_id], pk_indices: [1] } diff --git a/src/frontend/test_runner/tests/testdata/distribution_derive.yaml b/src/frontend/test_runner/tests/testdata/distribution_derive.yaml index 75ee994e4caa9..a2dc66cc3ec1c 100644 --- a/src/frontend/test_runner/tests/testdata/distribution_derive.yaml +++ b/src/frontend/test_runner/tests/testdata/distribution_derive.yaml @@ -46,7 +46,7 @@ BatchProject { exprs: [($0 + $1), $0] } BatchScan { table: t, columns: [a, b] } stream_plan: | - StreamMaterialize { columns: [max_a, expr#0(hidden)], pk_columns: [expr#0] } + StreamMaterialize { columns: [max_a, (t.a + t.b)(hidden)], pk_columns: [(t.a + t.b)] } StreamProject { exprs: [$2, $0] } StreamHashAgg { group_key: [$0], aggs: [count, max($1)] } StreamExchange { dist: HashShard([0]) } @@ -67,6 +67,6 @@ BatchExchange { order: [], dist: Single } BatchScan { table: t1, columns: [row_id, uid, v, created_at] } stream_plan: | - StreamMaterialize { columns: [row_id, uid, v, created_at, window_start, window_end, _row_id(hidden)], pk_columns: [_row_id, window_start] } + StreamMaterialize { columns: [row_id, uid, v, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start] } StreamHopWindow { time_col: $3, slide: 00:15:00, size: 00:30:00, output_indices: [0, 1, 2, 3, 5, 6, 4] } StreamTableScan { table: t1, columns: [row_id, uid, v, created_at, _row_id], pk_indices: [4] } diff --git a/src/frontend/test_runner/tests/testdata/expr.yaml b/src/frontend/test_runner/tests/testdata/expr.yaml index e7c767a1e4b99..b1368a052fd23 100644 --- a/src/frontend/test_runner/tests/testdata/expr.yaml +++ b/src/frontend/test_runner/tests/testdata/expr.yaml @@ -162,7 +162,7 @@ BatchProject { exprs: [Case(($0 = 1:Int32), 1:Int32::Decimal, ($0 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal)] } BatchScan { table: t, columns: [v1] } stream_plan: | - StreamMaterialize { columns: [expr, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } StreamProject { exprs: [Case(($0 = 1:Int32), 1:Int32::Decimal, ($0 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal), $1] } StreamTableScan { table: t, columns: [v1, _row_id], pk_indices: [1] } - sql: | @@ -199,7 +199,7 @@ BatchProject { exprs: [Case(($0 = 1:Int32), null:Int32, $0)] } BatchScan { table: t, columns: [v1] } stream_plan: | - StreamMaterialize { columns: [expr, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } StreamProject { exprs: [Case(($0 = 1:Int32), null:Int32, $0), $1] } StreamTableScan { table: t, columns: [v1, _row_id], pk_indices: [1] } - sql: | @@ -218,7 +218,7 @@ BatchProject { exprs: [Coalesce($0, 1:Int32)] } BatchScan { table: t, columns: [v1] } stream_plan: | - StreamMaterialize { columns: [expr, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } StreamProject { exprs: [Coalesce($0, 1:Int32), $1] } StreamTableScan { table: t, columns: [v1, _row_id], pk_indices: [1] } - sql: | @@ -244,7 +244,7 @@ BatchProject { exprs: [ConcatWs($0, 1:Int32::Varchar)] } BatchScan { table: t, columns: [v1] } stream_plan: | - StreamMaterialize { columns: [expr, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } StreamProject { exprs: [ConcatWs($0, 1:Int32::Varchar), $1] } StreamTableScan { table: t, columns: [v1, _row_id], pk_indices: [1] } - sql: | @@ -270,7 +270,7 @@ BatchProject { exprs: [ConcatWs('':Varchar, $0, $1::Varchar, $2::Varchar, 1:Int32::Varchar)] } BatchScan { table: t, columns: [v1, v2, v3] } stream_plan: | - StreamMaterialize { columns: [expr, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } StreamProject { exprs: [ConcatWs('':Varchar, $0, $1::Varchar, $2::Varchar, 1:Int32::Varchar), $3] } StreamTableScan { table: t, columns: [v1, v2, v3, _row_id], pk_indices: [3] } - sql: | diff --git a/src/frontend/test_runner/tests/testdata/index.yaml b/src/frontend/test_runner/tests/testdata/index.yaml index 2e32354039f0c..ca3709392dea7 100644 --- a/src/frontend/test_runner/tests/testdata/index.yaml +++ b/src/frontend/test_runner/tests/testdata/index.yaml @@ -7,11 +7,11 @@ /* should generate delta join plan, and stream index scan */ select * from t1, t2 where t1.v1 = t2.v3; stream_plan: | - StreamMaterialize { columns: [v1, v2, v3, v4, v5, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1] } + StreamMaterialize { columns: [v1, v2, v3, v4, v5, t1_v1.t1._row_id(hidden), t2_v3.t2._row_id(hidden)], pk_columns: [t1_v1.t1._row_id, t2_v3.t2._row_id] } StreamExchange { dist: HashShard([5, 6]) } StreamDeltaJoin { type: Inner, predicate: $0 = $3, output_indices: [0, 1, 3, 4, 5, 2, 6] } - StreamIndexScan { index: t1_v1, columns: [v1, v2, _row_id], pk_indices: [2] } - StreamIndexScan { index: t2_v3, columns: [v3, v4, v5, _row_id], pk_indices: [3] } + StreamIndexScan { index: t1_v1, columns: [v1, v2, t1._row_id], pk_indices: [2] } + StreamIndexScan { index: t2_v3, columns: [v3, v4, v5, t2._row_id], pk_indices: [3] } - id: index_slt sql: | create table iii_t1 (v1 int, v2 int); @@ -25,18 +25,18 @@ sql: | select * from iii_t1, iii_t2 where iii_t1.v1 = iii_t2.v3; stream_plan: | - StreamMaterialize { columns: [v1, v2, v3, v4, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1] } + StreamMaterialize { columns: [v1, v2, v3, v4, iii_index_1.iii_t1._row_id(hidden), iii_index_2.iii_t2._row_id(hidden)], pk_columns: [iii_index_1.iii_t1._row_id, iii_index_2.iii_t2._row_id] } StreamExchange { dist: HashShard([4, 5]) } StreamDeltaJoin { type: Inner, predicate: $0 = $3, output_indices: [0, 1, 3, 4, 2, 5] } - StreamIndexScan { index: iii_index_1, columns: [v1, v2, _row_id], pk_indices: [2] } - StreamIndexScan { index: iii_index_2, columns: [v3, v4, _row_id], pk_indices: [2] } + StreamIndexScan { index: iii_index_1, columns: [v1, v2, iii_t1._row_id], pk_indices: [2] } + StreamIndexScan { index: iii_index_2, columns: [v3, v4, iii_t2._row_id], pk_indices: [2] } - before: - index_slt sql: | select v4 from iii_t1, iii_t2 where iii_t1.v1 = iii_t2.v3; stream_plan: | - StreamMaterialize { columns: [v4, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1] } + StreamMaterialize { columns: [v4, iii_index_1.iii_t1._row_id(hidden), iii_index_2.iii_t2._row_id(hidden)], pk_columns: [iii_index_1.iii_t1._row_id, iii_index_2.iii_t2._row_id] } StreamExchange { dist: HashShard([1, 2]) } StreamDeltaJoin { type: Inner, predicate: $0 = $2, output_indices: [3, 1, 4] } - StreamIndexScan { index: iii_index_1, columns: [v1, _row_id], pk_indices: [1] } - StreamIndexScan { index: iii_index_2, columns: [v3, v4, _row_id], pk_indices: [2] } + StreamIndexScan { index: iii_index_1, columns: [v1, iii_t1._row_id], pk_indices: [1] } + StreamIndexScan { index: iii_index_2, columns: [v3, v4, iii_t2._row_id], pk_indices: [2] } diff --git a/src/frontend/test_runner/tests/testdata/join.yaml b/src/frontend/test_runner/tests/testdata/join.yaml index 377852b4014e5..6e3545a1344ae 100644 --- a/src/frontend/test_runner/tests/testdata/join.yaml +++ b/src/frontend/test_runner/tests/testdata/join.yaml @@ -13,7 +13,7 @@ LogicalScan { table: t2, columns: [_row_id, v3, v4] } LogicalScan { table: t3, columns: [_row_id, v5, v6] } stream_plan: | - StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, _row_id(hidden), _row_id#1(hidden), _row_id#2(hidden)], pk_columns: [_row_id, _row_id#1, _row_id#2] } + StreamMaterialize { columns: [v1, v2, v3, v4, v5, v6, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, t3._row_id] } StreamExchange { dist: HashShard([6, 7, 8]) } StreamHashJoin { type: Inner, predicate: $0 = $6, output_indices: [0, 1, 2, 3, 6, 7, 4, 5, 8] } StreamHashJoin { type: Inner, predicate: $0 = $3, output_indices: [0, 1, 3, 4, 2, 5] } @@ -33,7 +33,7 @@ LogicalScan { table: t, columns: [_row_id, v1, v2] } LogicalScan { table: t, columns: [_row_id, v1, v2] } stream_plan: | - StreamMaterialize { columns: [t1v1, t2v1, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1] } + StreamMaterialize { columns: [t1v1, t2v1, t._row_id(hidden), t._row_id#1(hidden)], pk_columns: [t._row_id, t._row_id#1] } StreamExchange { dist: HashShard([2, 3]) } StreamHashJoin { type: Inner, predicate: $0 = $2, output_indices: [0, 2, 1, 3] } StreamExchange { dist: HashShard([0]) } @@ -66,7 +66,7 @@ BatchExchange { order: [], dist: Single } BatchScan { table: t3, columns: [v1, v2] } stream_plan: | - StreamMaterialize { columns: [t1_v1, t1_v2, t2_v1, t2_v2, t3_v1, t3_v2, _row_id(hidden), _row_id#1(hidden), _row_id#2(hidden)], pk_columns: [_row_id, _row_id#1, _row_id#2] } + StreamMaterialize { columns: [t1_v1, t1_v2, t2_v1, t2_v2, t3_v1, t3_v2, t1._row_id(hidden), t2._row_id(hidden), t3._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, t3._row_id] } StreamExchange { dist: HashShard([6, 7, 8]) } StreamHashJoin { type: Inner, predicate: $3 = $7, output_indices: [0, 1, 2, 3, 6, 7, 4, 5, 8] } StreamExchange { dist: HashShard([3]) } @@ -95,7 +95,7 @@ BatchExchange { order: [], dist: Single } BatchScan { table: t2, columns: [v1, v2] } stream_plan: | - StreamMaterialize { columns: [t1_v2, t2_v2, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1] } + StreamMaterialize { columns: [t1_v2, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id] } StreamExchange { dist: HashShard([2, 3]) } StreamHashJoin { type: Inner, predicate: $0 = $3, output_indices: [1, 4, 2, 5] } StreamExchange { dist: HashShard([0]) } @@ -125,7 +125,7 @@ create table t2 (v3 int, v4 numeric, v5 bigint); select * from t1, t2 where t1.v1 = t2.v3; stream_plan: | - StreamMaterialize { columns: [v1, v2, v3, v4, v5, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1] } + StreamMaterialize { columns: [v1, v2, v3, v4, v5, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id] } StreamExchange { dist: HashShard([5, 6]) } StreamDeltaHashJoin { type: Inner, predicate: $0 = $3, output_indices: [0, 1, 3, 4, 5, 2, 6] } StreamExchange { dist: HashShard([0]) } @@ -209,5 +209,5 @@ BatchExchange { order: [], dist: Single } BatchScan { table: t1, columns: [v1, v2] } with_config_map: + QUERY_MODE: local RW_BATCH_ENABLE_LOOKUP_JOIN: "true" - QUERY_MODE: "local" diff --git a/src/frontend/test_runner/tests/testdata/mv_on_mv.yaml b/src/frontend/test_runner/tests/testdata/mv_on_mv.yaml index df0b948b7220e..5d1e68d8dca10 100644 --- a/src/frontend/test_runner/tests/testdata/mv_on_mv.yaml +++ b/src/frontend/test_runner/tests/testdata/mv_on_mv.yaml @@ -11,10 +11,10 @@ sql: | select m1.v1 as m1v1, m1.v2 as m1v2, m2.v1 as m2v1, m2.v2 as m2v2 from m1 join m2 on m1.v1 = m2.v1; stream_plan: | - StreamMaterialize { columns: [m1v1, m1v2, m2v1, m2v2, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1] } + StreamMaterialize { columns: [m1v1, m1v2, m2v1, m2v2, m1.t1._row_id(hidden), m2.t1._row_id(hidden)], pk_columns: [m1.t1._row_id, m2.t1._row_id] } StreamExchange { dist: HashShard([4, 5]) } StreamHashJoin { type: Inner, predicate: $0 = $3, output_indices: [0, 1, 3, 4, 2, 5] } StreamExchange { dist: HashShard([0]) } - StreamTableScan { table: m1, columns: [v1, v2, _row_id], pk_indices: [2] } + StreamTableScan { table: m1, columns: [v1, v2, t1._row_id], pk_indices: [2] } StreamExchange { dist: HashShard([0]) } - StreamTableScan { table: m2, columns: [v1, v2, _row_id], pk_indices: [2] } + StreamTableScan { table: m2, columns: [v1, v2, t1._row_id], pk_indices: [2] } diff --git a/src/frontend/test_runner/tests/testdata/nexmark.yaml b/src/frontend/test_runner/tests/testdata/nexmark.yaml index 74b7c5262ccb4..cdf8a572c5472 100644 --- a/src/frontend/test_runner/tests/testdata/nexmark.yaml +++ b/src/frontend/test_runner/tests/testdata/nexmark.yaml @@ -48,7 +48,7 @@ BatchExchange { order: [], dist: Single } BatchScan { table: bid, columns: [auction, bidder, price, dateTime] } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, dateTime, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, dateTime, bid._row_id(hidden)], pk_columns: [bid._row_id] } StreamTableScan { table: bid, columns: [auction, bidder, price, dateTime, _row_id], pk_indices: [4] } - id: nexmark_q1 before: @@ -65,7 +65,7 @@ BatchProject { exprs: [$0, $1, (0.908:Decimal * $2), $3] } BatchScan { table: bid, columns: [auction, bidder, price, dateTime] } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, dateTime, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, dateTime, bid._row_id(hidden)], pk_columns: [bid._row_id] } StreamProject { exprs: [$0, $1, (0.908:Decimal * $2), $3, $4] } StreamTableScan { table: bid, columns: [auction, bidder, price, dateTime, _row_id], pk_indices: [4] } - id: nexmark_q2 @@ -77,7 +77,7 @@ BatchFilter { predicate: ((((($0 = 1007:Int32) OR ($0 = 1020:Int32)) OR ($0 = 2001:Int32)) OR ($0 = 2019:Int32)) OR ($0 = 2087:Int32)) } BatchScan { table: bid, columns: [auction, price] } stream_plan: | - StreamMaterialize { columns: [auction, price, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, price, bid._row_id(hidden)], pk_columns: [bid._row_id] } StreamFilter { predicate: ((((($0 = 1007:Int32) OR ($0 = 1020:Int32)) OR ($0 = 2001:Int32)) OR ($0 = 2019:Int32)) OR ($0 = 2087:Int32)) } StreamTableScan { table: bid, columns: [auction, price, _row_id], pk_indices: [2] } - id: nexmark_q3 @@ -102,7 +102,7 @@ BatchFilter { predicate: ((($3 = 'or':Varchar) OR ($3 = 'id':Varchar)) OR ($3 = 'ca':Varchar)) } BatchScan { table: person, columns: [id, name, city, state] } stream_plan: | - StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1] } + StreamMaterialize { columns: [name, city, state, id, auction._row_id(hidden), person._row_id(hidden)], pk_columns: [auction._row_id, person._row_id] } StreamExchange { dist: HashShard([4, 5]) } StreamProject { exprs: [$1, $2, $3, $0, $4, $5] } StreamHashJoin { type: Inner, predicate: $1 = $3, output_indices: [0, 4, 5, 6, 2, 7] } @@ -252,7 +252,7 @@ BatchProject { exprs: [(TumbleStart($1, '00:00:10':Interval) + '00:00:10':Interval), $0] } BatchScan { table: bid, columns: [price, dateTime] } stream_plan: | - StreamMaterialize { columns: [auction, price, bidder, dateTime, _row_id(hidden), expr#0(hidden)], pk_columns: [_row_id, expr#0] } + StreamMaterialize { columns: [auction, price, bidder, dateTime, bid._row_id(hidden), (TumbleStart(bid.dateTime, '00:00:10':Interval) + '00:00:10':Interval)(hidden)], pk_columns: [bid._row_id, (TumbleStart(bid.dateTime, '00:00:10':Interval) + '00:00:10':Interval)] } StreamExchange { dist: HashShard([4, 5]) } StreamProject { exprs: [$0, $2, $1, $3, $4, $6] } StreamFilter { predicate: ($3 >= ($6 - '00:00:10':Interval)) AND ($3 <= $6) } @@ -315,7 +315,7 @@ BatchProject { exprs: [$1, TumbleStart($0, '00:00:10':Interval), (TumbleStart($0, '00:00:10':Interval) + '00:00:10':Interval)] } BatchScan { table: auction, columns: [dateTime, seller] } stream_plan: | - StreamMaterialize { columns: [id, name, starttime, expr#3(hidden), seller(hidden), expr#1(hidden), expr#2(hidden)], pk_columns: [id, name, starttime, expr#3, seller, expr#1, expr#2] } + StreamMaterialize { columns: [id, name, starttime, (TumbleStart(person.dateTime, '00:00:10':Interval) + '00:00:10':Interval)(hidden), auction.seller(hidden), TumbleStart(auction.dateTime, '00:00:10':Interval)(hidden), (TumbleStart(auction.dateTime, '00:00:10':Interval) + '00:00:10':Interval)(hidden)], pk_columns: [id, name, starttime, (TumbleStart(person.dateTime, '00:00:10':Interval) + '00:00:10':Interval), auction.seller, TumbleStart(auction.dateTime, '00:00:10':Interval), (TumbleStart(auction.dateTime, '00:00:10':Interval) + '00:00:10':Interval)] } StreamHashJoin { type: Inner, predicate: $0 = $5 AND $2 = $6 AND $3 = $7, output_indices: [0, 1, 2, 3, 5, 6, 7] } StreamExchange { dist: HashShard([0, 2, 3]) } StreamHashAgg { group_key: [$0, $1, $2, $3], aggs: [count] } @@ -351,7 +351,7 @@ BatchProject { exprs: [$0, $1, $2, $3, ToChar($3, 'YYYY-MM-DD':Varchar), ToChar($3, 'HH:MI':Varchar)] } BatchScan { table: bid, columns: [auction, bidder, price, dateTime] } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, dateTime, date, time, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, dateTime, date, time, bid._row_id(hidden)], pk_columns: [bid._row_id] } StreamProject { exprs: [$0, $1, $2, $3, ToChar($3, 'YYYY-MM-DD':Varchar), ToChar($3, 'HH:MI':Varchar), $4] } StreamTableScan { table: bid, columns: [auction, bidder, price, dateTime, _row_id], pk_indices: [4] } - id: nexmark_q11 @@ -424,7 +424,7 @@ BatchFilter { predicate: ((0.908:Decimal * $2) > 1000000:Int32) AND ((0.908:Decimal * $2) < 50000000:Int32) } BatchScan { table: bid, columns: [auction, bidder, price, dateTime, extra] } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, bidTimeType, dateTime, extra, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, bidTimeType, dateTime, extra, bid._row_id(hidden)], pk_columns: [bid._row_id] } StreamProject { exprs: [$0, $1, (0.908:Decimal * $2), Case(((Extract('HOUR':Varchar, $3) >= 8:Int32) AND (Extract('HOUR':Varchar, $3) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, $3) <= 6:Int32) OR (Extract('HOUR':Varchar, $3) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar), $3, $4, $5] } StreamFilter { predicate: ((0.908:Decimal * $2) > 1000000:Int32) AND ((0.908:Decimal * $2) < 50000000:Int32) } StreamTableScan { table: bid, columns: [auction, bidder, price, dateTime, extra, _row_id], pk_indices: [5] } @@ -458,7 +458,7 @@ BatchProject { exprs: [ToChar($3, 'yyyy-MM-dd':Varchar), $2, $1, $0] } BatchScan { table: bid, columns: [auction, bidder, price, dateTime] } stream_plan: | - StreamMaterialize { columns: [day, agg#0(hidden), total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day] } + StreamMaterialize { columns: [day, count(hidden), total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day] } StreamHashAgg { group_key: [$0], aggs: [count, sum($11) filter(($9 = 0:Int64)), sum($12) filter(($9 = 0:Int64)), sum($13) filter(($9 = 0:Int64)), sum($14) filter(($9 = 0:Int64)), count($1) filter(($9 = 1:Int64)), count($2) filter((($15 > 0:Int64) AND ($9 = 2:Int64))), count($3) filter((($16 > 0:Int64) AND ($9 = 3:Int64))), count($4) filter((($17 > 0:Int64) AND ($9 = 4:Int64))), count($5) filter(($9 = 5:Int64)), count($6) filter((($18 > 0:Int64) AND ($9 = 6:Int64))), count($7) filter((($19 > 0:Int64) AND ($9 = 7:Int64))), count($8) filter((($20 > 0:Int64) AND ($9 = 8:Int64)))] } StreamExchange { dist: HashShard([0]) } StreamHashAgg { group_key: [$0, $2, $2, $2, $2, $3, $3, $3, $3, $5], aggs: [count, count, count filter(($1 < 10000:Int32)), count filter((($1 >= 10000:Int32) AND ($1 < 1000000:Int32))), count filter(($1 >= 1000000:Int32)), count filter(($1 < 10000:Int32)), count filter((($1 >= 10000:Int32) AND ($1 < 1000000:Int32))), count filter(($1 >= 1000000:Int32)), count filter(($1 < 10000:Int32)), count filter((($1 >= 10000:Int32) AND ($1 < 1000000:Int32))), count filter(($1 >= 1000000:Int32))] } @@ -498,7 +498,7 @@ BatchProject { exprs: [$3, ToChar($4, 'yyyy-MM-dd':Varchar), ToChar($4, 'HH:mm':Varchar), $2, $1, $0] } BatchScan { table: bid, columns: [auction, bidder, price, channel, dateTime] } stream_plan: | - StreamMaterialize { columns: [channel, day, agg#0(hidden), minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day] } + StreamMaterialize { columns: [channel, day, count(hidden), minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day] } StreamHashAgg { group_key: [$0, $1], aggs: [count, max($12) filter(($10 = 0:Int64)), sum($13) filter(($10 = 0:Int64)), sum($14) filter(($10 = 0:Int64)), sum($15) filter(($10 = 0:Int64)), sum($16) filter(($10 = 0:Int64)), count($2) filter(($10 = 1:Int64)), count($3) filter((($17 > 0:Int64) AND ($10 = 2:Int64))), count($4) filter((($18 > 0:Int64) AND ($10 = 3:Int64))), count($5) filter((($19 > 0:Int64) AND ($10 = 4:Int64))), count($6) filter(($10 = 5:Int64)), count($7) filter((($20 > 0:Int64) AND ($10 = 6:Int64))), count($8) filter((($21 > 0:Int64) AND ($10 = 7:Int64))), count($9) filter((($22 > 0:Int64) AND ($10 = 8:Int64)))] } StreamExchange { dist: HashShard([0, 1]) } StreamHashAgg { group_key: [$0, $1, $4, $4, $4, $4, $5, $5, $5, $5, $7], aggs: [count, max($2), count, count filter(($3 < 10000:Int32)), count filter((($3 >= 10000:Int32) AND ($3 < 1000000:Int32))), count filter(($3 >= 1000000:Int32)), count filter(($3 < 10000:Int32)), count filter((($3 >= 10000:Int32) AND ($3 < 1000000:Int32))), count filter(($3 >= 1000000:Int32)), count filter(($3 < 10000:Int32)), count filter((($3 >= 10000:Int32) AND ($3 < 1000000:Int32))), count filter(($3 >= 1000000:Int32))] } @@ -573,7 +573,7 @@ BatchFilter { predicate: ($8 = 10:Int32) } BatchScan { table: auction, columns: [id, itemName, description, initialBid, reserve, dateTime, expires, seller, category] } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, channel, url, dateTimeB, itemName, description, initialBid, reserve, dateTimeA, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1] } + StreamMaterialize { columns: [auction, bidder, price, channel, url, dateTimeB, itemName, description, initialBid, reserve, dateTimeA, expires, seller, category, bid._row_id(hidden), auction._row_id(hidden)], pk_columns: [bid._row_id, auction._row_id] } StreamExchange { dist: HashShard([14, 15]) } StreamHashJoin { type: Inner, predicate: $0 = $7, output_indices: [0, 1, 2, 3, 4, 5, 8, 9, 10, 11, 12, 13, 14, 15, 6, 16] } StreamExchange { dist: HashShard([0]) } @@ -612,6 +612,6 @@ BatchProject { exprs: [$0, $1, $2, $3, SplitPart($4, '/':Varchar, 4:Int32), SplitPart($4, '/':Varchar, 5:Int32), SplitPart($4, '/':Varchar, 6:Int32)] } BatchScan { table: bid, columns: [auction, bidder, price, channel, url] } stream_plan: | - StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, bid._row_id(hidden)], pk_columns: [bid._row_id] } StreamProject { exprs: [$0, $1, $2, $3, SplitPart($4, '/':Varchar, 4:Int32), SplitPart($4, '/':Varchar, 5:Int32), SplitPart($4, '/':Varchar, 6:Int32), $5] } StreamTableScan { table: bid, columns: [auction, bidder, price, channel, url, _row_id], pk_indices: [5] } diff --git a/src/frontend/test_runner/tests/testdata/order_by.yaml b/src/frontend/test_runner/tests/testdata/order_by.yaml index 389835f0d6ea9..fb426d3c11aa3 100644 --- a/src/frontend/test_runner/tests/testdata/order_by.yaml +++ b/src/frontend/test_runner/tests/testdata/order_by.yaml @@ -8,7 +8,7 @@ BatchSort { order: [$0 DESC] } BatchScan { table: t, columns: [v1, v2] } stream_plan: | - StreamMaterialize { columns: [v1, v2, _row_id(hidden)], pk_columns: [_row_id], order_descs: [v1, _row_id] } + StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [v1, t._row_id] } StreamTableScan { table: t, columns: [v1, v2, _row_id], pk_indices: [2] } - sql: | /* output names are not quailified after table names */ @@ -64,7 +64,7 @@ BatchProject { exprs: [$0, $1, (1:Int32 + 1:Int32)] } BatchScan { table: t, columns: [v1, v2] } stream_plan: | - StreamMaterialize { columns: [v1, v2, expr#2(hidden), _row_id(hidden)], pk_columns: [_row_id], order_descs: [expr#2, _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: [$0, $1, (1:Int32 + 1:Int32), $2] } StreamTableScan { table: t, columns: [v1, v2, _row_id], pk_indices: [2] } - sql: | @@ -80,8 +80,8 @@ BatchTopN { order: [$0 DESC], limit: 5, offset: 0 } BatchScan { table: t, columns: [v1, v2] } stream_plan: | - StreamMaterialize { columns: [v1, v2, _row_id(hidden)], pk_columns: [_row_id], order_descs: [v1, _row_id] } - StreamTopN { order: [$0 DESC], limit: 5, offset: 0 } + StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [v1, t._row_id] } + StreamTopN { order: "[$0 DESC]", limit: 5, offset: 0 } StreamExchange { dist: Single } StreamTableScan { table: t, columns: [v1, v2, _row_id], pk_indices: [2] } - sql: | @@ -109,8 +109,8 @@ BatchTopN { order: [$0 DESC], limit: 12, offset: 0 } BatchScan { table: t, columns: [v1, v2] } stream_plan: | - StreamMaterialize { columns: [v1, v2, _row_id(hidden)], pk_columns: [_row_id], order_descs: [v1, _row_id] } - StreamTopN { order: [$0 DESC], limit: 5, offset: 7 } + StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [v1, t._row_id] } + StreamTopN { order: "[$0 DESC]", limit: 5, offset: 7 } StreamExchange { dist: Single } StreamTableScan { table: t, columns: [v1, v2, _row_id], pk_indices: [2] } - sql: | @@ -127,7 +127,7 @@ BatchProject { exprs: [$0, $1, ($0 + $1), $2] } BatchScan { table: t, columns: [x, y, z] } stream_plan: | - StreamMaterialize { columns: [x, y, expr#2(hidden), z(hidden), _row_id(hidden)], pk_columns: [_row_id], order_descs: [expr#2, z, _row_id] } + StreamMaterialize { columns: [x, y, (t.x + t.y)(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [(t.x + t.y), t.z, t._row_id] } StreamProject { exprs: [$0, $1, ($0 + $1), $2, $3] } StreamTableScan { table: t, columns: [x, y, z, _row_id], pk_indices: [3] } - sql: | diff --git a/src/frontend/test_runner/tests/testdata/pk_derive.yaml b/src/frontend/test_runner/tests/testdata/pk_derive.yaml index 8dc550e2e2b41..0cb8f405dc736 100644 --- a/src/frontend/test_runner/tests/testdata/pk_derive.yaml +++ b/src/frontend/test_runner/tests/testdata/pk_derive.yaml @@ -20,7 +20,7 @@ ON Tone.id = Ttwo.id; stream_plan: | - StreamMaterialize { columns: [max_v1, max_v2, id(hidden), id#1(hidden)], pk_columns: [id, id#1] } + StreamMaterialize { columns: [max_v1, max_v2, t1.id(hidden), t2.id(hidden)], pk_columns: [t1.id, t2.id] } StreamHashJoin { type: Inner, predicate: $1 = $3, output_indices: [0, 2, 1, 3] } StreamProject { exprs: [$2, $0] } StreamHashAgg { group_key: [$0], aggs: [count, max($1)] } @@ -50,7 +50,7 @@ ON Tone.id = Ttwo.id; stream_plan: | - StreamMaterialize { columns: [max_v, min_v, id(hidden), id#1(hidden)], pk_columns: [id, id#1] } + StreamMaterialize { columns: [max_v, min_v, t.id(hidden), t.id#1(hidden)], pk_columns: [t.id, t.id#1] } StreamHashJoin { type: Inner, predicate: $1 = $3, output_indices: [0, 2, 1, 3] } StreamProject { exprs: [$2, $0] } StreamHashAgg { group_key: [$0], aggs: [count, max($1)] } @@ -74,7 +74,7 @@ LogicalAgg { group_key: [0, 1, 2], agg_calls: [] } LogicalScan { table: t, columns: [v1, v2, v3] } stream_plan: | - StreamMaterialize { columns: [v1, v2, v3, agg#0(hidden)], pk_columns: [v1, v2, v3] } + StreamMaterialize { columns: [v1, v2, v3, count(hidden)], pk_columns: [v1, v2, v3] } StreamHashAgg { group_key: [$0, $1, $2], aggs: [count] } StreamExchange { dist: HashShard([0, 1, 2]) } StreamTableScan { table: t, columns: [v1, v2, v3, _row_id], pk_indices: [3] } @@ -98,6 +98,6 @@ optimized_logical_plan: | LogicalScan { table: mv, output_columns: [v1], required_columns: [$0:v1, $2:v3], predicate: (($2 = 'world':Varchar) OR ($2 = 'hello':Varchar)) } stream_plan: | - StreamMaterialize { columns: [v1, v2(hidden), v3(hidden)], pk_columns: [v1, v2, v3] } + StreamMaterialize { columns: [v1, mv.v2(hidden), mv.v3(hidden)], pk_columns: [v1, mv.v2, mv.v3] } StreamFilter { predicate: (($2 = 'world':Varchar) OR ($2 = 'hello':Varchar)) } StreamTableScan { table: mv, columns: [v1, v2, v3], pk_indices: [0, 1, 2] } diff --git a/src/frontend/test_runner/tests/testdata/stream_proto.yaml b/src/frontend/test_runner/tests/testdata/stream_proto.yaml index 9dd1710173d66..021a8b35e2955 100644 --- a/src/frontend/test_runner/tests/testdata/stream_proto.yaml +++ b/src/frontend/test_runner/tests/testdata/stream_proto.yaml @@ -38,11 +38,11 @@ - dataType: typeName: INT32 isNullable: true - name: v1 + name: t.v1 - dataType: typeName: INT64 isNullable: true - name: _row_id + name: t._row_id chain: tableId: 4 upstreamFields: @@ -63,11 +63,11 @@ - dataType: typeName: INT32 isNullable: true - name: v1 + name: t.v1 - dataType: typeName: INT64 isNullable: true - name: _row_id + name: t._row_id filter: searchCondition: exprType: LESS_THAN @@ -93,11 +93,11 @@ - dataType: typeName: INT32 isNullable: true - name: v1 + name: t.v1 - dataType: typeName: INT64 isNullable: true - name: _row_id + name: t._row_id materialize: columnOrders: - orderType: ASCENDING @@ -118,7 +118,7 @@ typeName: INT64 isNullable: true columnId: 1 - name: _row_id + name: t._row_id isHidden: true orderKey: - orderType: ASCENDING @@ -143,7 +143,7 @@ typeName: INT64 isNullable: true columnId: 1 - name: _row_id + name: t._row_id isHidden: true orderKey: - orderType: ASCENDING @@ -191,11 +191,11 @@ - dataType: typeName: INT32 isNullable: true - name: v1 + name: t.v1 - dataType: typeName: INT64 isNullable: true - name: _row_id + name: t._row_id chain: tableId: 4 upstreamFields: @@ -216,11 +216,11 @@ - dataType: typeName: INT32 isNullable: true - name: v1 + name: t.v1 - dataType: typeName: INT64 isNullable: true - name: _row_id + name: t._row_id materialize: columnOrders: - orderType: ASCENDING @@ -241,7 +241,7 @@ typeName: INT64 isNullable: true columnId: 1 - name: _row_id + name: t._row_id isHidden: true orderKey: - orderType: ASCENDING @@ -266,7 +266,7 @@ typeName: INT64 isNullable: true columnId: 1 - name: _row_id + name: t._row_id isHidden: true orderKey: - orderType: ASCENDING @@ -319,11 +319,11 @@ - dataType: typeName: INT32 isNullable: true - name: v1 + name: t.v1 - dataType: typeName: INT64 isNullable: true - name: _row_id + name: t._row_id chain: tableId: 4 upstreamFields: @@ -348,11 +348,11 @@ - dataType: typeName: INT32 isNullable: true - name: v1 + name: t.v1 - dataType: typeName: INT64 isNullable: true - name: _row_id + name: t._row_id materialize: columnOrders: - orderType: ASCENDING @@ -373,7 +373,7 @@ typeName: INT64 isNullable: true columnId: 1 - name: _row_id + name: t._row_id isHidden: true orderKey: - orderType: ASCENDING @@ -398,7 +398,7 @@ typeName: INT64 isNullable: true columnId: 1 - name: _row_id + name: t._row_id isHidden: true orderKey: - orderType: ASCENDING @@ -449,11 +449,11 @@ - dataType: typeName: INT32 isNullable: true - name: v1 + name: t.v1 - dataType: typeName: INT64 isNullable: true - name: _row_id + name: t._row_id chain: tableId: 4 upstreamFields: @@ -472,11 +472,11 @@ - dataType: typeName: INT64 isNullable: true - name: "agg#0" + name: count - dataType: typeName: INT64 isNullable: true - name: "agg#1" + name: sum(t.v1) localSimpleAgg: aggCalls: - type: COUNT @@ -521,11 +521,11 @@ - dataType: typeName: INT64 isNullable: true - name: "agg#0" + name: count - dataType: typeName: INT64 isNullable: true - name: "agg#1" + name: sum(t.v1) exchange: strategy: type: SIMPLE @@ -533,11 +533,11 @@ - dataType: typeName: INT64 isNullable: true - name: "agg#0" + name: sum(count) - dataType: typeName: INT64 isNullable: true - name: "agg#1" + name: sum(sum(t.v1)) globalSimpleAgg: aggCalls: - type: SUM @@ -582,11 +582,11 @@ - dataType: typeName: INT64 isNullable: true - name: "agg#0" + name: sum(count) - dataType: typeName: INT64 isNullable: true - name: "agg#1" + name: sum(sum(t.v1)) materialize: table: id: 4294967294 @@ -598,7 +598,7 @@ columnType: typeName: INT64 isNullable: true - name: "agg#0" + name: sum(count) isHidden: true - columnDesc: columnType: @@ -616,7 +616,7 @@ columnType: typeName: INT64 isNullable: true - name: "agg#0" + name: sum(count) isHidden: true - columnDesc: columnType: @@ -673,15 +673,15 @@ - dataType: typeName: INT32 isNullable: true - name: v1 + name: t.v1 - dataType: typeName: INT32 isNullable: true - name: v2 + name: t.v2 - dataType: typeName: INT64 isNullable: true - name: _row_id + name: t._row_id chain: tableId: 4 upstreamFields: @@ -707,15 +707,15 @@ - dataType: typeName: INT32 isNullable: true - name: v2 + name: t.v2 - dataType: typeName: INT32 isNullable: true - name: v1 + name: t.v1 - dataType: typeName: INT64 isNullable: true - name: _row_id + name: t._row_id project: selectList: - exprType: INPUT_REF @@ -741,15 +741,15 @@ - dataType: typeName: INT32 isNullable: true - name: v2 + name: t.v2 - dataType: typeName: INT32 isNullable: true - name: v1 + name: t.v1 - dataType: typeName: INT64 isNullable: true - name: _row_id + name: t._row_id exchange: strategy: type: HASH @@ -761,15 +761,15 @@ - dataType: typeName: INT32 isNullable: true - name: v2 + name: t.v2 - dataType: typeName: INT64 isNullable: true - name: "agg#0" + name: count - dataType: typeName: INT64 isNullable: true - name: "agg#1" + name: sum(t.v1) hashAgg: groupKey: - 0 @@ -797,7 +797,7 @@ columnType: typeName: INT32 isNullable: true - name: v2 + name: t.v2 - columnDesc: columnType: typeName: INT64 @@ -818,7 +818,7 @@ columnType: typeName: INT32 isNullable: true - name: v2 + name: t.v2 - columnDesc: columnType: typeName: INT64 @@ -837,11 +837,11 @@ - dataType: typeName: INT64 isNullable: true - name: "agg#1" + name: sum(t.v1) - dataType: typeName: INT32 isNullable: true - name: v2 + name: t.v2 project: selectList: - exprType: INPUT_REF @@ -861,11 +861,11 @@ - dataType: typeName: INT64 isNullable: true - name: "agg#1" + name: sum(t.v1) - dataType: typeName: INT32 isNullable: true - name: v2 + name: t.v2 materialize: columnOrders: - orderType: ASCENDING @@ -886,7 +886,7 @@ typeName: INT32 isNullable: true columnId: 1 - name: v2 + name: t.v2 isHidden: true orderKey: - orderType: ASCENDING @@ -911,7 +911,7 @@ typeName: INT32 isNullable: true columnId: 1 - name: v2 + name: t.v2 isHidden: true orderKey: - orderType: ASCENDING diff --git a/src/frontend/test_runner/tests/testdata/struct_query.yaml b/src/frontend/test_runner/tests/testdata/struct_query.yaml index 1098bb6f951f6..a26e3f48a3d84 100644 --- a/src/frontend/test_runner/tests/testdata/struct_query.yaml +++ b/src/frontend/test_runner/tests/testdata/struct_query.yaml @@ -6,7 +6,7 @@ BatchExchange { order: [], dist: Single } BatchScan { table: t, columns: [country] } stream_plan: | - StreamMaterialize { columns: [country, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [country, t._row_id(hidden)], pk_columns: [t._row_id] } StreamTableScan { table: t, columns: [country, _row_id], pk_indices: [1] } create_source: row_format: protobuf diff --git a/src/frontend/test_runner/tests/testdata/time_window.yaml b/src/frontend/test_runner/tests/testdata/time_window.yaml index 220645cd88b3c..5e8355cff1193 100644 --- a/src/frontend/test_runner/tests/testdata/time_window.yaml +++ b/src/frontend/test_runner/tests/testdata/time_window.yaml @@ -44,7 +44,7 @@ LogicalHopWindow { time_col: $2, slide: 1 day 00:00:00, size: 3 days 00:00:00, output_indices: all } LogicalScan { table: t1, columns: [_row_id, id, created_at] } stream_plan: | - StreamMaterialize { columns: [id, created_at, window_start, window_end, _row_id(hidden)], pk_columns: [_row_id, window_start] } + StreamMaterialize { columns: [id, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start] } StreamHopWindow { time_col: $1, slide: 1 day 00:00:00, size: 3 days 00:00:00, output_indices: [0, 1, 3, 4, 2] } StreamTableScan { table: t1, columns: [id, created_at, _row_id], pk_indices: [2] } - sql: | @@ -55,7 +55,7 @@ LogicalHopWindow { time_col: $2, slide: 1 day 00:00:00, size: 3 days 00:00:00, output_indices: all } LogicalScan { table: t1, columns: [_row_id, id, created_at] } stream_plan: | - StreamMaterialize { columns: [id, created_at, window_start, _row_id(hidden)], pk_columns: [_row_id, window_start] } + StreamMaterialize { columns: [id, created_at, window_start, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start] } StreamHopWindow { time_col: $1, slide: 1 day 00:00:00, size: 3 days 00:00:00, output_indices: [0, 1, 3, 2] } StreamTableScan { table: t1, columns: [id, created_at, _row_id], pk_indices: [2] } - sql: | @@ -66,7 +66,7 @@ LogicalHopWindow { time_col: $2, slide: 1 day 00:00:00, size: 3 days 00:00:00, output_indices: all } LogicalScan { table: t1, columns: [_row_id, id, created_at] } stream_plan: | - StreamMaterialize { columns: [id, created_at, window_end, _row_id(hidden)], pk_columns: [_row_id, window_end] } + StreamMaterialize { columns: [id, created_at, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_end] } StreamHopWindow { time_col: $1, slide: 1 day 00:00:00, size: 3 days 00:00:00, output_indices: [0, 1, 4, 2] } StreamTableScan { table: t1, columns: [id, created_at, _row_id], pk_indices: [2] } - sql: | @@ -81,7 +81,7 @@ BatchExchange { order: [], dist: Single } BatchScan { table: t1, columns: [id, created_at] } stream_plan: | - StreamMaterialize { columns: [id, created_at, window_start(hidden), _row_id(hidden)], pk_columns: [_row_id, window_start] } + StreamMaterialize { columns: [id, created_at, window_start(hidden), t1._row_id(hidden)], pk_columns: [t1._row_id, window_start] } StreamHopWindow { time_col: $1, slide: 1 day 00:00:00, size: 3 days 00:00:00, output_indices: [0, 1, 3, 2] } StreamTableScan { table: t1, columns: [id, created_at, _row_id], pk_indices: [2] } - sql: | @@ -96,7 +96,7 @@ BatchExchange { order: [], dist: Single } BatchScan { table: t1, columns: [id, created_at] } stream_plan: | - StreamMaterialize { columns: [id, created_at, window_start(hidden), _row_id(hidden)], pk_columns: [_row_id, window_start] } + StreamMaterialize { columns: [id, created_at, window_start(hidden), t1._row_id(hidden)], pk_columns: [t1._row_id, window_start] } StreamHopWindow { time_col: $1, slide: 1 day 00:00:00, size: 3 days 00:00:00, output_indices: [0, 1, 3, 2] } StreamTableScan { table: t1, columns: [id, created_at, _row_id], pk_indices: [2] } - sql: | @@ -137,7 +137,7 @@ BatchProject { exprs: [$0, $1, $2, TumbleStart($2, '3 days 00:00:00':Interval), (TumbleStart($2, '3 days 00:00:00':Interval) + '3 days 00:00:00':Interval)] } BatchScan { table: t1, columns: [id, v1, created_at] } stream_plan: | - StreamMaterialize { columns: [id, v1, created_at, window_start, window_end, _row_id(hidden)], pk_columns: [_row_id] } + StreamMaterialize { columns: [id, v1, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id] } StreamProject { exprs: [$0, $1, $2, TumbleStart($2, '3 days 00:00:00':Interval), (TumbleStart($2, '3 days 00:00:00':Interval) + '3 days 00:00:00':Interval), $3] } StreamTableScan { table: t1, columns: [id, v1, created_at, _row_id], pk_indices: [3] } - sql: | @@ -156,7 +156,7 @@ BatchFilter { predicate: ($1 >= 10:Int32) } BatchScan { table: t1, columns: [id, v1, created_at] } stream_plan: | - StreamMaterialize { columns: [id, v1, created_at, window_start, window_end, _row_id(hidden)], pk_columns: [_row_id, window_start] } + StreamMaterialize { columns: [id, v1, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id, window_start] } StreamHopWindow { time_col: $2, slide: 1 day 00:00:00, size: 3 days 00:00:00, output_indices: [0, 1, 2, 4, 5, 3] } StreamFilter { predicate: ($1 >= 10:Int32) } StreamTableScan { table: t1, columns: [id, v1, created_at, _row_id], pk_indices: [3] } diff --git a/src/frontend/test_runner/tests/testdata/tpch.yaml b/src/frontend/test_runner/tests/testdata/tpch.yaml index 7e599f31fb6e8..9f0b8bace2e7c 100644 --- a/src/frontend/test_runner/tests/testdata/tpch.yaml +++ b/src/frontend/test_runner/tests/testdata/tpch.yaml @@ -272,8 +272,8 @@ BatchFilter { predicate: ($1 = 'AFRICA':Varchar) } BatchScan { table: region, columns: [r_regionkey, r_name] } stream_plan: | - StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), _row_id#1(hidden), _row_id#2(hidden), _row_id#3(hidden), _row_id#4(hidden), ps_partkey(hidden)], pk_columns: [_row_id, _row_id#1, _row_id#2, _row_id#3, _row_id#4, ps_partkey], order_descs: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, _row_id#2, _row_id#3, _row_id#4, ps_partkey] } - StreamTopN { order: [$0 DESC, $2 ASC, $1 ASC, $3 ASC], limit: 100, offset: 0 } + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, partsupp._row_id(hidden), part._row_id(hidden), supplier._row_id(hidden), nation._row_id(hidden), region._row_id(hidden), partsupp.ps_partkey(hidden)], pk_columns: [partsupp._row_id, part._row_id, supplier._row_id, nation._row_id, region._row_id, partsupp.ps_partkey], order_descs: [s_acctbal, n_name, s_name, p_partkey, partsupp._row_id, part._row_id, supplier._row_id, nation._row_id, region._row_id, partsupp.ps_partkey] } + StreamTopN { order: "[$0 DESC, $2 ASC, $1 ASC, $3 ASC]", limit: 100, offset: 0 } StreamExchange { dist: Single } StreamProject { exprs: [$5, $2, $7, $0, $1, $3, $4, $6, $8, $9, $10, $11, $12, $13] } StreamHashJoin { type: Inner, predicate: $1 = $14 AND $0 = $16, output_indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14] } @@ -388,7 +388,7 @@ BatchScan { table: lineitem, columns: [l_orderkey, l_extendedprice, l_discount, l_shipdate] } stream_plan: | StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], pk_columns: [l_orderkey, o_orderdate, o_shippriority], order_descs: [revenue, o_orderdate, l_orderkey, o_shippriority] } - StreamTopN { order: [$1 DESC, $2 ASC], limit: 10, offset: 0 } + StreamTopN { order: "[$1 DESC, $2 ASC]", limit: 10, offset: 0 } StreamExchange { dist: Single } StreamProject { exprs: [$0, $4, $1, $2] } StreamHashAgg { group_key: [$0, $1, $2], aggs: [count, sum($3)] } @@ -463,7 +463,7 @@ BatchFilter { predicate: ($1 < $2) } BatchScan { table: lineitem, columns: [l_orderkey, l_commitdate, l_receiptdate] } stream_plan: | - StreamMaterialize { columns: [o_orderpriority, agg#0(hidden), order_count], pk_columns: [o_orderpriority] } + StreamMaterialize { columns: [o_orderpriority, count(hidden), order_count], pk_columns: [o_orderpriority] } StreamHashAgg { group_key: [$0], aggs: [count, count] } StreamExchange { dist: HashShard([0]) } StreamHashJoin { type: LeftSemi, predicate: $0 = $3, output_indices: [1, 2] } @@ -565,7 +565,7 @@ BatchFilter { predicate: ($1 = 'MIDDLE EAST':Varchar) } BatchScan { table: region, columns: [r_regionkey, r_name] } stream_plan: | - StreamMaterialize { columns: [n_name, agg#0(hidden), revenue], pk_columns: [n_name], order_descs: [revenue, n_name] } + StreamMaterialize { columns: [n_name, count(hidden), revenue], pk_columns: [n_name], order_descs: [revenue, n_name] } StreamHashAgg { group_key: [$0], aggs: [count, sum($1)] } StreamExchange { dist: HashShard([0]) } StreamProject { exprs: [$2, ($0 * (1:Int32 - $1)), $3, $4, $5, $6, $7, $8] } @@ -625,7 +625,7 @@ BatchFilter { predicate: ($3 >= '1994-01-01':Varchar::Date) AND ($3 < ('1994-01-01':Varchar::Date + '1 year 00:00:00':Interval)) AND ($1 >= (0.08:Decimal - 0.01:Decimal)) AND ($1 <= (0.08:Decimal + 0.01:Decimal)) AND ($2 < 24:Int32) } BatchScan { table: lineitem, columns: [l_extendedprice, l_discount, l_quantity, l_shipdate] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), revenue], pk_columns: [] } + StreamMaterialize { columns: [sum(count)(hidden), revenue], pk_columns: [] } StreamGlobalSimpleAgg { aggs: [sum($0), sum($1)] } StreamExchange { dist: Single } StreamLocalSimpleAgg { aggs: [count, sum($0)] } @@ -736,7 +736,7 @@ BatchExchange { order: [], dist: HashShard([0]) } BatchScan { table: nation, columns: [n_nationkey, n_name] } stream_plan: | - StreamMaterialize { columns: [supp_nation, cust_nation, l_year, agg#0(hidden), revenue], pk_columns: [supp_nation, cust_nation, l_year] } + StreamMaterialize { columns: [supp_nation, cust_nation, l_year, count(hidden), revenue], pk_columns: [supp_nation, cust_nation, l_year] } StreamHashAgg { group_key: [$0, $1, $2], aggs: [count, sum($3)] } StreamExchange { dist: HashShard([0, 1, 2]) } StreamProject { exprs: [$4, $11, Extract('YEAR':Varchar, $2), ($0 * (1:Int32 - $1)), $5, $6, $7, $8, $9, $12] } @@ -1144,7 +1144,7 @@ BatchScan { table: nation, columns: [n_nationkey, n_name] } stream_plan: | StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], pk_columns: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], order_descs: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment] } - StreamTopN { order: [$2 DESC], limit: 20, offset: 0 } + StreamTopN { order: "[$2 DESC]", limit: 20, offset: 0 } StreamExchange { dist: Single } StreamProject { exprs: [$0, $1, $8, $2, $4, $5, $3, $6] } StreamHashAgg { group_key: [$0, $1, $2, $3, $4, $5, $6], aggs: [count, sum($7)] } @@ -1368,7 +1368,7 @@ BatchFilter { predicate: In($1, 'FOB':Varchar, 'SHIP':Varchar) AND ($3 < $4) AND ($2 < $3) AND ($4 >= '1994-01-01':Varchar::Date) AND ($4 < ('1994-01-01':Varchar::Date + '1 year 00:00:00':Interval)) } BatchScan { table: lineitem, columns: [l_orderkey, l_shipmode, l_shipdate, l_commitdate, l_receiptdate] } stream_plan: | - StreamMaterialize { columns: [l_shipmode, agg#0(hidden), high_line_count, low_line_count], pk_columns: [l_shipmode] } + StreamMaterialize { columns: [l_shipmode, count(hidden), high_line_count, low_line_count], pk_columns: [l_shipmode] } StreamHashAgg { group_key: [$0], aggs: [count, sum($1), sum($2)] } StreamExchange { dist: HashShard([0]) } StreamProject { exprs: [$1, Case((($0 = '1-URGENT':Varchar) OR ($0 = '2-HIGH':Varchar)), 1:Int32, 0:Int32), Case((($0 <> '1-URGENT':Varchar) AND ($0 <> '2-HIGH':Varchar)), 1:Int32, 0:Int32), $2, $3] } @@ -1435,7 +1435,7 @@ BatchFilter { predicate: Not(Like($2, '%:1%:2%':Varchar)) } BatchScan { table: orders, columns: [o_orderkey, o_custkey, o_comment] } stream_plan: | - StreamMaterialize { columns: [c_count, agg#0(hidden), custdist], pk_columns: [c_count], order_descs: [custdist, c_count] } + StreamMaterialize { columns: [c_count, count(hidden), custdist], pk_columns: [c_count], order_descs: [custdist, c_count] } StreamHashAgg { group_key: [$0], aggs: [count, count] } StreamExchange { dist: HashShard([0]) } StreamProject { exprs: [$2, $0] } @@ -1596,7 +1596,7 @@ BatchFilter { predicate: ($3 >= '1993-01-01':Varchar::Date) AND ($3 < ('1993-01-01':Varchar::Date + '3 mons 00:00:00':Interval)) } BatchScan { table: lineitem, columns: [l_suppkey, l_extendedprice, l_discount, l_shipdate] } stream_plan: | - StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, _row_id(hidden), l_suppkey(hidden)], pk_columns: [_row_id, l_suppkey], order_descs: [s_suppkey, _row_id, l_suppkey] } + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, supplier._row_id(hidden), lineitem.l_suppkey(hidden)], pk_columns: [supplier._row_id, lineitem.l_suppkey], order_descs: [s_suppkey, supplier._row_id, lineitem.l_suppkey] } StreamExchange { dist: HashShard([5, 6]) } StreamHashJoin { type: Inner, predicate: $4 = $8, output_indices: [0, 1, 2, 3, 4, 5, 6] } StreamExchange { dist: HashShard([4]) } @@ -1695,7 +1695,7 @@ BatchFilter { predicate: Like($1, '%Customer%Complaints%':Varchar) } BatchScan { table: supplier, columns: [s_suppkey, s_comment] } stream_plan: | - StreamMaterialize { columns: [p_brand, p_type, p_size, agg#0(hidden), supplier_cnt], pk_columns: [p_brand, p_type, p_size], order_descs: [supplier_cnt, p_brand, p_type, p_size] } + StreamMaterialize { columns: [p_brand, p_type, p_size, count(hidden), supplier_cnt], pk_columns: [p_brand, p_type, p_size], order_descs: [supplier_cnt, p_brand, p_type, p_size] } StreamHashAgg { group_key: [$0, $1, $2], aggs: [count, count($3) filter(($4 = 0:Int64))] } StreamExchange { dist: HashShard([0, 1, 2]) } StreamHashAgg { group_key: [$0, $1, $2, $3, $6], aggs: [count] } @@ -1891,8 +1891,8 @@ BatchExchange { order: [], dist: HashShard([0]) } BatchScan { table: lineitem, columns: [l_orderkey, l_quantity] } stream_plan: | - StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, agg#0(hidden), quantity], pk_columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice], order_descs: [o_totalprice, o_orderdate, c_name, c_custkey, o_orderkey] } - StreamTopN { order: [$4 DESC, $3 ASC], limit: 100, offset: 0 } + StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, count(hidden), quantity], pk_columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice], order_descs: [o_totalprice, o_orderdate, c_name, c_custkey, o_orderkey] } + StreamTopN { order: "[$4 DESC, $3 ASC]", limit: 100, offset: 0 } StreamExchange { dist: Single } StreamHashAgg { group_key: [$0, $1, $2, $3, $4], aggs: [count, sum($5)] } StreamProject { exprs: [$1, $0, $2, $4, $3, $5, $6, $7, $8] } @@ -1979,7 +1979,7 @@ BatchFilter { predicate: ($2 >= 1:Int32) } BatchScan { table: part, columns: [p_partkey, p_brand, p_size, p_container] } stream_plan: | - StreamMaterialize { columns: [agg#0(hidden), revenue], pk_columns: [] } + StreamMaterialize { columns: [sum(count)(hidden), revenue], pk_columns: [] } StreamGlobalSimpleAgg { aggs: [sum($0), sum($1)] } StreamExchange { dist: Single } StreamLocalSimpleAgg { aggs: [count, sum($0)] } @@ -2097,7 +2097,7 @@ BatchFilter { predicate: ($3 >= '1994-01-01':Varchar::Date) AND ($3 < ('1994-01-01':Varchar::Date + '1 year 00:00:00':Interval)) } BatchScan { table: lineitem, columns: [l_partkey, l_suppkey, l_quantity, l_shipdate] } stream_plan: | - StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden)], pk_columns: [_row_id, _row_id#1], order_descs: [s_name, _row_id, _row_id#1] } + StreamMaterialize { columns: [s_name, s_address, supplier._row_id(hidden), nation._row_id(hidden)], pk_columns: [supplier._row_id, nation._row_id], order_descs: [s_name, supplier._row_id, nation._row_id] } StreamExchange { dist: HashShard([2, 3]) } StreamHashJoin { type: LeftSemi, predicate: $0 = $5, output_indices: [1, 2, 3, 4] } StreamExchange { dist: HashShard([0]) } @@ -2240,8 +2240,8 @@ BatchFilter { predicate: ($3 > $2) } BatchScan { table: lineitem, columns: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate] } stream_plan: | - StreamMaterialize { columns: [s_name, agg#0(hidden), numwait], pk_columns: [s_name], order_descs: [numwait, s_name] } - StreamTopN { order: [$2 DESC, $0 ASC], limit: 100, offset: 0 } + StreamMaterialize { columns: [s_name, count(hidden), numwait], pk_columns: [s_name], order_descs: [numwait, s_name] } + StreamTopN { order: "[$2 DESC, $0 ASC]", limit: 100, offset: 0 } StreamExchange { dist: Single } StreamHashAgg { group_key: [$0], aggs: [count, count] } StreamExchange { dist: HashShard([0]) } @@ -2363,7 +2363,7 @@ BatchFilter { predicate: ($0 > 0.00:Decimal) AND In(Substr($1, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } BatchScan { table: customer, columns: [c_acctbal, c_phone] } stream_plan: | - StreamMaterialize { columns: [cntrycode, agg#0(hidden), numcust, totacctbal], pk_columns: [cntrycode] } + StreamMaterialize { columns: [cntrycode, count(hidden), numcust, totacctbal], pk_columns: [cntrycode] } StreamHashAgg { group_key: [$0], aggs: [count, count, sum($1)] } StreamExchange { dist: HashShard([0]) } StreamProject { exprs: [Substr($0, 1:Int32, 2:Int32), $1, $2] }