From d3df882df55e9618093ebab871d3e05c86948ed3 Mon Sep 17 00:00:00 2001 From: irenjj Date: Tue, 11 Mar 2025 23:53:17 +0800 Subject: [PATCH 01/16] Simpler to see expressions in tree explain mode --- .../physical-expr-common/src/physical_expr.rs | 2 + .../physical-expr/src/expressions/binary.rs | 5 +++ .../physical-expr/src/expressions/case.rs | 5 +++ .../physical-expr/src/expressions/cast.rs | 5 +++ .../physical-expr/src/expressions/column.rs | 4 ++ .../physical-expr/src/expressions/in_list.rs | 5 +++ .../src/expressions/is_not_null.rs | 5 +++ .../physical-expr/src/expressions/is_null.rs | 5 +++ .../physical-expr/src/expressions/like.rs | 39 +++++++++++++++++++ .../physical-expr/src/expressions/literal.rs | 5 +++ .../physical-expr/src/expressions/negative.rs | 5 +++ .../physical-expr/src/expressions/no_op.rs | 5 +++ .../physical-expr/src/expressions/not.rs | 5 +++ .../physical-expr/src/expressions/try_cast.rs | 5 +++ .../src/expressions/unknown_column.rs | 5 +++ .../physical-expr/src/scalar_function.rs | 5 +++ datafusion/physical-expr/src/utils/mod.rs | 2 + 17 files changed, 112 insertions(+) diff --git a/datafusion/physical-expr-common/src/physical_expr.rs b/datafusion/physical-expr-common/src/physical_expr.rs index cc2ff2f24790f..d90c312ffac61 100644 --- a/datafusion/physical-expr-common/src/physical_expr.rs +++ b/datafusion/physical-expr-common/src/physical_expr.rs @@ -266,6 +266,8 @@ pub trait PhysicalExpr: Send + Sync + Display + Debug + DynEq + DynHash { fn get_properties(&self, _children: &[ExprProperties]) -> Result { Ok(ExprProperties::new_unknown()) } + + fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result; } /// [`PhysicalExpr`] can't be constrained by [`Eq`] directly because it must remain object diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 1f16c5471ed77..5559e12965d2e 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -571,6 +571,11 @@ impl PhysicalExpr for BinaryExpr { _ => Ok(ExprProperties::new_unknown()), } } + + fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + // TODO: simplify + std::fmt::Display::fmt(self, f) + } } /// Casts dictionary array to result type for binary numerical operators. Such operators diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index 78606f05ae817..37ebaef962b58 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -559,6 +559,11 @@ impl PhysicalExpr for CaseExpr { )?)) } } + + fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + // TODO: simplify + std::fmt::Display::fmt(self, f) + } } /// Create a CASE expression diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index 8a093e0ae92ea..1fe34e0116b59 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -194,6 +194,11 @@ impl PhysicalExpr for CastExpr { Ok(ExprProperties::new_unknown().with_range(unbounded)) } } + + fn fmt_sql(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + // TODO: simplify + fmt::Display::fmt(self, f) + } } /// Return a PhysicalExpression representing `expr` casted to diff --git a/datafusion/physical-expr/src/expressions/column.rs b/datafusion/physical-expr/src/expressions/column.rs index 0ec985887c3f8..ab5b359847535 100644 --- a/datafusion/physical-expr/src/expressions/column.rs +++ b/datafusion/physical-expr/src/expressions/column.rs @@ -137,6 +137,10 @@ impl PhysicalExpr for Column { ) -> Result> { Ok(self) } + + fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.name) + } } impl Column { diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index dfe9a905dfeaa..7e6eeae44023b 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -398,6 +398,11 @@ impl PhysicalExpr for InListExpr { self.static_filter.clone(), ))) } + + fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + // TODO: simplify + std::fmt::Display::fmt(self, f) + } } impl PartialEq for InListExpr { diff --git a/datafusion/physical-expr/src/expressions/is_not_null.rs b/datafusion/physical-expr/src/expressions/is_not_null.rs index 47dc53d125550..0c09b1f34590c 100644 --- a/datafusion/physical-expr/src/expressions/is_not_null.rs +++ b/datafusion/physical-expr/src/expressions/is_not_null.rs @@ -104,6 +104,11 @@ impl PhysicalExpr for IsNotNullExpr { ) -> Result> { Ok(Arc::new(IsNotNullExpr::new(Arc::clone(&children[0])))) } + + fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + // TODO: simplify + std::fmt::Display::fmt(self, f) + } } /// Create an IS NOT NULL expression diff --git a/datafusion/physical-expr/src/expressions/is_null.rs b/datafusion/physical-expr/src/expressions/is_null.rs index 5e883dff997aa..f64cfc31289bf 100644 --- a/datafusion/physical-expr/src/expressions/is_null.rs +++ b/datafusion/physical-expr/src/expressions/is_null.rs @@ -103,6 +103,11 @@ impl PhysicalExpr for IsNullExpr { ) -> Result> { Ok(Arc::new(IsNullExpr::new(Arc::clone(&children[0])))) } + + fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + // TODO: simplify + std::fmt::Display::fmt(self, f) + } } /// Create an IS NULL expression diff --git a/datafusion/physical-expr/src/expressions/like.rs b/datafusion/physical-expr/src/expressions/like.rs index b26927b77f1ff..1fe82305283a5 100644 --- a/datafusion/physical-expr/src/expressions/like.rs +++ b/datafusion/physical-expr/src/expressions/like.rs @@ -145,6 +145,12 @@ impl PhysicalExpr for LikeExpr { Arc::clone(&children[1]), ))) } + + fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.expr.fmt_sql(f)?; + write!(f, " {} ", self.op_name())?; + self.pattern.fmt_sql(f) + } } /// used for optimize Dictionary like @@ -182,6 +188,7 @@ pub fn like( mod test { use super::*; use crate::expressions::col; + use crate::utils::fmt_display; use arrow::array::*; use arrow::datatypes::Field; use datafusion_common::cast::as_boolean_array; @@ -256,4 +263,36 @@ mod test { Ok(()) } + + #[test] + fn test_like_fmt_sql() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("a", DataType::Utf8, false), + Field::new("b", DataType::Utf8, false), + ]); + + let test_cases = vec![ + // (negated, case_insensitive, expected_output) + (false, false, "a LIKE b"), + (true, false, "a NOT LIKE b"), + (false, true, "a ILIKE b"), + (true, true, "a NOT ILIKE b"), + ]; + + for (negated, case_insensitive, expected) in test_cases { + let expr = like( + negated, + case_insensitive, + col("a", &schema)?, + col("b", &schema)?, + &schema, + )?; + + // Test fmt_sql formatting. + let sql_string = fmt_display(expr.as_ref()).to_string(); + assert_eq!(sql_string, expected); + } + + Ok(()) + } } diff --git a/datafusion/physical-expr/src/expressions/literal.rs b/datafusion/physical-expr/src/expressions/literal.rs index 232f9769b056a..d33a02572ef77 100644 --- a/datafusion/physical-expr/src/expressions/literal.rs +++ b/datafusion/physical-expr/src/expressions/literal.rs @@ -93,6 +93,11 @@ impl PhysicalExpr for Literal { preserves_lex_ordering: true, }) } + + fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + // TODO: simplify + std::fmt::Display::fmt(self, f) + } } /// Create a literal expression diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index 8795545274a2e..01062e064b56b 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -167,6 +167,11 @@ impl PhysicalExpr for NegativeExpr { preserves_lex_ordering: false, }) } + + fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + // TODO: simplify + std::fmt::Display::fmt(self, f) + } } /// Creates a unary expression NEGATIVE diff --git a/datafusion/physical-expr/src/expressions/no_op.rs b/datafusion/physical-expr/src/expressions/no_op.rs index c17b52f5cdfff..8eca551e55797 100644 --- a/datafusion/physical-expr/src/expressions/no_op.rs +++ b/datafusion/physical-expr/src/expressions/no_op.rs @@ -77,4 +77,9 @@ impl PhysicalExpr for NoOp { ) -> Result> { Ok(self) } + + fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + // TODO: simplify + std::fmt::Display::fmt(self, f) + } } diff --git a/datafusion/physical-expr/src/expressions/not.rs b/datafusion/physical-expr/src/expressions/not.rs index ddf7c739b692a..06397463589a7 100644 --- a/datafusion/physical-expr/src/expressions/not.rs +++ b/datafusion/physical-expr/src/expressions/not.rs @@ -175,6 +175,11 @@ impl PhysicalExpr for NotExpr { _ => internal_err!("NotExpr can only operate on Boolean datatypes"), } } + + fn fmt_sql(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + // TODO: simplify + fmt::Display::fmt(self, f) + } } /// Creates a unary expression NOT diff --git a/datafusion/physical-expr/src/expressions/try_cast.rs b/datafusion/physical-expr/src/expressions/try_cast.rs index 06f4e929992e5..f6a2e7395536a 100644 --- a/datafusion/physical-expr/src/expressions/try_cast.rs +++ b/datafusion/physical-expr/src/expressions/try_cast.rs @@ -123,6 +123,11 @@ impl PhysicalExpr for TryCastExpr { self.cast_type.clone(), ))) } + + fn fmt_sql(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + // TODO: simplify + fmt::Display::fmt(self, f) + } } /// Return a PhysicalExpression representing `expr` casted to diff --git a/datafusion/physical-expr/src/expressions/unknown_column.rs b/datafusion/physical-expr/src/expressions/unknown_column.rs index a63caf7e13056..13ac73514f45a 100644 --- a/datafusion/physical-expr/src/expressions/unknown_column.rs +++ b/datafusion/physical-expr/src/expressions/unknown_column.rs @@ -86,6 +86,11 @@ impl PhysicalExpr for UnKnownColumn { ) -> Result> { Ok(self) } + + fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + // TODO: simplify + std::fmt::Display::fmt(self, f) + } } impl Hash for UnKnownColumn { diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs index bd38fb22ccbc3..9f01791a78dcd 100644 --- a/datafusion/physical-expr/src/scalar_function.rs +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -260,6 +260,11 @@ impl PhysicalExpr for ScalarFunctionExpr { preserves_lex_ordering, }) } + + fn fmt_sql(&self, f: &mut Formatter<'_>) -> fmt::Result { + // TODO: simplify + fmt::Display::fmt(self, f) + } } /// Create a physical expression for the UDF. diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 7e4c7f0e10ba8..ec8433d726561 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -15,7 +15,9 @@ // specific language governing permissions and limitations // under the License. +mod fmt_display; mod guarantee; +pub use fmt_display::fmt_display; pub use guarantee::{Guarantee, LiteralGuarantee}; use std::borrow::Borrow; From 47ba909219095ca011909e941547eca9765aa6b0 Mon Sep 17 00:00:00 2001 From: irenjj Date: Wed, 12 Mar 2025 21:14:27 +0800 Subject: [PATCH 02/16] add more --- .../physical-expr/src/expressions/binary.rs | 98 ++++++++++++++++++- .../physical-expr/src/expressions/case.rs | 54 +++++++++- .../physical-expr/src/expressions/in_list.rs | 56 ++++++++++- .../src/expressions/is_not_null.rs | 30 +++++- .../physical-expr/src/expressions/is_null.rs | 19 +++- .../physical-expr/src/expressions/like.rs | 40 ++++---- .../physical-expr/src/expressions/literal.rs | 15 ++- .../physical-expr/src/expressions/negative.rs | 17 +++- .../physical-expr/src/expressions/no_op.rs | 1 - .../physical-expr/src/expressions/not.rs | 24 ++++- .../physical-expr/src/expressions/try_cast.rs | 28 +++++- .../src/expressions/unknown_column.rs | 1 - .../physical-expr/src/scalar_function.rs | 10 +- .../physical-expr/src/utils/fmt_display.rs | 18 ++++ datafusion/physical-expr/src/utils/mod.rs | 2 +- 15 files changed, 366 insertions(+), 47 deletions(-) create mode 100644 datafusion/physical-expr/src/utils/fmt_display.rs diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 5559e12965d2e..a22a19f6b1996 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -573,8 +573,29 @@ impl PhysicalExpr for BinaryExpr { } fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - // TODO: simplify - std::fmt::Display::fmt(self, f) + fn write_child( + f: &mut std::fmt::Formatter, + expr: &dyn PhysicalExpr, + precedence: u8, + ) -> std::fmt::Result { + if let Some(child) = expr.as_any().downcast_ref::() { + let p = child.op.precedence(); + if p == 0 || p < precedence { + write!(f, "(")?; + child.fmt_sql(f)?; + write!(f, ")") + } else { + child.fmt_sql(f) + } + } else { + expr.fmt_sql(f) + } + } + + let precedence = self.op.precedence(); + write_child(f, self.left.as_ref(), precedence)?; + write!(f, " {} ", self.op)?; + write_child(f, self.right.as_ref(), precedence) } } @@ -772,7 +793,10 @@ pub fn similar_to( #[cfg(test)] mod tests { use super::*; - use crate::expressions::{col, lit, try_cast, Column, Literal}; + use crate::{ + expressions::{col, lit, try_cast, Column, Literal}, + utils::sql_formatter, + }; use datafusion_common::plan_datafusion_err; @@ -4677,4 +4701,72 @@ mod tests { Ok(()) } + + #[test] + fn test_fmt_sql() -> Result<()> { + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + ]); + + // Test basic binary expressions + let simple_expr = binary_expr( + col("a", &schema)?, + Operator::Plus, + col("b", &schema)?, + &schema, + )?; + let display_string = simple_expr.to_string(); + assert_eq!(display_string, "a@0 + b@1"); + let sql_string = sql_formatter(&simple_expr).to_string(); + assert_eq!(sql_string, "a + b"); + + // Test nested expressions with different operator precedence + let nested_expr = binary_expr( + Arc::new(binary_expr( + col("a", &schema)?, + Operator::Plus, + col("b", &schema)?, + &schema, + )?), + Operator::Multiply, + col("b", &schema)?, + &schema, + )?; + let display_string = nested_expr.to_string(); + assert_eq!(display_string, "(a@0 + b@1) * b@1"); + let sql_string = sql_formatter(&nested_expr).to_string(); + assert_eq!(sql_string, "(a + b) * b"); + + // Test nested expressions with same operator precedence + let nested_same_prec = binary_expr( + Arc::new(binary_expr( + col("a", &schema)?, + Operator::Plus, + col("b", &schema)?, + &schema, + )?), + Operator::Plus, + col("b", &schema)?, + &schema, + )?; + let display_string = nested_same_prec.to_string(); + assert_eq!(display_string, "a@0 + b@1 + b@1"); + let sql_string = sql_formatter(&nested_same_prec).to_string(); + assert_eq!(sql_string, "a + b + b"); + + // Test with literals + let lit_expr = binary_expr( + col("a", &schema)?, + Operator::Eq, + lit(ScalarValue::Int32(Some(42))), + &schema, + )?; + let display_string = lit_expr.to_string(); + assert_eq!(display_string, "a@0 = 42"); + let sql_string = sql_formatter(&lit_expr).to_string(); + assert_eq!(sql_string, "a = 42"); + + Ok(()) + } } diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index 37ebaef962b58..e2f3bd053e923 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -561,8 +561,26 @@ impl PhysicalExpr for CaseExpr { } fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - // TODO: simplify - std::fmt::Display::fmt(self, f) + write!(f, "CASE ")?; + if let Some(e) = &self.expr { + e.fmt_sql(f)?; + write!(f, " ")?; + } + + for (w, t) in &self.when_then_expr { + write!(f, "WHEN ")?; + w.fmt_sql(f)?; + write!(f, " THEN ")?; + t.fmt_sql(f)?; + write!(f, " ")?; + } + + if let Some(e) = &self.else_expr { + write!(f, "ELSE ")?; + e.fmt_sql(f)?; + write!(f, " ")?; + } + write!(f, "END") } } @@ -580,6 +598,7 @@ mod tests { use super::*; use crate::expressions::{binary, cast, col, lit, BinaryExpr}; + use crate::utils::sql_formatter; use arrow::buffer::Buffer; use arrow::datatypes::DataType::Float64; use arrow::datatypes::*; @@ -1383,4 +1402,35 @@ mod tests { comparison_coercion(&left_type, right_type) }) } + + #[test] + fn test_fmt_sql() -> Result<()> { + let schema = Schema::new(vec![Field::new("a", DataType::Utf8, true)]); + + // CASE WHEN a = 'foo' THEN 123.3 ELSE 999 END + let when = binary(col("a", &schema)?, Operator::Eq, lit("foo"), &schema)?; + let then = lit(123.3f64); + let else_value = lit(999i32); + + let expr = generate_case_when_with_type_coercion( + None, + vec![(when, then)], + Some(else_value), + &schema, + )?; + + let display_string = expr.to_string(); + assert_eq!( + display_string, + "CASE WHEN a@0 = foo THEN 123.3 ELSE TRY_CAST(999 AS Float64) END" + ); + + let sql_string = sql_formatter(expr.as_ref()).to_string(); + assert_eq!( + sql_string, + "CASE WHEN a = foo THEN 123.3 ELSE TRY_CAST(999 AS Float64) END" + ); + + Ok(()) + } } diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 7e6eeae44023b..80ac94f79b495 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -400,8 +400,19 @@ impl PhysicalExpr for InListExpr { } fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - // TODO: simplify - std::fmt::Display::fmt(self, f) + self.expr.fmt_sql(f)?; + if self.negated { + write!(f, " NOT")?; + } + + write!(f, " IN (")?; + for (i, expr) in self.list.iter().enumerate() { + if i > 0 { + write!(f, ", ")?; + } + expr.fmt_sql(f)?; + } + write!(f, ")") } } @@ -456,6 +467,7 @@ mod tests { use super::*; use crate::expressions; use crate::expressions::{col, lit, try_cast}; + use crate::utils::sql_formatter; use datafusion_common::plan_err; use datafusion_expr::type_coercion::binary::comparison_coercion; @@ -1427,4 +1439,44 @@ mod tests { Ok(()) } + + #[test] + fn test_fmt_sql() -> Result<()> { + let schema = Schema::new(vec![Field::new("a", DataType::Utf8, true)]); + let col_a = col("a", &schema)?; + + // Test: a IN ('a', 'b') + let list = vec![lit("a"), lit("b")]; + let expr = in_list(Arc::clone(&col_a), list, &false, &schema)?; + let sql_string = sql_formatter(expr.as_ref()).to_string(); + let display_string = expr.to_string(); + assert_eq!(sql_string, "a IN (a, b)"); + assert_eq!(display_string, "Use a@0 IN (SET) ([Literal { value: Utf8(\"a\") }, Literal { value: Utf8(\"b\") }])"); + + // Test: a NOT IN ('a', 'b') + let list = vec![lit("a"), lit("b")]; + let expr = in_list(Arc::clone(&col_a), list, &true, &schema)?; + let sql_string = sql_formatter(expr.as_ref()).to_string(); + let display_string = expr.to_string(); + assert_eq!(sql_string, "a NOT IN (a, b)"); + assert_eq!(display_string, "a@0 NOT IN (SET) ([Literal { value: Utf8(\"a\") }, Literal { value: Utf8(\"b\") }])"); + + // Test: a IN ('a', 'b', NULL) + let list = vec![lit("a"), lit("b"), lit(ScalarValue::Utf8(None))]; + let expr = in_list(Arc::clone(&col_a), list, &false, &schema)?; + let sql_string = sql_formatter(expr.as_ref()).to_string(); + let display_string = expr.to_string(); + assert_eq!(sql_string, "a IN (a, b, NULL)"); + assert_eq!(display_string, "Use a@0 IN (SET) ([Literal { value: Utf8(\"a\") }, Literal { value: Utf8(\"b\") }, Literal { value: Utf8(NULL) }])"); + + // Test: a NOT IN ('a', 'b', NULL) + let list = vec![lit("a"), lit("b"), lit(ScalarValue::Utf8(None))]; + let expr = in_list(Arc::clone(&col_a), list, &true, &schema)?; + let sql_string = sql_formatter(expr.as_ref()).to_string(); + let display_string = expr.to_string(); + assert_eq!(sql_string, "a NOT IN (a, b, NULL)"); + assert_eq!(display_string, "a@0 NOT IN (SET) ([Literal { value: Utf8(\"a\") }, Literal { value: Utf8(\"b\") }, Literal { value: Utf8(NULL) }])"); + + Ok(()) + } } diff --git a/datafusion/physical-expr/src/expressions/is_not_null.rs b/datafusion/physical-expr/src/expressions/is_not_null.rs index 0c09b1f34590c..dd25ce756a06e 100644 --- a/datafusion/physical-expr/src/expressions/is_not_null.rs +++ b/datafusion/physical-expr/src/expressions/is_not_null.rs @@ -106,8 +106,8 @@ impl PhysicalExpr for IsNotNullExpr { } fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - // TODO: simplify - std::fmt::Display::fmt(self, f) + self.arg.fmt_sql(f)?; + write!(f, " IS NOT NULL") } } @@ -120,6 +120,7 @@ pub fn is_not_null(arg: Arc) -> Result> mod tests { use super::*; use crate::expressions::col; + use crate::utils::sql_formatter; use arrow::array::{ Array, BooleanArray, Float64Array, Int32Array, StringArray, UnionArray, }; @@ -192,4 +193,29 @@ mod tests { assert_eq!(expected, actual); } + + #[test] + fn test_fmt_sql() -> Result<()> { + let union_fields: UnionFields = [ + (0, Arc::new(Field::new("A", DataType::Int32, true))), + (1, Arc::new(Field::new("B", DataType::Float64, true))), + ] + .into_iter() + .collect(); + + let field = Field::new( + "my_union", + DataType::Union(union_fields, UnionMode::Sparse), + true, + ); + + let schema = Schema::new(vec![field]); + let expr = is_not_null(col("my_union", &schema).unwrap()).unwrap(); + let display_string = expr.to_string(); + assert_eq!(display_string, "my_union@0 IS NOT NULL"); + let sql_string = sql_formatter(expr.as_ref()).to_string(); + assert_eq!(sql_string, "my_union IS NOT NULL"); + + Ok(()) + } } diff --git a/datafusion/physical-expr/src/expressions/is_null.rs b/datafusion/physical-expr/src/expressions/is_null.rs index f64cfc31289bf..5df5e9b47b2e8 100644 --- a/datafusion/physical-expr/src/expressions/is_null.rs +++ b/datafusion/physical-expr/src/expressions/is_null.rs @@ -105,8 +105,8 @@ impl PhysicalExpr for IsNullExpr { } fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - // TODO: simplify - std::fmt::Display::fmt(self, f) + self.arg.fmt_sql(f)?; + write!(f, " IS NULL") } } @@ -119,6 +119,7 @@ pub fn is_null(arg: Arc) -> Result> { mod tests { use super::*; use crate::expressions::col; + use crate::utils::sql_formatter; use arrow::array::{ Array, BooleanArray, Float64Array, Int32Array, StringArray, UnionArray, }; @@ -214,4 +215,18 @@ mod tests { let expected = &BooleanArray::from(vec![false, true, false, true, false, true]); assert_eq!(expected, &result); } + + #[test] + fn test_fmt_sql() -> Result<()> { + let schema = Schema::new(vec![Field::new("a", DataType::Utf8, true)]); + + // expression: "a is null" + let expr = is_null(col("a", &schema)?).unwrap(); + let display_string = expr.to_string(); + assert_eq!(display_string, "a@0 IS NULL"); + let sql_string = sql_formatter(expr.as_ref()).to_string(); + assert_eq!(sql_string, "a IS NULL"); + + Ok(()) + } } diff --git a/datafusion/physical-expr/src/expressions/like.rs b/datafusion/physical-expr/src/expressions/like.rs index 1fe82305283a5..02755f9c6d67f 100644 --- a/datafusion/physical-expr/src/expressions/like.rs +++ b/datafusion/physical-expr/src/expressions/like.rs @@ -188,7 +188,7 @@ pub fn like( mod test { use super::*; use crate::expressions::col; - use crate::utils::fmt_display; + use crate::utils::sql_formatter; use arrow::array::*; use arrow::datatypes::Field; use datafusion_common::cast::as_boolean_array; @@ -265,33 +265,27 @@ mod test { } #[test] - fn test_like_fmt_sql() -> Result<()> { + fn test_fmt_sql() -> Result<()> { let schema = Schema::new(vec![ Field::new("a", DataType::Utf8, false), Field::new("b", DataType::Utf8, false), ]); - let test_cases = vec![ - // (negated, case_insensitive, expected_output) - (false, false, "a LIKE b"), - (true, false, "a NOT LIKE b"), - (false, true, "a ILIKE b"), - (true, true, "a NOT ILIKE b"), - ]; - - for (negated, case_insensitive, expected) in test_cases { - let expr = like( - negated, - case_insensitive, - col("a", &schema)?, - col("b", &schema)?, - &schema, - )?; - - // Test fmt_sql formatting. - let sql_string = fmt_display(expr.as_ref()).to_string(); - assert_eq!(sql_string, expected); - } + let expr = like( + false, + false, + col("a", &schema)?, + col("b", &schema)?, + &schema, + )?; + + // Display format + let display_string = expr.to_string(); + assert_eq!(display_string, "a@0 LIKE b@1"); + + // fmt_sql format + let sql_string = sql_formatter(expr.as_ref()).to_string(); + assert_eq!(sql_string, "a LIKE b"); Ok(()) } diff --git a/datafusion/physical-expr/src/expressions/literal.rs b/datafusion/physical-expr/src/expressions/literal.rs index d33a02572ef77..7ad0aeb7c73b2 100644 --- a/datafusion/physical-expr/src/expressions/literal.rs +++ b/datafusion/physical-expr/src/expressions/literal.rs @@ -95,7 +95,6 @@ impl PhysicalExpr for Literal { } fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - // TODO: simplify std::fmt::Display::fmt(self, f) } } @@ -110,6 +109,8 @@ pub fn lit(value: T) -> Arc { #[cfg(test)] mod tests { + use crate::utils::sql_formatter; + use super::*; use arrow::array::Int32Array; @@ -141,4 +142,16 @@ mod tests { Ok(()) } + + #[test] + fn test_fmt_sql() -> Result<()> { + // create and evaluate a literal expression + let expr = lit(42i32); + let display_string = expr.to_string(); + assert_eq!(display_string, "42"); + let sql_string = sql_formatter(expr.as_ref()).to_string(); + assert_eq!(sql_string, "42"); + + Ok(()) + } } diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index 01062e064b56b..72f721397b1f2 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -169,8 +169,9 @@ impl PhysicalExpr for NegativeExpr { } fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - // TODO: simplify - std::fmt::Display::fmt(self, f) + write!(f, "(- ")?; + self.arg.fmt_sql(f)?; + write!(f, ")") } } @@ -200,6 +201,7 @@ pub fn negative( mod tests { use super::*; use crate::expressions::{col, Column}; + use crate::utils::sql_formatter; use arrow::array::*; use arrow::datatypes::DataType::{Float32, Float64, Int16, Int32, Int64, Int8}; @@ -384,4 +386,15 @@ mod tests { matches!(expr, DataFusionError::Plan(_)); Ok(()) } + + #[test] + fn test_fmt_sql() -> Result<()> { + let expr = NegativeExpr::new(Arc::new(Column::new("a", 0))); + let display_string = expr.to_string(); + assert_eq!(display_string, "(- a@0)"); + let sql_string = sql_formatter(&expr).to_string(); + assert_eq!(sql_string, "(- a)"); + + Ok(()) + } } diff --git a/datafusion/physical-expr/src/expressions/no_op.rs b/datafusion/physical-expr/src/expressions/no_op.rs index 8eca551e55797..24d2f4d9e074d 100644 --- a/datafusion/physical-expr/src/expressions/no_op.rs +++ b/datafusion/physical-expr/src/expressions/no_op.rs @@ -79,7 +79,6 @@ impl PhysicalExpr for NoOp { } fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - // TODO: simplify std::fmt::Display::fmt(self, f) } } diff --git a/datafusion/physical-expr/src/expressions/not.rs b/datafusion/physical-expr/src/expressions/not.rs index 06397463589a7..e370661d0faf8 100644 --- a/datafusion/physical-expr/src/expressions/not.rs +++ b/datafusion/physical-expr/src/expressions/not.rs @@ -177,8 +177,8 @@ impl PhysicalExpr for NotExpr { } fn fmt_sql(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - // TODO: simplify - fmt::Display::fmt(self, f) + write!(f, "NOT ")?; + self.arg.fmt_sql(f) } } @@ -192,7 +192,10 @@ mod tests { use std::sync::LazyLock; use super::*; - use crate::expressions::{col, Column}; + use crate::{ + expressions::{col, Column}, + utils::sql_formatter, + }; use arrow::{array::BooleanArray, datatypes::*}; @@ -327,6 +330,21 @@ mod tests { Ok(()) } + #[test] + fn test_fmt_sql() -> Result<()> { + let schema = schema(); + + let expr = not(col("a", &schema)?)?; + + let display_string = expr.to_string(); + assert_eq!(display_string, "NOT a@0"); + + let sql_string = sql_formatter(expr.as_ref()).to_string(); + assert_eq!(sql_string, "NOT a"); + + Ok(()) + } + fn schema() -> SchemaRef { static SCHEMA: LazyLock = LazyLock::new(|| { Arc::new(Schema::new(vec![Field::new("a", DataType::Boolean, true)])) diff --git a/datafusion/physical-expr/src/expressions/try_cast.rs b/datafusion/physical-expr/src/expressions/try_cast.rs index f6a2e7395536a..6933cb2e2638f 100644 --- a/datafusion/physical-expr/src/expressions/try_cast.rs +++ b/datafusion/physical-expr/src/expressions/try_cast.rs @@ -125,8 +125,9 @@ impl PhysicalExpr for TryCastExpr { } fn fmt_sql(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - // TODO: simplify - fmt::Display::fmt(self, f) + write!(f, "TRY_CAST(")?; + self.expr.fmt_sql(f)?; + write!(f, " AS {:?})", self.cast_type) } } @@ -153,6 +154,7 @@ pub fn try_cast( mod tests { use super::*; use crate::expressions::col; + use crate::utils::sql_formatter; use arrow::array::{ Decimal128Array, Decimal128Builder, StringArray, Time64NanosecondArray, }; @@ -578,4 +580,26 @@ mod tests { .with_precision_and_scale(precision, scale) .unwrap() } + + #[test] + fn test_fmt_sql() -> Result<()> { + let schema = Schema::new(vec![Field::new("a", DataType::Int32, true)]); + + // Test numeric casting + let expr = try_cast(col("a", &schema)?, &schema, DataType::Int64)?; + let display_string = expr.to_string(); + assert_eq!(display_string, "TRY_CAST(a@0 AS Int64)"); + let sql_string = sql_formatter(expr.as_ref()).to_string(); + assert_eq!(sql_string, "TRY_CAST(a AS Int64)"); + + // Test string casting + let schema = Schema::new(vec![Field::new("b", DataType::Utf8, true)]); + let expr = try_cast(col("b", &schema)?, &schema, DataType::Int32)?; + let display_string = expr.to_string(); + assert_eq!(display_string, "TRY_CAST(b@0 AS Int32)"); + let sql_string = sql_formatter(expr.as_ref()).to_string(); + assert_eq!(sql_string, "TRY_CAST(b AS Int32)"); + + Ok(()) + } } diff --git a/datafusion/physical-expr/src/expressions/unknown_column.rs b/datafusion/physical-expr/src/expressions/unknown_column.rs index 13ac73514f45a..2face4eb6bdb6 100644 --- a/datafusion/physical-expr/src/expressions/unknown_column.rs +++ b/datafusion/physical-expr/src/expressions/unknown_column.rs @@ -88,7 +88,6 @@ impl PhysicalExpr for UnKnownColumn { } fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - // TODO: simplify std::fmt::Display::fmt(self, f) } } diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs index 9f01791a78dcd..9b626b34b151e 100644 --- a/datafusion/physical-expr/src/scalar_function.rs +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -262,8 +262,14 @@ impl PhysicalExpr for ScalarFunctionExpr { } fn fmt_sql(&self, f: &mut Formatter<'_>) -> fmt::Result { - // TODO: simplify - fmt::Display::fmt(self, f) + write!(f, "{}(", self.name)?; + for (i, expr) in self.args.iter().enumerate() { + if i > 0 { + write!(f, ", ")?; + } + expr.fmt_sql(f)?; + } + write!(f, ")") } } diff --git a/datafusion/physical-expr/src/utils/fmt_display.rs b/datafusion/physical-expr/src/utils/fmt_display.rs new file mode 100644 index 0000000000000..8c39ad785bbeb --- /dev/null +++ b/datafusion/physical-expr/src/utils/fmt_display.rs @@ -0,0 +1,18 @@ +use std::fmt::{self, Formatter}; + +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; + +pub fn sql_formatter(expr: &dyn PhysicalExpr) -> impl fmt::Display + '_ { + struct Wrapper<'a> { + expr: &'a dyn PhysicalExpr, + } + + impl fmt::Display for Wrapper<'_> { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + self.expr.fmt_sql(f)?; + Ok(()) + } + } + + Wrapper { expr } +} diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index ec8433d726561..52c521443afc2 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -17,7 +17,7 @@ mod fmt_display; mod guarantee; -pub use fmt_display::fmt_display; +pub use fmt_display::sql_formatter; pub use guarantee::{Guarantee, LiteralGuarantee}; use std::borrow::Borrow; From 613c0c2eb3d65efb46121fe697cce12573e38569 Mon Sep 17 00:00:00 2001 From: irenjj Date: Wed, 12 Mar 2025 21:19:34 +0800 Subject: [PATCH 03/16] add cast --- .../physical-expr/src/expressions/cast.rs | 31 +++++++++++++++++-- 1 file changed, 28 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index 1fe34e0116b59..72222321c75f7 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -196,8 +196,11 @@ impl PhysicalExpr for CastExpr { } fn fmt_sql(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - // TODO: simplify - fmt::Display::fmt(self, f) + write!(f, "CAST(")?; + self.expr.fmt_sql(f)?; + write!(f, " AS {:?}", self.cast_type)?; + + write!(f, ")") } } @@ -237,7 +240,7 @@ pub fn cast( mod tests { use super::*; - use crate::expressions::column::col; + use crate::{expressions::column::col, utils::sql_formatter}; use arrow::{ array::{ @@ -771,4 +774,26 @@ mod tests { expression.evaluate(&batch)?; Ok(()) } + + #[test] + fn test_fmt_sql() -> Result<()> { + let schema = Schema::new(vec![Field::new("a", Int32, true)]); + + // Test numeric casting + let expr = cast(col("a", &schema)?, &schema, Int64)?; + let display_string = expr.to_string(); + assert_eq!(display_string, "CAST(a@0 AS Int64)"); + let sql_string = sql_formatter(expr.as_ref()).to_string(); + assert_eq!(sql_string, "CAST(a AS Int64)"); + + // Test string casting + let schema = Schema::new(vec![Field::new("b", Utf8, true)]); + let expr = cast(col("b", &schema)?, &schema, Int32)?; + let display_string = expr.to_string(); + assert_eq!(display_string, "CAST(b@0 AS Int32)"); + let sql_string = sql_formatter(expr.as_ref()).to_string(); + assert_eq!(sql_string, "CAST(b AS Int32)"); + + Ok(()) + } } From f49ec2ebab2ac8c0d86ccdf91a56715e1214f8a7 Mon Sep 17 00:00:00 2001 From: irenjj Date: Wed, 12 Mar 2025 21:22:25 +0800 Subject: [PATCH 04/16] rename --- .../physical-expr/src/utils/fmt_display.rs | 18 ------------------ datafusion/physical-expr/src/utils/mod.rs | 4 ++-- 2 files changed, 2 insertions(+), 20 deletions(-) delete mode 100644 datafusion/physical-expr/src/utils/fmt_display.rs diff --git a/datafusion/physical-expr/src/utils/fmt_display.rs b/datafusion/physical-expr/src/utils/fmt_display.rs deleted file mode 100644 index 8c39ad785bbeb..0000000000000 --- a/datafusion/physical-expr/src/utils/fmt_display.rs +++ /dev/null @@ -1,18 +0,0 @@ -use std::fmt::{self, Formatter}; - -use datafusion_physical_expr_common::physical_expr::PhysicalExpr; - -pub fn sql_formatter(expr: &dyn PhysicalExpr) -> impl fmt::Display + '_ { - struct Wrapper<'a> { - expr: &'a dyn PhysicalExpr, - } - - impl fmt::Display for Wrapper<'_> { - fn fmt(&self, f: &mut Formatter) -> fmt::Result { - self.expr.fmt_sql(f)?; - Ok(()) - } - } - - Wrapper { expr } -} diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 52c521443afc2..3b903129ca604 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -mod fmt_display; +mod sql_formatter; mod guarantee; -pub use fmt_display::sql_formatter; +pub use sql_formatter::sql_formatter; pub use guarantee::{Guarantee, LiteralGuarantee}; use std::borrow::Borrow; From 523b078793bc0b2101010280b511c6ef385d00d5 Mon Sep 17 00:00:00 2001 From: irenjj Date: Wed, 12 Mar 2025 21:24:31 +0800 Subject: [PATCH 05/16] fmt --- datafusion/physical-expr/src/utils/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 3b903129ca604..b4a918ef6f617 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. -mod sql_formatter; mod guarantee; -pub use sql_formatter::sql_formatter; +mod sql_formatter; pub use guarantee::{Guarantee, LiteralGuarantee}; +pub use sql_formatter::sql_formatter; use std::borrow::Borrow; use std::sync::Arc; From bf8533924b55cac649bdb8fa78600f6c31966852 Mon Sep 17 00:00:00 2001 From: irenjj Date: Wed, 12 Mar 2025 21:31:14 +0800 Subject: [PATCH 06/16] add sql_formatter --- .../physical-expr/src/utils/sql_formatter.rs | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100644 datafusion/physical-expr/src/utils/sql_formatter.rs diff --git a/datafusion/physical-expr/src/utils/sql_formatter.rs b/datafusion/physical-expr/src/utils/sql_formatter.rs new file mode 100644 index 0000000000000..8c39ad785bbeb --- /dev/null +++ b/datafusion/physical-expr/src/utils/sql_formatter.rs @@ -0,0 +1,18 @@ +use std::fmt::{self, Formatter}; + +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; + +pub fn sql_formatter(expr: &dyn PhysicalExpr) -> impl fmt::Display + '_ { + struct Wrapper<'a> { + expr: &'a dyn PhysicalExpr, + } + + impl fmt::Display for Wrapper<'_> { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + self.expr.fmt_sql(f)?; + Ok(()) + } + } + + Wrapper { expr } +} From c1a22eecb025c4e1e94f568a6d501503e0455d58 Mon Sep 17 00:00:00 2001 From: irenjj Date: Wed, 12 Mar 2025 21:36:24 +0800 Subject: [PATCH 07/16] add license --- .../physical-expr/src/utils/sql_formatter.rs | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/datafusion/physical-expr/src/utils/sql_formatter.rs b/datafusion/physical-expr/src/utils/sql_formatter.rs index 8c39ad785bbeb..495ee919383c9 100644 --- a/datafusion/physical-expr/src/utils/sql_formatter.rs +++ b/datafusion/physical-expr/src/utils/sql_formatter.rs @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use std::fmt::{self, Formatter}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; From 89890ecbc7c0f17b0dc864c7dac029a770c3e9cd Mon Sep 17 00:00:00 2001 From: irenjj Date: Wed, 12 Mar 2025 21:42:02 +0800 Subject: [PATCH 08/16] fix test --- datafusion/proto/tests/cases/roundtrip_physical_plan.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index b5bfef99a6f39..738ac9064d87d 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -16,7 +16,7 @@ // under the License. use std::any::Any; -use std::fmt::Display; +use std::fmt::{Display, Formatter}; use std::ops::Deref; use std::sync::Arc; use std::vec; @@ -860,6 +860,10 @@ fn roundtrip_parquet_exec_with_custom_predicate_expr() -> Result<()> { ) -> Result> { todo!() } + + fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + std::fmt::Display::fmt(self, f) + } } #[derive(Debug)] From 04b9599bec06fdcb67e0ecc063730d18b168de23 Mon Sep 17 00:00:00 2001 From: irenjj Date: Wed, 12 Mar 2025 22:20:07 +0800 Subject: [PATCH 09/16] fix doc --- datafusion/physical-expr-common/src/sort_expr.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 38b820edc544e..fe5c2ecfdf621 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -37,7 +37,7 @@ use itertools::Itertools; /// Example: /// ``` /// # use std::any::Any; -/// # use std::fmt::Display; +/// # use std::fmt::{Display, Formatter}; /// # use std::hash::Hasher; /// # use std::sync::Arc; /// # use arrow::array::RecordBatch; @@ -58,6 +58,7 @@ use itertools::Itertools; /// # fn evaluate(&self, batch: &RecordBatch) -> Result {todo!() } /// # fn children(&self) -> Vec<&Arc> {todo!()} /// # fn with_new_children(self: Arc, children: Vec>) -> Result> {todo!()} +/// # fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { todo!() } /// # } /// # impl Display for MyPhysicalExpr { /// # fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!(f, "a") } From 84a696166e98c586b1f9d875d7740eb74c5056a6 Mon Sep 17 00:00:00 2001 From: irenjj Date: Fri, 14 Mar 2025 19:10:55 +0800 Subject: [PATCH 10/16] Update datafusion/physical-expr-common/src/physical_expr.rs Co-authored-by: Andrew Lamb --- datafusion/physical-expr-common/src/physical_expr.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/datafusion/physical-expr-common/src/physical_expr.rs b/datafusion/physical-expr-common/src/physical_expr.rs index d90c312ffac61..0b191cd1be6fb 100644 --- a/datafusion/physical-expr-common/src/physical_expr.rs +++ b/datafusion/physical-expr-common/src/physical_expr.rs @@ -267,6 +267,12 @@ pub trait PhysicalExpr: Send + Sync + Display + Debug + DynEq + DynHash { Ok(ExprProperties::new_unknown()) } + /// Format this `PhysicalExpr` in nice human readable "SQL" format + /// + /// Specifically, this format is designed to be readable by humans, at the + /// expense of details. Use `Display` or `Debug` for more detailed + /// representation. + /// fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result; } From 1c62467c09d3d4067f60f66d1f0a3e0ea2ed17be Mon Sep 17 00:00:00 2001 From: irenjj Date: Fri, 14 Mar 2025 19:46:28 +0800 Subject: [PATCH 11/16] add example --- .../physical-expr-common/src/physical_expr.rs | 8 +- datafusion/physical-plan/src/projection.rs | 10 +- .../sqllogictest/test_files/explain_tree.slt | 704 +++++++++--------- 3 files changed, 343 insertions(+), 379 deletions(-) diff --git a/datafusion/physical-expr-common/src/physical_expr.rs b/datafusion/physical-expr-common/src/physical_expr.rs index 0b191cd1be6fb..22a35dbcab27f 100644 --- a/datafusion/physical-expr-common/src/physical_expr.rs +++ b/datafusion/physical-expr-common/src/physical_expr.rs @@ -53,6 +53,12 @@ pub type PhysicalExprRef = Arc; /// * [`SessionContext::create_physical_expr`]: A high level API /// * [`create_physical_expr`]: A low level API /// +/// # Formatting `PhysicalExpr` as strings +/// There are three ways to format `PhysicalExpr` as a string: +/// * [`Debug`]: Standard Rust debugging format (e.g. `Constant { value: ... }`) +/// * [`Display`]: Detailed SQL-like format that shows expression structure (e.g. (`Utf8 ("foobar")`). This is often used for debugging and tests +/// * [`Self::fmt_sql`]: SQL-like human readable format (e.g. ('foobar')`) +/// /// [`SessionContext::create_physical_expr`]: https://docs.rs/datafusion/latest/datafusion/execution/context/struct.SessionContext.html#method.create_physical_expr /// [`PhysicalPlanner`]: https://docs.rs/datafusion/latest/datafusion/physical_planner/trait.PhysicalPlanner.html /// [`Expr`]: https://docs.rs/datafusion/latest/datafusion/logical_expr/enum.Expr.html @@ -272,7 +278,7 @@ pub trait PhysicalExpr: Send + Sync + Display + Debug + DynEq + DynHash { /// Specifically, this format is designed to be readable by humans, at the /// expense of details. Use `Display` or `Debug` for more detailed /// representation. - /// + /// fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result; } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 3f901311a053c..16f34be8ad81f 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -168,13 +168,9 @@ impl DisplayAs for ProjectionExec { write!(f, "ProjectionExec: expr=[{}]", expr.join(", ")) } DisplayFormatType::TreeRender => { - for (i, (e, alias)) in self.expr().iter().enumerate() { - let e = e.to_string(); - if &e == alias { - writeln!(f, "expr{i}={e}")?; - } else { - writeln!(f, "{alias}={e}")?; - } + for (e, _) in self.expr().iter() { + e.fmt_sql(f)?; + writeln!(f)?; } Ok(()) } diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 9e047133fcfcf..5ed74455c87de 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -704,29 +704,26 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(*) ROWS BETWEEN │ -05)│ UNBOUNDED PRECEDING │ -06)│ AND UNBOUNDED FOLLOWING: │ -07)│ count(Int64(1)) ROWS │ -08)│ BETWEEN UNBOUNDED │ -09)│ PRECEDING AND UNBOUNDED │ -10)│ FOLLOWING@0 │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ WindowAggExec │ -14)│ -------------------- │ -15)│ select_list: │ -16)│ count(Int64(1)) ROWS │ -17)│ BETWEEN UNBOUNDED │ -18)│ PRECEDING AND UNBOUNDED │ -19)│ FOLLOWING │ -20)└─────────────┬─────────────┘ -21)┌─────────────┴─────────────┐ -22)│ DataSourceExec │ -23)│ -------------------- │ -24)│ files: 1 │ -25)│ format: csv │ -26)└───────────────────────────┘ +04)│ count(Int64(1)) ROWS │ +05)│ BETWEEN UNBOUNDED │ +06)│ PRECEDING AND UNBOUNDED │ +07)│ FOLLOWING │ +08)└─────────────┬─────────────┘ +09)┌─────────────┴─────────────┐ +10)│ WindowAggExec │ +11)│ -------------------- │ +12)│ select_list: │ +13)│ count(Int64(1)) ROWS │ +14)│ BETWEEN UNBOUNDED │ +15)│ PRECEDING AND UNBOUNDED │ +16)│ FOLLOWING │ +17)└─────────────┬─────────────┘ +18)┌─────────────┴─────────────┐ +19)│ DataSourceExec │ +20)│ -------------------- │ +21)│ files: 1 │ +22)│ format: csv │ +23)└───────────────────────────┘ # Query with bounded window agg. query TT @@ -739,43 +736,42 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ rolling_sum: │ -05)│ sum(t1.v1) ORDER BY [t1.v1│ -06)│ ASC NULLS LAST] ROWS │ -07)│ BETWEEN 1 PRECEDING │ -08)│ AND CURRENT ROW@1 │ -09)│ │ -10)│ v1: v1@0 │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ BoundedWindowAggExec │ -14)│ -------------------- │ -15)│ mode: Sorted │ -16)│ │ -17)│ select_list: │ -18)│ sum(t1.v1) ORDER BY [t1.v1│ -19)│ ASC NULLS LAST] ROWS │ -20)│ BETWEEN 1 PRECEDING │ -21)│ AND CURRENT ROW │ -22)└─────────────┬─────────────┘ -23)┌─────────────┴─────────────┐ -24)│ SortExec │ -25)│ -------------------- │ -26)│ v1@0 ASC NULLS LAST │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ ProjectionExec │ -30)│ -------------------- │ -31)│ v1: value@0 │ -32)└─────────────┬─────────────┘ -33)┌─────────────┴─────────────┐ -34)│ LazyMemoryExec │ -35)│ -------------------- │ -36)│ batch_generators: │ -37)│ generate_series: start=1, │ -38)│ end=1000, batch_size │ -39)│ =8192 │ -40)└───────────────────────────┘ +04)│ sum(t1.v1) ORDER BY [t1.v1│ +05)│ ASC NULLS LAST] ROWS │ +06)│ BETWEEN 1 PRECEDING │ +07)│ AND CURRENT ROW │ +08)│ │ +09)│ v1 │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ BoundedWindowAggExec │ +13)│ -------------------- │ +14)│ mode: Sorted │ +15)│ │ +16)│ select_list: │ +17)│ sum(t1.v1) ORDER BY [t1.v1│ +18)│ ASC NULLS LAST] ROWS │ +19)│ BETWEEN 1 PRECEDING │ +20)│ AND CURRENT ROW │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ SortExec │ +24)│ -------------------- │ +25)│ v1@0 ASC NULLS LAST │ +26)└─────────────┬─────────────┘ +27)┌─────────────┴─────────────┐ +28)│ ProjectionExec │ +29)│ -------------------- │ +30)│ value │ +31)└─────────────┬─────────────┘ +32)┌─────────────┴─────────────┐ +33)│ LazyMemoryExec │ +34)│ -------------------- │ +35)│ batch_generators: │ +36)│ generate_series: start=1, │ +37)│ end=1000, batch_size │ +38)│ =8192 │ +39)└───────────────────────────┘ query TT explain select @@ -787,39 +783,33 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(*) ROWS BETWEEN │ -05)│ UNBOUNDED PRECEDING │ -06)│ AND UNBOUNDED FOLLOWING: │ -07)│ count(Int64(1)) ROWS │ -08)│ BETWEEN UNBOUNDED │ -09)│ PRECEDING AND UNBOUNDED │ -10)│ FOLLOWING@0 │ -11)│ │ -12)│ row_number() ROWS BETWEEN │ -13)│ UNBOUNDED PRECEDING AND │ -14)│ UNBOUNDED FOLLOWING: │ -15)│ row_number() ROWS BETWEEN │ -16)│ UNBOUNDED PRECEDING AND │ -17)│ UNBOUNDED FOLLOWING@1 │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ WindowAggExec │ -21)│ -------------------- │ -22)│ select_list: │ -23)│ count(Int64(1)) ROWS │ -24)│ BETWEEN UNBOUNDED │ -25)│ PRECEDING AND UNBOUNDED │ -26)│ FOLLOWING, row_number() │ -27)│ ROWS BETWEEN UNBOUNDED │ -28)│ PRECEDING AND UNBOUNDED │ -29)│ FOLLOWING │ -30)└─────────────┬─────────────┘ -31)┌─────────────┴─────────────┐ -32)│ DataSourceExec │ -33)│ -------------------- │ -34)│ files: 1 │ -35)│ format: csv │ -36)└───────────────────────────┘ +04)│ count(Int64(1)) ROWS │ +05)│ BETWEEN UNBOUNDED │ +06)│ PRECEDING AND UNBOUNDED │ +07)│ FOLLOWING │ +08)│ │ +09)│ row_number() ROWS BETWEEN │ +10)│ UNBOUNDED PRECEDING AND │ +11)│ UNBOUNDED FOLLOWING │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ WindowAggExec │ +15)│ -------------------- │ +16)│ select_list: │ +17)│ count(Int64(1)) ROWS │ +18)│ BETWEEN UNBOUNDED │ +19)│ PRECEDING AND UNBOUNDED │ +20)│ FOLLOWING, row_number() │ +21)│ ROWS BETWEEN UNBOUNDED │ +22)│ PRECEDING AND UNBOUNDED │ +23)│ FOLLOWING │ +24)└─────────────┬─────────────┘ +25)┌─────────────┴─────────────┐ +26)│ DataSourceExec │ +27)│ -------------------- │ +28)│ files: 1 │ +29)│ format: csv │ +30)└───────────────────────────┘ # Query for sort. query TT @@ -865,30 +855,28 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col: │ -05)│ bigint_col@1 │ +04)│ CAST(int_col AS Int64) + │ +05)│ bigint_col │ 06)│ │ -07)│ int_col: int_col@0 │ +07)│ bigint_col │ 08)│ │ -09)│ sum_col: │ -10)│ CAST(int_col@0 AS Int64) +│ -11)│ bigint_col@1 │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ output_partition_count: │ -17)│ 1 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: csv │ -27)└───────────────────────────┘ +09)│ int_col │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ RepartitionExec │ +13)│ -------------------- │ +14)│ output_partition_count: │ +15)│ 1 │ +16)│ │ +17)│ partitioning_scheme: │ +18)│ RoundRobinBatch(4) │ +19)└─────────────┬─────────────┘ +20)┌─────────────┴─────────────┐ +21)│ DataSourceExec │ +22)│ -------------------- │ +23)│ files: 1 │ +24)│ format: csv │ +25)└───────────────────────────┘ query TT explain select @@ -904,67 +892,56 @@ physical_plan 05)│ .int_col DESC NULLS │ 06)│ FIRST] RANGE BETWEEN │ 07)│ UNBOUNDED PRECEDING AND │ -08)│ CURRENT ROW: │ -09)│ rank() ORDER BY [table1 │ -10)│ .int_col DESC NULLS │ -11)│ FIRST] RANGE BETWEEN │ -12)│ UNBOUNDED PRECEDING AND │ -13)│ CURRENT ROW@1 │ -14)│ │ -15)│ row_number() ORDER BY │ -16)│ [table1.int_col ASC │ -17)│ NULLS LAST] RANGE │ -18)│ BETWEEN UNBOUNDED │ -19)│ PRECEDING AND CURRENT │ -20)│ ROW: │ -21)│ row_number() ORDER BY │ -22)│ [table1.int_col ASC │ -23)│ NULLS LAST] RANGE │ -24)│ BETWEEN UNBOUNDED │ -25)│ PRECEDING AND CURRENT │ -26)│ ROW@2 │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ BoundedWindowAggExec │ -30)│ -------------------- │ -31)│ mode: Sorted │ -32)│ │ -33)│ select_list: │ -34)│ row_number() ORDER BY │ -35)│ [table1.int_col ASC │ -36)│ NULLS LAST] RANGE │ -37)│ BETWEEN UNBOUNDED │ -38)│ PRECEDING AND CURRENT │ -39)│ ROW │ -40)└─────────────┬─────────────┘ -41)┌─────────────┴─────────────┐ -42)│ SortExec │ -43)│ -------------------- │ -44)│ int_col@0 ASC NULLS LAST │ -45)└─────────────┬─────────────┘ -46)┌─────────────┴─────────────┐ -47)│ BoundedWindowAggExec │ -48)│ -------------------- │ -49)│ mode: Sorted │ -50)│ │ -51)│ select_list: │ -52)│ rank() ORDER BY [table1 │ -53)│ .int_col DESC NULLS │ -54)│ FIRST] RANGE BETWEEN │ -55)│ UNBOUNDED PRECEDING AND │ -56)│ CURRENT ROW │ -57)└─────────────┬─────────────┘ -58)┌─────────────┴─────────────┐ -59)│ SortExec │ -60)│ -------------------- │ -61)│ int_col@0 DESC │ -62)└─────────────┬─────────────┘ -63)┌─────────────┴─────────────┐ -64)│ DataSourceExec │ -65)│ -------------------- │ -66)│ files: 1 │ -67)│ format: csv │ -68)└───────────────────────────┘ +08)│ CURRENT ROW │ +09)│ │ +10)│ row_number() ORDER BY │ +11)│ [table1.int_col ASC │ +12)│ NULLS LAST] RANGE │ +13)│ BETWEEN UNBOUNDED │ +14)│ PRECEDING AND CURRENT │ +15)│ ROW │ +16)└─────────────┬─────────────┘ +17)┌─────────────┴─────────────┐ +18)│ BoundedWindowAggExec │ +19)│ -------------------- │ +20)│ mode: Sorted │ +21)│ │ +22)│ select_list: │ +23)│ row_number() ORDER BY │ +24)│ [table1.int_col ASC │ +25)│ NULLS LAST] RANGE │ +26)│ BETWEEN UNBOUNDED │ +27)│ PRECEDING AND CURRENT │ +28)│ ROW │ +29)└─────────────┬─────────────┘ +30)┌─────────────┴─────────────┐ +31)│ SortExec │ +32)│ -------------------- │ +33)│ int_col@0 ASC NULLS LAST │ +34)└─────────────┬─────────────┘ +35)┌─────────────┴─────────────┐ +36)│ BoundedWindowAggExec │ +37)│ -------------------- │ +38)│ mode: Sorted │ +39)│ │ +40)│ select_list: │ +41)│ rank() ORDER BY [table1 │ +42)│ .int_col DESC NULLS │ +43)│ FIRST] RANGE BETWEEN │ +44)│ UNBOUNDED PRECEDING AND │ +45)│ CURRENT ROW │ +46)└─────────────┬─────────────┘ +47)┌─────────────┴─────────────┐ +48)│ SortExec │ +49)│ -------------------- │ +50)│ int_col@0 DESC │ +51)└─────────────┬─────────────┘ +52)┌─────────────┴─────────────┐ +53)│ DataSourceExec │ +54)│ -------------------- │ +55)│ files: 1 │ +56)│ format: csv │ +57)└───────────────────────────┘ # Query with projection on parquet query TT @@ -974,30 +951,28 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col: │ -05)│ bigint_col@1 │ +04)│ CAST(int_col AS Int64) + │ +05)│ bigint_col │ 06)│ │ -07)│ int_col: int_col@0 │ +07)│ bigint_col │ 08)│ │ -09)│ sum_col: │ -10)│ CAST(int_col@0 AS Int64) +│ -11)│ bigint_col@1 │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ output_partition_count: │ -17)│ 1 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: parquet │ -27)└───────────────────────────┘ +09)│ int_col │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ RepartitionExec │ +13)│ -------------------- │ +14)│ output_partition_count: │ +15)│ 1 │ +16)│ │ +17)│ partitioning_scheme: │ +18)│ RoundRobinBatch(4) │ +19)└─────────────┬─────────────┘ +20)┌─────────────┴─────────────┐ +21)│ DataSourceExec │ +22)│ -------------------- │ +23)│ files: 1 │ +24)│ format: parquet │ +25)└───────────────────────────┘ # Query with projection on memory @@ -1008,22 +983,20 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col: │ -05)│ bigint_col@1 │ +04)│ CAST(int_col AS Int64) + │ +05)│ bigint_col │ 06)│ │ -07)│ int_col: int_col@0 │ +07)│ bigint_col │ 08)│ │ -09)│ sum_col: │ -10)│ CAST(int_col@0 AS Int64) +│ -11)│ bigint_col@1 │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ DataSourceExec │ -15)│ -------------------- │ -16)│ bytes: 1560 │ -17)│ format: memory │ -18)│ rows: 1 │ -19)└───────────────────────────┘ +09)│ int_col │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ DataSourceExec │ +13)│ -------------------- │ +14)│ bytes: 1560 │ +15)│ format: memory │ +16)│ rows: 1 │ +17)└───────────────────────────┘ # Query with projection on json query TT @@ -1033,29 +1006,27 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col: │ -05)│ bigint_col@0 │ -06)│ │ -07)│ int_col: int_col@1 │ -08)│ │ -09)│ sum_col: │ -10)│ int_col@1 + bigint_col@0 │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ RepartitionExec │ -14)│ -------------------- │ -15)│ output_partition_count: │ -16)│ 1 │ -17)│ │ -18)│ partitioning_scheme: │ -19)│ RoundRobinBatch(4) │ -20)└─────────────┬─────────────┘ -21)┌─────────────┴─────────────┐ -22)│ DataSourceExec │ -23)│ -------------------- │ -24)│ files: 1 │ -25)│ format: json │ -26)└───────────────────────────┘ +04)│ bigint_col │ +05)│ │ +06)│ int_col │ +07)│ │ +08)│ int_col + bigint_col │ +09)└─────────────┬─────────────┘ +10)┌─────────────┴─────────────┐ +11)│ RepartitionExec │ +12)│ -------------------- │ +13)│ output_partition_count: │ +14)│ 1 │ +15)│ │ +16)│ partitioning_scheme: │ +17)│ RoundRobinBatch(4) │ +18)└─────────────┬─────────────┘ +19)┌─────────────┴─────────────┐ +20)│ DataSourceExec │ +21)│ -------------------- │ +22)│ files: 1 │ +23)│ format: json │ +24)└───────────────────────────┘ # Query with projection on arrow @@ -1066,30 +1037,28 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col: │ -05)│ bigint_col@1 │ +04)│ CAST(int_col AS Int64) + │ +05)│ bigint_col │ 06)│ │ -07)│ int_col: int_col@0 │ +07)│ bigint_col │ 08)│ │ -09)│ sum_col: │ -10)│ CAST(int_col@0 AS Int64) +│ -11)│ bigint_col@1 │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ output_partition_count: │ -17)│ 1 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: arrow │ -27)└───────────────────────────┘ +09)│ int_col │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ RepartitionExec │ +13)│ -------------------- │ +14)│ output_partition_count: │ +15)│ 1 │ +16)│ │ +17)│ partitioning_scheme: │ +18)│ RoundRobinBatch(4) │ +19)└─────────────┬─────────────┘ +20)┌─────────────┴─────────────┐ +21)│ DataSourceExec │ +22)│ -------------------- │ +23)│ files: 1 │ +24)│ format: arrow │ +25)└───────────────────────────┘ # Query with PartialSortExec. query TT @@ -1175,35 +1144,32 @@ physical_plan 33)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 34)│ ProjectionExec ││ RepartitionExec │ 35)│ -------------------- ││ -------------------- │ -36)│ CAST(table1.string_col AS ││ output_partition_count: │ -37)│ Utf8View): ││ 1 │ -38)│ CAST(string_col@1 AS ││ │ -39)│ Utf8View) ││ partitioning_scheme: │ +36)│ CAST(string_col AS ││ output_partition_count: │ +37)│ Utf8View) ││ 1 │ +38)│ ││ │ +39)│ bigint_col ││ partitioning_scheme: │ 40)│ ││ RoundRobinBatch(4) │ -41)│ bigint_col: ││ │ -42)│ bigint_col@2 ││ │ -43)│ ││ │ -44)│ date_col: date_col@3 ││ │ -45)│ int_col: int_col@0 ││ │ -46)│ ││ │ -47)│ string_col: ││ │ -48)│ string_col@1 ││ │ -49)└─────────────┬─────────────┘└─────────────┬─────────────┘ -50)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -51)│ RepartitionExec ││ DataSourceExec │ -52)│ -------------------- ││ -------------------- │ -53)│ output_partition_count: ││ files: 1 │ -54)│ 1 ││ format: parquet │ -55)│ ││ │ -56)│ partitioning_scheme: ││ │ -57)│ RoundRobinBatch(4) ││ │ -58)└─────────────┬─────────────┘└───────────────────────────┘ -59)┌─────────────┴─────────────┐ -60)│ DataSourceExec │ -61)│ -------------------- │ -62)│ files: 1 │ -63)│ format: csv │ -64)└───────────────────────────┘ +41)│ date_col ││ │ +42)│ ││ │ +43)│ int_col ││ │ +44)│ ││ │ +45)│ string_col ││ │ +46)└─────────────┬─────────────┘└─────────────┬─────────────┘ +47)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +48)│ RepartitionExec ││ DataSourceExec │ +49)│ -------------------- ││ -------------------- │ +50)│ output_partition_count: ││ files: 1 │ +51)│ 1 ││ format: parquet │ +52)│ ││ │ +53)│ partitioning_scheme: ││ │ +54)│ RoundRobinBatch(4) ││ │ +55)└─────────────┬─────────────┘└───────────────────────────┘ +56)┌─────────────┴─────────────┐ +57)│ DataSourceExec │ +58)│ -------------------- │ +59)│ files: 1 │ +60)│ format: csv │ +61)└───────────────────────────┘ # Query with outer hash join. query TT @@ -1247,35 +1213,32 @@ physical_plan 35)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 36)│ ProjectionExec ││ RepartitionExec │ 37)│ -------------------- ││ -------------------- │ -38)│ CAST(table1.string_col AS ││ output_partition_count: │ -39)│ Utf8View): ││ 1 │ -40)│ CAST(string_col@1 AS ││ │ -41)│ Utf8View) ││ partitioning_scheme: │ +38)│ CAST(string_col AS ││ output_partition_count: │ +39)│ Utf8View) ││ 1 │ +40)│ ││ │ +41)│ bigint_col ││ partitioning_scheme: │ 42)│ ││ RoundRobinBatch(4) │ -43)│ bigint_col: ││ │ -44)│ bigint_col@2 ││ │ -45)│ ││ │ -46)│ date_col: date_col@3 ││ │ -47)│ int_col: int_col@0 ││ │ -48)│ ││ │ -49)│ string_col: ││ │ -50)│ string_col@1 ││ │ -51)└─────────────┬─────────────┘└─────────────┬─────────────┘ -52)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -53)│ RepartitionExec ││ DataSourceExec │ -54)│ -------------------- ││ -------------------- │ -55)│ output_partition_count: ││ files: 1 │ -56)│ 1 ││ format: parquet │ -57)│ ││ │ -58)│ partitioning_scheme: ││ │ -59)│ RoundRobinBatch(4) ││ │ -60)└─────────────┬─────────────┘└───────────────────────────┘ -61)┌─────────────┴─────────────┐ -62)│ DataSourceExec │ -63)│ -------------------- │ -64)│ files: 1 │ -65)│ format: csv │ -66)└───────────────────────────┘ +43)│ date_col ││ │ +44)│ ││ │ +45)│ int_col ││ │ +46)│ ││ │ +47)│ string_col ││ │ +48)└─────────────┬─────────────┘└─────────────┬─────────────┘ +49)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +50)│ RepartitionExec ││ DataSourceExec │ +51)│ -------------------- ││ -------------------- │ +52)│ output_partition_count: ││ files: 1 │ +53)│ 1 ││ format: parquet │ +54)│ ││ │ +55)│ partitioning_scheme: ││ │ +56)│ RoundRobinBatch(4) ││ │ +57)└─────────────┬─────────────┘└───────────────────────────┘ +58)┌─────────────┴─────────────┐ +59)│ DataSourceExec │ +60)│ -------------------- │ +61)│ files: 1 │ +62)│ format: csv │ +63)└───────────────────────────┘ # Query with nested loop join. query TT @@ -1676,12 +1639,12 @@ physical_plan 04)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 05)│ ProjectionExec ││ CoalescePartitionsExec │ 06)│ -------------------- ││ │ -07)│ id: 1 ││ │ +07)│ 1 ││ │ 08)└─────────────┬─────────────┘└─────────────┬─────────────┘ 09)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 10)│ PlaceholderRowExec ││ ProjectionExec │ 11)│ ││ -------------------- │ -12)│ ││ id: id@0 + 1 │ +12)│ ││ id + 1 │ 13)└───────────────────────────┘└─────────────┬─────────────┘ 14)-----------------------------┌─────────────┴─────────────┐ 15)-----------------------------│ CoalesceBatchesExec │ @@ -1752,62 +1715,61 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(*): │ -05)│ count(Int64(1))@0 │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ AggregateExec │ -09)│ -------------------- │ -10)│ aggr: count(Int64(1)) │ -11)│ mode: Final │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ CoalescePartitionsExec │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ AggregateExec │ -18)│ -------------------- │ -19)│ aggr: count(Int64(1)) │ -20)│ mode: Partial │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ RepartitionExec │ -24)│ -------------------- │ -25)│ output_partition_count: │ -26)│ 1 │ -27)│ │ -28)│ partitioning_scheme: │ -29)│ RoundRobinBatch(4) │ -30)└─────────────┬─────────────┘ -31)┌─────────────┴─────────────┐ -32)│ ProjectionExec │ -33)└─────────────┬─────────────┘ -34)┌─────────────┴─────────────┐ -35)│ GlobalLimitExec │ -36)│ -------------------- │ -37)│ limit: 3 │ -38)│ skip: 6 │ -39)└─────────────┬─────────────┘ -40)┌─────────────┴─────────────┐ -41)│ CoalesceBatchesExec │ -42)│ -------------------- │ -43)│ limit: 9 │ -44)│ │ -45)│ target_batch_size: │ -46)│ 8192 │ -47)└─────────────┬─────────────┘ -48)┌─────────────┴─────────────┐ -49)│ FilterExec │ -50)│ -------------------- │ -51)│ predicate: a@0 > 3 │ -52)└─────────────┬─────────────┘ -53)┌─────────────┴─────────────┐ -54)│ DataSourceExec │ -55)│ -------------------- │ -56)│ bytes: 160 │ -57)│ format: memory │ -58)│ rows: 1 │ -59)└───────────────────────────┘ +04)│ count(Int64(1)) │ +05)└─────────────┬─────────────┘ +06)┌─────────────┴─────────────┐ +07)│ AggregateExec │ +08)│ -------------------- │ +09)│ aggr: count(Int64(1)) │ +10)│ mode: Final │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ CoalescePartitionsExec │ +14)└─────────────┬─────────────┘ +15)┌─────────────┴─────────────┐ +16)│ AggregateExec │ +17)│ -------------------- │ +18)│ aggr: count(Int64(1)) │ +19)│ mode: Partial │ +20)└─────────────┬─────────────┘ +21)┌─────────────┴─────────────┐ +22)│ RepartitionExec │ +23)│ -------------------- │ +24)│ output_partition_count: │ +25)│ 1 │ +26)│ │ +27)│ partitioning_scheme: │ +28)│ RoundRobinBatch(4) │ +29)└─────────────┬─────────────┘ +30)┌─────────────┴─────────────┐ +31)│ ProjectionExec │ +32)└─────────────┬─────────────┘ +33)┌─────────────┴─────────────┐ +34)│ GlobalLimitExec │ +35)│ -------------------- │ +36)│ limit: 3 │ +37)│ skip: 6 │ +38)└─────────────┬─────────────┘ +39)┌─────────────┴─────────────┐ +40)│ CoalesceBatchesExec │ +41)│ -------------------- │ +42)│ limit: 9 │ +43)│ │ +44)│ target_batch_size: │ +45)│ 8192 │ +46)└─────────────┬─────────────┘ +47)┌─────────────┴─────────────┐ +48)│ FilterExec │ +49)│ -------------------- │ +50)│ predicate: a@0 > 3 │ +51)└─────────────┬─────────────┘ +52)┌─────────────┴─────────────┐ +53)│ DataSourceExec │ +54)│ -------------------- │ +55)│ bytes: 160 │ +56)│ format: memory │ +57)│ rows: 1 │ +58)└───────────────────────────┘ # Test explain tree for LazyMemoryExec query TT From 237fe40ee07e67c7eb252bee017fad6fd49ae6cc Mon Sep 17 00:00:00 2001 From: irenjj Date: Fri, 14 Mar 2025 19:52:59 +0800 Subject: [PATCH 12/16] fix --- datafusion/physical-plan/src/projection.rs | 8 +- .../sqllogictest/test_files/explain_tree.slt | 696 +++++++++--------- 2 files changed, 374 insertions(+), 330 deletions(-) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 16f34be8ad81f..ffabd1b860ada 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -168,10 +168,16 @@ impl DisplayAs for ProjectionExec { write!(f, "ProjectionExec: expr=[{}]", expr.join(", ")) } DisplayFormatType::TreeRender => { - for (e, _) in self.expr().iter() { + for (i, (e, alias)) in self.expr().iter().enumerate() { + if &e.to_string() == alias { + write!(f, "expr{i}=")?; + } else { + write!(f, "{alias}=")?; + } e.fmt_sql(f)?; writeln!(f)?; } + Ok(()) } } diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 5ed74455c87de..d9b80b8c24c1e 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -704,26 +704,29 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(Int64(1)) ROWS │ -05)│ BETWEEN UNBOUNDED │ -06)│ PRECEDING AND UNBOUNDED │ -07)│ FOLLOWING │ -08)└─────────────┬─────────────┘ -09)┌─────────────┴─────────────┐ -10)│ WindowAggExec │ -11)│ -------------------- │ -12)│ select_list: │ -13)│ count(Int64(1)) ROWS │ -14)│ BETWEEN UNBOUNDED │ -15)│ PRECEDING AND UNBOUNDED │ -16)│ FOLLOWING │ -17)└─────────────┬─────────────┘ -18)┌─────────────┴─────────────┐ -19)│ DataSourceExec │ -20)│ -------------------- │ -21)│ files: 1 │ -22)│ format: csv │ -23)└───────────────────────────┘ +04)│ count(*) ROWS BETWEEN │ +05)│ UNBOUNDED PRECEDING │ +06)│ AND UNBOUNDED FOLLOWING: │ +07)│ count(Int64(1)) ROWS │ +08)│ BETWEEN UNBOUNDED │ +09)│ PRECEDING AND UNBOUNDED │ +10)│ FOLLOWING │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ WindowAggExec │ +14)│ -------------------- │ +15)│ select_list: │ +16)│ count(Int64(1)) ROWS │ +17)│ BETWEEN UNBOUNDED │ +18)│ PRECEDING AND UNBOUNDED │ +19)│ FOLLOWING │ +20)└─────────────┬─────────────┘ +21)┌─────────────┴─────────────┐ +22)│ DataSourceExec │ +23)│ -------------------- │ +24)│ files: 1 │ +25)│ format: csv │ +26)└───────────────────────────┘ # Query with bounded window agg. query TT @@ -736,42 +739,43 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ sum(t1.v1) ORDER BY [t1.v1│ -05)│ ASC NULLS LAST] ROWS │ -06)│ BETWEEN 1 PRECEDING │ -07)│ AND CURRENT ROW │ -08)│ │ -09)│ v1 │ -10)└─────────────┬─────────────┘ -11)┌─────────────┴─────────────┐ -12)│ BoundedWindowAggExec │ -13)│ -------------------- │ -14)│ mode: Sorted │ -15)│ │ -16)│ select_list: │ -17)│ sum(t1.v1) ORDER BY [t1.v1│ -18)│ ASC NULLS LAST] ROWS │ -19)│ BETWEEN 1 PRECEDING │ -20)│ AND CURRENT ROW │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ SortExec │ -24)│ -------------------- │ -25)│ v1@0 ASC NULLS LAST │ -26)└─────────────┬─────────────┘ -27)┌─────────────┴─────────────┐ -28)│ ProjectionExec │ -29)│ -------------------- │ -30)│ value │ -31)└─────────────┬─────────────┘ -32)┌─────────────┴─────────────┐ -33)│ LazyMemoryExec │ -34)│ -------------------- │ -35)│ batch_generators: │ -36)│ generate_series: start=1, │ -37)│ end=1000, batch_size │ -38)│ =8192 │ -39)└───────────────────────────┘ +04)│ rolling_sum: │ +05)│ sum(t1.v1) ORDER BY [t1.v1│ +06)│ ASC NULLS LAST] ROWS │ +07)│ BETWEEN 1 PRECEDING │ +08)│ AND CURRENT ROW │ +09)│ │ +10)│ v1: v1 │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ BoundedWindowAggExec │ +14)│ -------------------- │ +15)│ mode: Sorted │ +16)│ │ +17)│ select_list: │ +18)│ sum(t1.v1) ORDER BY [t1.v1│ +19)│ ASC NULLS LAST] ROWS │ +20)│ BETWEEN 1 PRECEDING │ +21)│ AND CURRENT ROW │ +22)└─────────────┬─────────────┘ +23)┌─────────────┴─────────────┐ +24)│ SortExec │ +25)│ -------------------- │ +26)│ v1@0 ASC NULLS LAST │ +27)└─────────────┬─────────────┘ +28)┌─────────────┴─────────────┐ +29)│ ProjectionExec │ +30)│ -------------------- │ +31)│ v1: value │ +32)└─────────────┬─────────────┘ +33)┌─────────────┴─────────────┐ +34)│ LazyMemoryExec │ +35)│ -------------------- │ +36)│ batch_generators: │ +37)│ generate_series: start=1, │ +38)│ end=1000, batch_size │ +39)│ =8192 │ +40)└───────────────────────────┘ query TT explain select @@ -783,33 +787,39 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(Int64(1)) ROWS │ -05)│ BETWEEN UNBOUNDED │ -06)│ PRECEDING AND UNBOUNDED │ -07)│ FOLLOWING │ -08)│ │ -09)│ row_number() ROWS BETWEEN │ -10)│ UNBOUNDED PRECEDING AND │ -11)│ UNBOUNDED FOLLOWING │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ WindowAggExec │ -15)│ -------------------- │ -16)│ select_list: │ -17)│ count(Int64(1)) ROWS │ -18)│ BETWEEN UNBOUNDED │ -19)│ PRECEDING AND UNBOUNDED │ -20)│ FOLLOWING, row_number() │ -21)│ ROWS BETWEEN UNBOUNDED │ -22)│ PRECEDING AND UNBOUNDED │ -23)│ FOLLOWING │ -24)└─────────────┬─────────────┘ -25)┌─────────────┴─────────────┐ -26)│ DataSourceExec │ -27)│ -------------------- │ -28)│ files: 1 │ -29)│ format: csv │ -30)└───────────────────────────┘ +04)│ count(*) ROWS BETWEEN │ +05)│ UNBOUNDED PRECEDING │ +06)│ AND UNBOUNDED FOLLOWING: │ +07)│ count(Int64(1)) ROWS │ +08)│ BETWEEN UNBOUNDED │ +09)│ PRECEDING AND UNBOUNDED │ +10)│ FOLLOWING │ +11)│ │ +12)│ row_number() ROWS BETWEEN │ +13)│ UNBOUNDED PRECEDING AND │ +14)│ UNBOUNDED FOLLOWING: │ +15)│ row_number() ROWS BETWEEN │ +16)│ UNBOUNDED PRECEDING AND │ +17)│ UNBOUNDED FOLLOWING │ +18)└─────────────┬─────────────┘ +19)┌─────────────┴─────────────┐ +20)│ WindowAggExec │ +21)│ -------------------- │ +22)│ select_list: │ +23)│ count(Int64(1)) ROWS │ +24)│ BETWEEN UNBOUNDED │ +25)│ PRECEDING AND UNBOUNDED │ +26)│ FOLLOWING, row_number() │ +27)│ ROWS BETWEEN UNBOUNDED │ +28)│ PRECEDING AND UNBOUNDED │ +29)│ FOLLOWING │ +30)└─────────────┬─────────────┘ +31)┌─────────────┴─────────────┐ +32)│ DataSourceExec │ +33)│ -------------------- │ +34)│ files: 1 │ +35)│ format: csv │ +36)└───────────────────────────┘ # Query for sort. query TT @@ -855,28 +865,30 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ CAST(int_col AS Int64) + │ +04)│ bigint_col: │ 05)│ bigint_col │ 06)│ │ -07)│ bigint_col │ +07)│ int_col: int_col │ 08)│ │ -09)│ int_col │ -10)└─────────────┬─────────────┘ -11)┌─────────────┴─────────────┐ -12)│ RepartitionExec │ -13)│ -------------------- │ -14)│ output_partition_count: │ -15)│ 1 │ -16)│ │ -17)│ partitioning_scheme: │ -18)│ RoundRobinBatch(4) │ -19)└─────────────┬─────────────┘ -20)┌─────────────┴─────────────┐ -21)│ DataSourceExec │ -22)│ -------------------- │ -23)│ files: 1 │ -24)│ format: csv │ -25)└───────────────────────────┘ +09)│ sum_col: │ +10)│ CAST(int_col AS Int64) + │ +11)│ bigint_col │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ output_partition_count: │ +17)│ 1 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ DataSourceExec │ +24)│ -------------------- │ +25)│ files: 1 │ +26)│ format: csv │ +27)└───────────────────────────┘ query TT explain select @@ -892,56 +904,67 @@ physical_plan 05)│ .int_col DESC NULLS │ 06)│ FIRST] RANGE BETWEEN │ 07)│ UNBOUNDED PRECEDING AND │ -08)│ CURRENT ROW │ -09)│ │ -10)│ row_number() ORDER BY │ -11)│ [table1.int_col ASC │ -12)│ NULLS LAST] RANGE │ -13)│ BETWEEN UNBOUNDED │ -14)│ PRECEDING AND CURRENT │ -15)│ ROW │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ BoundedWindowAggExec │ -19)│ -------------------- │ -20)│ mode: Sorted │ -21)│ │ -22)│ select_list: │ -23)│ row_number() ORDER BY │ -24)│ [table1.int_col ASC │ -25)│ NULLS LAST] RANGE │ -26)│ BETWEEN UNBOUNDED │ -27)│ PRECEDING AND CURRENT │ -28)│ ROW │ -29)└─────────────┬─────────────┘ -30)┌─────────────┴─────────────┐ -31)│ SortExec │ -32)│ -------------------- │ -33)│ int_col@0 ASC NULLS LAST │ -34)└─────────────┬─────────────┘ -35)┌─────────────┴─────────────┐ -36)│ BoundedWindowAggExec │ -37)│ -------------------- │ -38)│ mode: Sorted │ -39)│ │ -40)│ select_list: │ -41)│ rank() ORDER BY [table1 │ -42)│ .int_col DESC NULLS │ -43)│ FIRST] RANGE BETWEEN │ -44)│ UNBOUNDED PRECEDING AND │ -45)│ CURRENT ROW │ -46)└─────────────┬─────────────┘ -47)┌─────────────┴─────────────┐ -48)│ SortExec │ -49)│ -------------------- │ -50)│ int_col@0 DESC │ -51)└─────────────┬─────────────┘ -52)┌─────────────┴─────────────┐ -53)│ DataSourceExec │ -54)│ -------------------- │ -55)│ files: 1 │ -56)│ format: csv │ -57)└───────────────────────────┘ +08)│ CURRENT ROW: │ +09)│ rank() ORDER BY [table1 │ +10)│ .int_col DESC NULLS │ +11)│ FIRST] RANGE BETWEEN │ +12)│ UNBOUNDED PRECEDING AND │ +13)│ CURRENT ROW │ +14)│ │ +15)│ row_number() ORDER BY │ +16)│ [table1.int_col ASC │ +17)│ NULLS LAST] RANGE │ +18)│ BETWEEN UNBOUNDED │ +19)│ PRECEDING AND CURRENT │ +20)│ ROW: │ +21)│ row_number() ORDER BY │ +22)│ [table1.int_col ASC │ +23)│ NULLS LAST] RANGE │ +24)│ BETWEEN UNBOUNDED │ +25)│ PRECEDING AND CURRENT │ +26)│ ROW │ +27)└─────────────┬─────────────┘ +28)┌─────────────┴─────────────┐ +29)│ BoundedWindowAggExec │ +30)│ -------------------- │ +31)│ mode: Sorted │ +32)│ │ +33)│ select_list: │ +34)│ row_number() ORDER BY │ +35)│ [table1.int_col ASC │ +36)│ NULLS LAST] RANGE │ +37)│ BETWEEN UNBOUNDED │ +38)│ PRECEDING AND CURRENT │ +39)│ ROW │ +40)└─────────────┬─────────────┘ +41)┌─────────────┴─────────────┐ +42)│ SortExec │ +43)│ -------------------- │ +44)│ int_col@0 ASC NULLS LAST │ +45)└─────────────┬─────────────┘ +46)┌─────────────┴─────────────┐ +47)│ BoundedWindowAggExec │ +48)│ -------------------- │ +49)│ mode: Sorted │ +50)│ │ +51)│ select_list: │ +52)│ rank() ORDER BY [table1 │ +53)│ .int_col DESC NULLS │ +54)│ FIRST] RANGE BETWEEN │ +55)│ UNBOUNDED PRECEDING AND │ +56)│ CURRENT ROW │ +57)└─────────────┬─────────────┘ +58)┌─────────────┴─────────────┐ +59)│ SortExec │ +60)│ -------------------- │ +61)│ int_col@0 DESC │ +62)└─────────────┬─────────────┘ +63)┌─────────────┴─────────────┐ +64)│ DataSourceExec │ +65)│ -------------------- │ +66)│ files: 1 │ +67)│ format: csv │ +68)└───────────────────────────┘ # Query with projection on parquet query TT @@ -951,28 +974,30 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ CAST(int_col AS Int64) + │ +04)│ bigint_col: │ 05)│ bigint_col │ 06)│ │ -07)│ bigint_col │ +07)│ int_col: int_col │ 08)│ │ -09)│ int_col │ -10)└─────────────┬─────────────┘ -11)┌─────────────┴─────────────┐ -12)│ RepartitionExec │ -13)│ -------------------- │ -14)│ output_partition_count: │ -15)│ 1 │ -16)│ │ -17)│ partitioning_scheme: │ -18)│ RoundRobinBatch(4) │ -19)└─────────────┬─────────────┘ -20)┌─────────────┴─────────────┐ -21)│ DataSourceExec │ -22)│ -------------------- │ -23)│ files: 1 │ -24)│ format: parquet │ -25)└───────────────────────────┘ +09)│ sum_col: │ +10)│ CAST(int_col AS Int64) + │ +11)│ bigint_col │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ output_partition_count: │ +17)│ 1 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ DataSourceExec │ +24)│ -------------------- │ +25)│ files: 1 │ +26)│ format: parquet │ +27)└───────────────────────────┘ # Query with projection on memory @@ -983,20 +1008,22 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ CAST(int_col AS Int64) + │ +04)│ bigint_col: │ 05)│ bigint_col │ 06)│ │ -07)│ bigint_col │ +07)│ int_col: int_col │ 08)│ │ -09)│ int_col │ -10)└─────────────┬─────────────┘ -11)┌─────────────┴─────────────┐ -12)│ DataSourceExec │ -13)│ -------------------- │ -14)│ bytes: 1560 │ -15)│ format: memory │ -16)│ rows: 1 │ -17)└───────────────────────────┘ +09)│ sum_col: │ +10)│ CAST(int_col AS Int64) + │ +11)│ bigint_col │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ DataSourceExec │ +15)│ -------------------- │ +16)│ bytes: 1560 │ +17)│ format: memory │ +18)│ rows: 1 │ +19)└───────────────────────────┘ # Query with projection on json query TT @@ -1006,27 +1033,29 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col │ -05)│ │ -06)│ int_col │ -07)│ │ -08)│ int_col + bigint_col │ -09)└─────────────┬─────────────┘ -10)┌─────────────┴─────────────┐ -11)│ RepartitionExec │ -12)│ -------------------- │ -13)│ output_partition_count: │ -14)│ 1 │ -15)│ │ -16)│ partitioning_scheme: │ -17)│ RoundRobinBatch(4) │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ DataSourceExec │ -21)│ -------------------- │ -22)│ files: 1 │ -23)│ format: json │ -24)└───────────────────────────┘ +04)│ bigint_col: │ +05)│ bigint_col │ +06)│ │ +07)│ int_col: int_col │ +08)│ │ +09)│ sum_col: │ +10)│ int_col + bigint_col │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ RepartitionExec │ +14)│ -------------------- │ +15)│ output_partition_count: │ +16)│ 1 │ +17)│ │ +18)│ partitioning_scheme: │ +19)│ RoundRobinBatch(4) │ +20)└─────────────┬─────────────┘ +21)┌─────────────┴─────────────┐ +22)│ DataSourceExec │ +23)│ -------------------- │ +24)│ files: 1 │ +25)│ format: json │ +26)└───────────────────────────┘ # Query with projection on arrow @@ -1037,28 +1066,30 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ CAST(int_col AS Int64) + │ +04)│ bigint_col: │ 05)│ bigint_col │ 06)│ │ -07)│ bigint_col │ +07)│ int_col: int_col │ 08)│ │ -09)│ int_col │ -10)└─────────────┬─────────────┘ -11)┌─────────────┴─────────────┐ -12)│ RepartitionExec │ -13)│ -------------------- │ -14)│ output_partition_count: │ -15)│ 1 │ -16)│ │ -17)│ partitioning_scheme: │ -18)│ RoundRobinBatch(4) │ -19)└─────────────┬─────────────┘ -20)┌─────────────┴─────────────┐ -21)│ DataSourceExec │ -22)│ -------------------- │ -23)│ files: 1 │ -24)│ format: arrow │ -25)└───────────────────────────┘ +09)│ sum_col: │ +10)│ CAST(int_col AS Int64) + │ +11)│ bigint_col │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ output_partition_count: │ +17)│ 1 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ DataSourceExec │ +24)│ -------------------- │ +25)│ files: 1 │ +26)│ format: arrow │ +27)└───────────────────────────┘ # Query with PartialSortExec. query TT @@ -1144,32 +1175,35 @@ physical_plan 33)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 34)│ ProjectionExec ││ RepartitionExec │ 35)│ -------------------- ││ -------------------- │ -36)│ CAST(string_col AS ││ output_partition_count: │ -37)│ Utf8View) ││ 1 │ -38)│ ││ │ -39)│ bigint_col ││ partitioning_scheme: │ +36)│ CAST(table1.string_col AS ││ output_partition_count: │ +37)│ Utf8View): ││ 1 │ +38)│ CAST(string_col AS ││ │ +39)│ Utf8View) ││ partitioning_scheme: │ 40)│ ││ RoundRobinBatch(4) │ -41)│ date_col ││ │ -42)│ ││ │ -43)│ int_col ││ │ -44)│ ││ │ -45)│ string_col ││ │ -46)└─────────────┬─────────────┘└─────────────┬─────────────┘ -47)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -48)│ RepartitionExec ││ DataSourceExec │ -49)│ -------------------- ││ -------------------- │ -50)│ output_partition_count: ││ files: 1 │ -51)│ 1 ││ format: parquet │ -52)│ ││ │ -53)│ partitioning_scheme: ││ │ -54)│ RoundRobinBatch(4) ││ │ -55)└─────────────┬─────────────┘└───────────────────────────┘ -56)┌─────────────┴─────────────┐ -57)│ DataSourceExec │ -58)│ -------------------- │ -59)│ files: 1 │ -60)│ format: csv │ -61)└───────────────────────────┘ +41)│ bigint_col: ││ │ +42)│ bigint_col ││ │ +43)│ ││ │ +44)│ date_col: date_col ││ │ +45)│ int_col: int_col ││ │ +46)│ ││ │ +47)│ string_col: ││ │ +48)│ string_col ││ │ +49)└─────────────┬─────────────┘└─────────────┬─────────────┘ +50)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +51)│ RepartitionExec ││ DataSourceExec │ +52)│ -------------------- ││ -------------------- │ +53)│ output_partition_count: ││ files: 1 │ +54)│ 1 ││ format: parquet │ +55)│ ││ │ +56)│ partitioning_scheme: ││ │ +57)│ RoundRobinBatch(4) ││ │ +58)└─────────────┬─────────────┘└───────────────────────────┘ +59)┌─────────────┴─────────────┐ +60)│ DataSourceExec │ +61)│ -------------------- │ +62)│ files: 1 │ +63)│ format: csv │ +64)└───────────────────────────┘ # Query with outer hash join. query TT @@ -1213,32 +1247,35 @@ physical_plan 35)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 36)│ ProjectionExec ││ RepartitionExec │ 37)│ -------------------- ││ -------------------- │ -38)│ CAST(string_col AS ││ output_partition_count: │ -39)│ Utf8View) ││ 1 │ -40)│ ││ │ -41)│ bigint_col ││ partitioning_scheme: │ +38)│ CAST(table1.string_col AS ││ output_partition_count: │ +39)│ Utf8View): ││ 1 │ +40)│ CAST(string_col AS ││ │ +41)│ Utf8View) ││ partitioning_scheme: │ 42)│ ││ RoundRobinBatch(4) │ -43)│ date_col ││ │ -44)│ ││ │ -45)│ int_col ││ │ -46)│ ││ │ -47)│ string_col ││ │ -48)└─────────────┬─────────────┘└─────────────┬─────────────┘ -49)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -50)│ RepartitionExec ││ DataSourceExec │ -51)│ -------------------- ││ -------------------- │ -52)│ output_partition_count: ││ files: 1 │ -53)│ 1 ││ format: parquet │ -54)│ ││ │ -55)│ partitioning_scheme: ││ │ -56)│ RoundRobinBatch(4) ││ │ -57)└─────────────┬─────────────┘└───────────────────────────┘ -58)┌─────────────┴─────────────┐ -59)│ DataSourceExec │ -60)│ -------------------- │ -61)│ files: 1 │ -62)│ format: csv │ -63)└───────────────────────────┘ +43)│ bigint_col: ││ │ +44)│ bigint_col ││ │ +45)│ ││ │ +46)│ date_col: date_col ││ │ +47)│ int_col: int_col ││ │ +48)│ ││ │ +49)│ string_col: ││ │ +50)│ string_col ││ │ +51)└─────────────┬─────────────┘└─────────────┬─────────────┘ +52)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +53)│ RepartitionExec ││ DataSourceExec │ +54)│ -------------------- ││ -------------------- │ +55)│ output_partition_count: ││ files: 1 │ +56)│ 1 ││ format: parquet │ +57)│ ││ │ +58)│ partitioning_scheme: ││ │ +59)│ RoundRobinBatch(4) ││ │ +60)└─────────────┬─────────────┘└───────────────────────────┘ +61)┌─────────────┴─────────────┐ +62)│ DataSourceExec │ +63)│ -------------------- │ +64)│ files: 1 │ +65)│ format: csv │ +66)└───────────────────────────┘ # Query with nested loop join. query TT @@ -1639,12 +1676,12 @@ physical_plan 04)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 05)│ ProjectionExec ││ CoalescePartitionsExec │ 06)│ -------------------- ││ │ -07)│ 1 ││ │ +07)│ id: 1 ││ │ 08)└─────────────┬─────────────┘└─────────────┬─────────────┘ 09)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 10)│ PlaceholderRowExec ││ ProjectionExec │ 11)│ ││ -------------------- │ -12)│ ││ id + 1 │ +12)│ ││ id: id + 1 │ 13)└───────────────────────────┘└─────────────┬─────────────┘ 14)-----------------------------┌─────────────┴─────────────┐ 15)-----------------------------│ CoalesceBatchesExec │ @@ -1715,61 +1752,62 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(Int64(1)) │ -05)└─────────────┬─────────────┘ -06)┌─────────────┴─────────────┐ -07)│ AggregateExec │ -08)│ -------------------- │ -09)│ aggr: count(Int64(1)) │ -10)│ mode: Final │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ CoalescePartitionsExec │ -14)└─────────────┬─────────────┘ -15)┌─────────────┴─────────────┐ -16)│ AggregateExec │ -17)│ -------------------- │ -18)│ aggr: count(Int64(1)) │ -19)│ mode: Partial │ -20)└─────────────┬─────────────┘ -21)┌─────────────┴─────────────┐ -22)│ RepartitionExec │ -23)│ -------------------- │ -24)│ output_partition_count: │ -25)│ 1 │ -26)│ │ -27)│ partitioning_scheme: │ -28)│ RoundRobinBatch(4) │ -29)└─────────────┬─────────────┘ -30)┌─────────────┴─────────────┐ -31)│ ProjectionExec │ -32)└─────────────┬─────────────┘ -33)┌─────────────┴─────────────┐ -34)│ GlobalLimitExec │ -35)│ -------------------- │ -36)│ limit: 3 │ -37)│ skip: 6 │ -38)└─────────────┬─────────────┘ -39)┌─────────────┴─────────────┐ -40)│ CoalesceBatchesExec │ -41)│ -------------------- │ -42)│ limit: 9 │ -43)│ │ -44)│ target_batch_size: │ -45)│ 8192 │ -46)└─────────────┬─────────────┘ -47)┌─────────────┴─────────────┐ -48)│ FilterExec │ -49)│ -------------------- │ -50)│ predicate: a@0 > 3 │ -51)└─────────────┬─────────────┘ -52)┌─────────────┴─────────────┐ -53)│ DataSourceExec │ -54)│ -------------------- │ -55)│ bytes: 160 │ -56)│ format: memory │ -57)│ rows: 1 │ -58)└───────────────────────────┘ +04)│ count(*): │ +05)│ count(Int64(1)) │ +06)└─────────────┬─────────────┘ +07)┌─────────────┴─────────────┐ +08)│ AggregateExec │ +09)│ -------------------- │ +10)│ aggr: count(Int64(1)) │ +11)│ mode: Final │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ CoalescePartitionsExec │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ AggregateExec │ +18)│ -------------------- │ +19)│ aggr: count(Int64(1)) │ +20)│ mode: Partial │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ RepartitionExec │ +24)│ -------------------- │ +25)│ output_partition_count: │ +26)│ 1 │ +27)│ │ +28)│ partitioning_scheme: │ +29)│ RoundRobinBatch(4) │ +30)└─────────────┬─────────────┘ +31)┌─────────────┴─────────────┐ +32)│ ProjectionExec │ +33)└─────────────┬─────────────┘ +34)┌─────────────┴─────────────┐ +35)│ GlobalLimitExec │ +36)│ -------------------- │ +37)│ limit: 3 │ +38)│ skip: 6 │ +39)└─────────────┬─────────────┘ +40)┌─────────────┴─────────────┐ +41)│ CoalesceBatchesExec │ +42)│ -------------------- │ +43)│ limit: 9 │ +44)│ │ +45)│ target_batch_size: │ +46)│ 8192 │ +47)└─────────────┬─────────────┘ +48)┌─────────────┴─────────────┐ +49)│ FilterExec │ +50)│ -------------------- │ +51)│ predicate: a@0 > 3 │ +52)└─────────────┬─────────────┘ +53)┌─────────────┴─────────────┐ +54)│ DataSourceExec │ +55)│ -------------------- │ +56)│ bytes: 160 │ +57)│ format: memory │ +58)│ rows: 1 │ +59)└───────────────────────────┘ # Test explain tree for LazyMemoryExec query TT From b271f983874513a9a33aa568f95578849a68470a Mon Sep 17 00:00:00 2001 From: irenjj Date: Fri, 14 Mar 2025 20:20:05 +0800 Subject: [PATCH 13/16] simplify col --- datafusion/physical-plan/src/projection.rs | 14 +- .../sqllogictest/test_files/explain_tree.slt | 675 ++++++++---------- 2 files changed, 314 insertions(+), 375 deletions(-) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index ffabd1b860ada..20eade6267f18 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -169,13 +169,17 @@ impl DisplayAs for ProjectionExec { } DisplayFormatType::TreeRender => { for (i, (e, alias)) in self.expr().iter().enumerate() { - if &e.to_string() == alias { - write!(f, "expr{i}=")?; + if e.as_any().downcast_ref::().is_some() { + e.fmt_sql(f)?; } else { - write!(f, "{alias}=")?; + if &e.to_string() == alias { + write!(f, "expr{i}=")?; + } else { + write!(f, "{alias}=")?; + } + e.fmt_sql(f)?; + writeln!(f)?; } - e.fmt_sql(f)?; - writeln!(f)?; } Ok(()) diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index d9b80b8c24c1e..282b18730b67f 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -704,29 +704,26 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(*) ROWS BETWEEN │ -05)│ UNBOUNDED PRECEDING │ -06)│ AND UNBOUNDED FOLLOWING: │ -07)│ count(Int64(1)) ROWS │ -08)│ BETWEEN UNBOUNDED │ -09)│ PRECEDING AND UNBOUNDED │ -10)│ FOLLOWING │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ WindowAggExec │ -14)│ -------------------- │ -15)│ select_list: │ -16)│ count(Int64(1)) ROWS │ -17)│ BETWEEN UNBOUNDED │ -18)│ PRECEDING AND UNBOUNDED │ -19)│ FOLLOWING │ -20)└─────────────┬─────────────┘ -21)┌─────────────┴─────────────┐ -22)│ DataSourceExec │ -23)│ -------------------- │ -24)│ files: 1 │ -25)│ format: csv │ -26)└───────────────────────────┘ +04)│ count(Int64(1)) ROWS │ +05)│ BETWEEN UNBOUNDED │ +06)│ PRECEDING AND UNBOUNDED │ +07)│ FOLLOWING │ +08)└─────────────┬─────────────┘ +09)┌─────────────┴─────────────┐ +10)│ WindowAggExec │ +11)│ -------------------- │ +12)│ select_list: │ +13)│ count(Int64(1)) ROWS │ +14)│ BETWEEN UNBOUNDED │ +15)│ PRECEDING AND UNBOUNDED │ +16)│ FOLLOWING │ +17)└─────────────┬─────────────┘ +18)┌─────────────┴─────────────┐ +19)│ DataSourceExec │ +20)│ -------------------- │ +21)│ files: 1 │ +22)│ format: csv │ +23)└───────────────────────────┘ # Query with bounded window agg. query TT @@ -739,43 +736,40 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ rolling_sum: │ -05)│ sum(t1.v1) ORDER BY [t1.v1│ -06)│ ASC NULLS LAST] ROWS │ -07)│ BETWEEN 1 PRECEDING │ -08)│ AND CURRENT ROW │ -09)│ │ -10)│ v1: v1 │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ BoundedWindowAggExec │ -14)│ -------------------- │ -15)│ mode: Sorted │ -16)│ │ -17)│ select_list: │ -18)│ sum(t1.v1) ORDER BY [t1.v1│ -19)│ ASC NULLS LAST] ROWS │ -20)│ BETWEEN 1 PRECEDING │ -21)│ AND CURRENT ROW │ -22)└─────────────┬─────────────┘ -23)┌─────────────┴─────────────┐ -24)│ SortExec │ -25)│ -------------------- │ -26)│ v1@0 ASC NULLS LAST │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ ProjectionExec │ -30)│ -------------------- │ -31)│ v1: value │ -32)└─────────────┬─────────────┘ -33)┌─────────────┴─────────────┐ -34)│ LazyMemoryExec │ -35)│ -------------------- │ -36)│ batch_generators: │ -37)│ generate_series: start=1, │ -38)│ end=1000, batch_size │ -39)│ =8192 │ -40)└───────────────────────────┘ +04)│ v1sum(t1.v1) ORDER BY [t1 │ +05)│ .v1 ASC NULLS LAST] ROWS │ +06)│ BETWEEN 1 PRECEDING AND │ +07)│ CURRENT ROW │ +08)└─────────────┬─────────────┘ +09)┌─────────────┴─────────────┐ +10)│ BoundedWindowAggExec │ +11)│ -------------------- │ +12)│ mode: Sorted │ +13)│ │ +14)│ select_list: │ +15)│ sum(t1.v1) ORDER BY [t1.v1│ +16)│ ASC NULLS LAST] ROWS │ +17)│ BETWEEN 1 PRECEDING │ +18)│ AND CURRENT ROW │ +19)└─────────────┬─────────────┘ +20)┌─────────────┴─────────────┐ +21)│ SortExec │ +22)│ -------------------- │ +23)│ v1@0 ASC NULLS LAST │ +24)└─────────────┬─────────────┘ +25)┌─────────────┴─────────────┐ +26)│ ProjectionExec │ +27)│ -------------------- │ +28)│ value │ +29)└─────────────┬─────────────┘ +30)┌─────────────┴─────────────┐ +31)│ LazyMemoryExec │ +32)│ -------------------- │ +33)│ batch_generators: │ +34)│ generate_series: start=1, │ +35)│ end=1000, batch_size │ +36)│ =8192 │ +37)└───────────────────────────┘ query TT explain select @@ -787,39 +781,33 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(*) ROWS BETWEEN │ -05)│ UNBOUNDED PRECEDING │ -06)│ AND UNBOUNDED FOLLOWING: │ -07)│ count(Int64(1)) ROWS │ -08)│ BETWEEN UNBOUNDED │ -09)│ PRECEDING AND UNBOUNDED │ -10)│ FOLLOWING │ -11)│ │ -12)│ row_number() ROWS BETWEEN │ -13)│ UNBOUNDED PRECEDING AND │ -14)│ UNBOUNDED FOLLOWING: │ -15)│ row_number() ROWS BETWEEN │ -16)│ UNBOUNDED PRECEDING AND │ -17)│ UNBOUNDED FOLLOWING │ -18)└─────────────┬─────────────┘ -19)┌─────────────┴─────────────┐ -20)│ WindowAggExec │ -21)│ -------------------- │ -22)│ select_list: │ -23)│ count(Int64(1)) ROWS │ -24)│ BETWEEN UNBOUNDED │ -25)│ PRECEDING AND UNBOUNDED │ -26)│ FOLLOWING, row_number() │ -27)│ ROWS BETWEEN UNBOUNDED │ -28)│ PRECEDING AND UNBOUNDED │ -29)│ FOLLOWING │ -30)└─────────────┬─────────────┘ -31)┌─────────────┴─────────────┐ -32)│ DataSourceExec │ -33)│ -------------------- │ -34)│ files: 1 │ -35)│ format: csv │ -36)└───────────────────────────┘ +04)│ count(Int64(1)) ROWS │ +05)│ BETWEEN UNBOUNDED │ +06)│ PRECEDING AND UNBOUNDED │ +07)│ FOLLOWINGrow_number() │ +08)│ ROWS BETWEEN │ +09)│ UNBOUNDED │ +10)│ PRECEDING AND │ +11)│ UNBOUNDED FOLLOWING │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ WindowAggExec │ +15)│ -------------------- │ +16)│ select_list: │ +17)│ count(Int64(1)) ROWS │ +18)│ BETWEEN UNBOUNDED │ +19)│ PRECEDING AND UNBOUNDED │ +20)│ FOLLOWING, row_number() │ +21)│ ROWS BETWEEN UNBOUNDED │ +22)│ PRECEDING AND UNBOUNDED │ +23)│ FOLLOWING │ +24)└─────────────┬─────────────┘ +25)┌─────────────┴─────────────┐ +26)│ DataSourceExec │ +27)│ -------------------- │ +28)│ files: 1 │ +29)│ format: csv │ +30)└───────────────────────────┘ # Query for sort. query TT @@ -865,30 +853,25 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col: │ -05)│ bigint_col │ -06)│ │ -07)│ int_col: int_col │ -08)│ │ -09)│ sum_col: │ -10)│ CAST(int_col AS Int64) + │ -11)│ bigint_col │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ output_partition_count: │ -17)│ 1 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: csv │ -27)└───────────────────────────┘ +04)│ int_colbigint_colsum_col: │ +05)│ CAST(int_col AS Int64) + │ +06)│ bigint_col │ +07)└─────────────┬─────────────┘ +08)┌─────────────┴─────────────┐ +09)│ RepartitionExec │ +10)│ -------------------- │ +11)│ output_partition_count: │ +12)│ 1 │ +13)│ │ +14)│ partitioning_scheme: │ +15)│ RoundRobinBatch(4) │ +16)└─────────────┬─────────────┘ +17)┌─────────────┴─────────────┐ +18)│ DataSourceExec │ +19)│ -------------------- │ +20)│ files: 1 │ +21)│ format: csv │ +22)└───────────────────────────┘ query TT explain select @@ -904,19 +887,19 @@ physical_plan 05)│ .int_col DESC NULLS │ 06)│ FIRST] RANGE BETWEEN │ 07)│ UNBOUNDED PRECEDING AND │ -08)│ CURRENT ROW: │ -09)│ rank() ORDER BY [table1 │ -10)│ .int_col DESC NULLS │ -11)│ FIRST] RANGE BETWEEN │ -12)│ UNBOUNDED PRECEDING AND │ -13)│ CURRENT ROW │ -14)│ │ -15)│ row_number() ORDER BY │ -16)│ [table1.int_col ASC │ -17)│ NULLS LAST] RANGE │ -18)│ BETWEEN UNBOUNDED │ -19)│ PRECEDING AND CURRENT │ -20)│ ROW: │ +08)│ CURRENT ROWrow_number() │ +09)│ ORDER BY [table1 │ +10)│ .int_col ASC NULLS │ +11)│ LAST] RANGE BETWEEN │ +12)│ UNBOUNDED PRECEDING │ +13)│ AND CURRENT ROW │ +14)└─────────────┬─────────────┘ +15)┌─────────────┴─────────────┐ +16)│ BoundedWindowAggExec │ +17)│ -------------------- │ +18)│ mode: Sorted │ +19)│ │ +20)│ select_list: │ 21)│ row_number() ORDER BY │ 22)│ [table1.int_col ASC │ 23)│ NULLS LAST] RANGE │ @@ -925,46 +908,33 @@ physical_plan 26)│ ROW │ 27)└─────────────┬─────────────┘ 28)┌─────────────┴─────────────┐ -29)│ BoundedWindowAggExec │ +29)│ SortExec │ 30)│ -------------------- │ -31)│ mode: Sorted │ -32)│ │ -33)│ select_list: │ -34)│ row_number() ORDER BY │ -35)│ [table1.int_col ASC │ -36)│ NULLS LAST] RANGE │ -37)│ BETWEEN UNBOUNDED │ -38)│ PRECEDING AND CURRENT │ -39)│ ROW │ -40)└─────────────┬─────────────┘ -41)┌─────────────┴─────────────┐ -42)│ SortExec │ -43)│ -------------------- │ -44)│ int_col@0 ASC NULLS LAST │ -45)└─────────────┬─────────────┘ -46)┌─────────────┴─────────────┐ -47)│ BoundedWindowAggExec │ -48)│ -------------------- │ -49)│ mode: Sorted │ -50)│ │ -51)│ select_list: │ -52)│ rank() ORDER BY [table1 │ -53)│ .int_col DESC NULLS │ -54)│ FIRST] RANGE BETWEEN │ -55)│ UNBOUNDED PRECEDING AND │ -56)│ CURRENT ROW │ -57)└─────────────┬─────────────┘ -58)┌─────────────┴─────────────┐ -59)│ SortExec │ -60)│ -------------------- │ -61)│ int_col@0 DESC │ -62)└─────────────┬─────────────┘ -63)┌─────────────┴─────────────┐ -64)│ DataSourceExec │ -65)│ -------------------- │ -66)│ files: 1 │ -67)│ format: csv │ -68)└───────────────────────────┘ +31)│ int_col@0 ASC NULLS LAST │ +32)└─────────────┬─────────────┘ +33)┌─────────────┴─────────────┐ +34)│ BoundedWindowAggExec │ +35)│ -------------------- │ +36)│ mode: Sorted │ +37)│ │ +38)│ select_list: │ +39)│ rank() ORDER BY [table1 │ +40)│ .int_col DESC NULLS │ +41)│ FIRST] RANGE BETWEEN │ +42)│ UNBOUNDED PRECEDING AND │ +43)│ CURRENT ROW │ +44)└─────────────┬─────────────┘ +45)┌─────────────┴─────────────┐ +46)│ SortExec │ +47)│ -------------------- │ +48)│ int_col@0 DESC │ +49)└─────────────┬─────────────┘ +50)┌─────────────┴─────────────┐ +51)│ DataSourceExec │ +52)│ -------------------- │ +53)│ files: 1 │ +54)│ format: csv │ +55)└───────────────────────────┘ # Query with projection on parquet query TT @@ -974,30 +944,25 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col: │ -05)│ bigint_col │ -06)│ │ -07)│ int_col: int_col │ -08)│ │ -09)│ sum_col: │ -10)│ CAST(int_col AS Int64) + │ -11)│ bigint_col │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ output_partition_count: │ -17)│ 1 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: parquet │ -27)└───────────────────────────┘ +04)│ int_colbigint_colsum_col: │ +05)│ CAST(int_col AS Int64) + │ +06)│ bigint_col │ +07)└─────────────┬─────────────┘ +08)┌─────────────┴─────────────┐ +09)│ RepartitionExec │ +10)│ -------------------- │ +11)│ output_partition_count: │ +12)│ 1 │ +13)│ │ +14)│ partitioning_scheme: │ +15)│ RoundRobinBatch(4) │ +16)└─────────────┬─────────────┘ +17)┌─────────────┴─────────────┐ +18)│ DataSourceExec │ +19)│ -------------------- │ +20)│ files: 1 │ +21)│ format: parquet │ +22)└───────────────────────────┘ # Query with projection on memory @@ -1008,22 +973,17 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col: │ -05)│ bigint_col │ -06)│ │ -07)│ int_col: int_col │ -08)│ │ -09)│ sum_col: │ -10)│ CAST(int_col AS Int64) + │ -11)│ bigint_col │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ DataSourceExec │ -15)│ -------------------- │ -16)│ bytes: 1560 │ -17)│ format: memory │ -18)│ rows: 1 │ -19)└───────────────────────────┘ +04)│ int_colbigint_colsum_col: │ +05)│ CAST(int_col AS Int64) + │ +06)│ bigint_col │ +07)└─────────────┬─────────────┘ +08)┌─────────────┴─────────────┐ +09)│ DataSourceExec │ +10)│ -------------------- │ +11)│ bytes: 1560 │ +12)│ format: memory │ +13)│ rows: 1 │ +14)└───────────────────────────┘ # Query with projection on json query TT @@ -1033,29 +993,24 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col: │ -05)│ bigint_col │ -06)│ │ -07)│ int_col: int_col │ -08)│ │ -09)│ sum_col: │ -10)│ int_col + bigint_col │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ RepartitionExec │ -14)│ -------------------- │ -15)│ output_partition_count: │ -16)│ 1 │ -17)│ │ -18)│ partitioning_scheme: │ -19)│ RoundRobinBatch(4) │ -20)└─────────────┬─────────────┘ -21)┌─────────────┴─────────────┐ -22)│ DataSourceExec │ -23)│ -------------------- │ -24)│ files: 1 │ -25)│ format: json │ -26)└───────────────────────────┘ +04)│ int_colbigint_colsum_col: │ +05)│ int_col + bigint_col │ +06)└─────────────┬─────────────┘ +07)┌─────────────┴─────────────┐ +08)│ RepartitionExec │ +09)│ -------------------- │ +10)│ output_partition_count: │ +11)│ 1 │ +12)│ │ +13)│ partitioning_scheme: │ +14)│ RoundRobinBatch(4) │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ DataSourceExec │ +18)│ -------------------- │ +19)│ files: 1 │ +20)│ format: json │ +21)└───────────────────────────┘ # Query with projection on arrow @@ -1066,30 +1021,25 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col: │ -05)│ bigint_col │ -06)│ │ -07)│ int_col: int_col │ -08)│ │ -09)│ sum_col: │ -10)│ CAST(int_col AS Int64) + │ -11)│ bigint_col │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ RepartitionExec │ -15)│ -------------------- │ -16)│ output_partition_count: │ -17)│ 1 │ -18)│ │ -19)│ partitioning_scheme: │ -20)│ RoundRobinBatch(4) │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ DataSourceExec │ -24)│ -------------------- │ -25)│ files: 1 │ -26)│ format: arrow │ -27)└───────────────────────────┘ +04)│ int_colbigint_colsum_col: │ +05)│ CAST(int_col AS Int64) + │ +06)│ bigint_col │ +07)└─────────────┬─────────────┘ +08)┌─────────────┴─────────────┐ +09)│ RepartitionExec │ +10)│ -------------------- │ +11)│ output_partition_count: │ +12)│ 1 │ +13)│ │ +14)│ partitioning_scheme: │ +15)│ RoundRobinBatch(4) │ +16)└─────────────┬─────────────┘ +17)┌─────────────┴─────────────┐ +18)│ DataSourceExec │ +19)│ -------------------- │ +20)│ files: 1 │ +21)│ format: arrow │ +22)└───────────────────────────┘ # Query with PartialSortExec. query TT @@ -1175,35 +1125,28 @@ physical_plan 33)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 34)│ ProjectionExec ││ RepartitionExec │ 35)│ -------------------- ││ -------------------- │ -36)│ CAST(table1.string_col AS ││ output_partition_count: │ -37)│ Utf8View): ││ 1 │ -38)│ CAST(string_col AS ││ │ -39)│ Utf8View) ││ partitioning_scheme: │ -40)│ ││ RoundRobinBatch(4) │ -41)│ bigint_col: ││ │ -42)│ bigint_col ││ │ -43)│ ││ │ -44)│ date_col: date_col ││ │ -45)│ int_col: int_col ││ │ -46)│ ││ │ -47)│ string_col: ││ │ -48)│ string_col ││ │ -49)└─────────────┬─────────────┘└─────────────┬─────────────┘ -50)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -51)│ RepartitionExec ││ DataSourceExec │ -52)│ -------------------- ││ -------------------- │ -53)│ output_partition_count: ││ files: 1 │ -54)│ 1 ││ format: parquet │ -55)│ ││ │ -56)│ partitioning_scheme: ││ │ -57)│ RoundRobinBatch(4) ││ │ -58)└─────────────┬─────────────┘└───────────────────────────┘ -59)┌─────────────┴─────────────┐ -60)│ DataSourceExec │ -61)│ -------------------- │ -62)│ files: 1 │ -63)│ format: csv │ -64)└───────────────────────────┘ +36)│int_colstring_colbigint_col││ output_partition_count: │ +37)│ date_colCAST(table1 ││ 1 │ +38)│ .string_col AS ││ │ +39)│ Utf8View): ││ partitioning_scheme: │ +40)│ CAST(string_col AS ││ RoundRobinBatch(4) │ +41)│ Utf8View) ││ │ +42)└─────────────┬─────────────┘└─────────────┬─────────────┘ +43)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +44)│ RepartitionExec ││ DataSourceExec │ +45)│ -------------------- ││ -------------------- │ +46)│ output_partition_count: ││ files: 1 │ +47)│ 1 ││ format: parquet │ +48)│ ││ │ +49)│ partitioning_scheme: ││ │ +50)│ RoundRobinBatch(4) ││ │ +51)└─────────────┬─────────────┘└───────────────────────────┘ +52)┌─────────────┴─────────────┐ +53)│ DataSourceExec │ +54)│ -------------------- │ +55)│ files: 1 │ +56)│ format: csv │ +57)└───────────────────────────┘ # Query with outer hash join. query TT @@ -1247,35 +1190,28 @@ physical_plan 35)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 36)│ ProjectionExec ││ RepartitionExec │ 37)│ -------------------- ││ -------------------- │ -38)│ CAST(table1.string_col AS ││ output_partition_count: │ -39)│ Utf8View): ││ 1 │ -40)│ CAST(string_col AS ││ │ -41)│ Utf8View) ││ partitioning_scheme: │ -42)│ ││ RoundRobinBatch(4) │ -43)│ bigint_col: ││ │ -44)│ bigint_col ││ │ -45)│ ││ │ -46)│ date_col: date_col ││ │ -47)│ int_col: int_col ││ │ -48)│ ││ │ -49)│ string_col: ││ │ -50)│ string_col ││ │ -51)└─────────────┬─────────────┘└─────────────┬─────────────┘ -52)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -53)│ RepartitionExec ││ DataSourceExec │ -54)│ -------------------- ││ -------------------- │ -55)│ output_partition_count: ││ files: 1 │ -56)│ 1 ││ format: parquet │ -57)│ ││ │ -58)│ partitioning_scheme: ││ │ -59)│ RoundRobinBatch(4) ││ │ -60)└─────────────┬─────────────┘└───────────────────────────┘ -61)┌─────────────┴─────────────┐ -62)│ DataSourceExec │ -63)│ -------------------- │ -64)│ files: 1 │ -65)│ format: csv │ -66)└───────────────────────────┘ +38)│int_colstring_colbigint_col││ output_partition_count: │ +39)│ date_colCAST(table1 ││ 1 │ +40)│ .string_col AS ││ │ +41)│ Utf8View): ││ partitioning_scheme: │ +42)│ CAST(string_col AS ││ RoundRobinBatch(4) │ +43)│ Utf8View) ││ │ +44)└─────────────┬─────────────┘└─────────────┬─────────────┘ +45)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +46)│ RepartitionExec ││ DataSourceExec │ +47)│ -------------------- ││ -------------------- │ +48)│ output_partition_count: ││ files: 1 │ +49)│ 1 ││ format: parquet │ +50)│ ││ │ +51)│ partitioning_scheme: ││ │ +52)│ RoundRobinBatch(4) ││ │ +53)└─────────────┬─────────────┘└───────────────────────────┘ +54)┌─────────────┴─────────────┐ +55)│ DataSourceExec │ +56)│ -------------------- │ +57)│ files: 1 │ +58)│ format: csv │ +59)└───────────────────────────┘ # Query with nested loop join. query TT @@ -1752,62 +1688,61 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(*): │ -05)│ count(Int64(1)) │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ AggregateExec │ -09)│ -------------------- │ -10)│ aggr: count(Int64(1)) │ -11)│ mode: Final │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ CoalescePartitionsExec │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ AggregateExec │ -18)│ -------------------- │ -19)│ aggr: count(Int64(1)) │ -20)│ mode: Partial │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ RepartitionExec │ -24)│ -------------------- │ -25)│ output_partition_count: │ -26)│ 1 │ -27)│ │ -28)│ partitioning_scheme: │ -29)│ RoundRobinBatch(4) │ -30)└─────────────┬─────────────┘ -31)┌─────────────┴─────────────┐ -32)│ ProjectionExec │ -33)└─────────────┬─────────────┘ -34)┌─────────────┴─────────────┐ -35)│ GlobalLimitExec │ -36)│ -------------------- │ -37)│ limit: 3 │ -38)│ skip: 6 │ -39)└─────────────┬─────────────┘ -40)┌─────────────┴─────────────┐ -41)│ CoalesceBatchesExec │ -42)│ -------------------- │ -43)│ limit: 9 │ -44)│ │ -45)│ target_batch_size: │ -46)│ 8192 │ -47)└─────────────┬─────────────┘ -48)┌─────────────┴─────────────┐ -49)│ FilterExec │ -50)│ -------------------- │ -51)│ predicate: a@0 > 3 │ -52)└─────────────┬─────────────┘ -53)┌─────────────┴─────────────┐ -54)│ DataSourceExec │ -55)│ -------------------- │ -56)│ bytes: 160 │ -57)│ format: memory │ -58)│ rows: 1 │ -59)└───────────────────────────┘ +04)│ count(Int64(1)) │ +05)└─────────────┬─────────────┘ +06)┌─────────────┴─────────────┐ +07)│ AggregateExec │ +08)│ -------------------- │ +09)│ aggr: count(Int64(1)) │ +10)│ mode: Final │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ CoalescePartitionsExec │ +14)└─────────────┬─────────────┘ +15)┌─────────────┴─────────────┐ +16)│ AggregateExec │ +17)│ -------------------- │ +18)│ aggr: count(Int64(1)) │ +19)│ mode: Partial │ +20)└─────────────┬─────────────┘ +21)┌─────────────┴─────────────┐ +22)│ RepartitionExec │ +23)│ -------------------- │ +24)│ output_partition_count: │ +25)│ 1 │ +26)│ │ +27)│ partitioning_scheme: │ +28)│ RoundRobinBatch(4) │ +29)└─────────────┬─────────────┘ +30)┌─────────────┴─────────────┐ +31)│ ProjectionExec │ +32)└─────────────┬─────────────┘ +33)┌─────────────┴─────────────┐ +34)│ GlobalLimitExec │ +35)│ -------------------- │ +36)│ limit: 3 │ +37)│ skip: 6 │ +38)└─────────────┬─────────────┘ +39)┌─────────────┴─────────────┐ +40)│ CoalesceBatchesExec │ +41)│ -------------------- │ +42)│ limit: 9 │ +43)│ │ +44)│ target_batch_size: │ +45)│ 8192 │ +46)└─────────────┬─────────────┘ +47)┌─────────────┴─────────────┐ +48)│ FilterExec │ +49)│ -------------------- │ +50)│ predicate: a@0 > 3 │ +51)└─────────────┬─────────────┘ +52)┌─────────────┴─────────────┐ +53)│ DataSourceExec │ +54)│ -------------------- │ +55)│ bytes: 160 │ +56)│ format: memory │ +57)│ rows: 1 │ +58)└───────────────────────────┘ # Test explain tree for LazyMemoryExec query TT From 12a6351ef51a1f28fe7ffce8725aeac44429527f Mon Sep 17 00:00:00 2001 From: irenjj Date: Fri, 14 Mar 2025 20:23:43 +0800 Subject: [PATCH 14/16] fix --- datafusion/physical-plan/src/projection.rs | 1 + .../sqllogictest/test_files/explain_tree.slt | 468 ++++++++++-------- 2 files changed, 253 insertions(+), 216 deletions(-) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 20eade6267f18..b36188426a979 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -171,6 +171,7 @@ impl DisplayAs for ProjectionExec { for (i, (e, alias)) in self.expr().iter().enumerate() { if e.as_any().downcast_ref::().is_some() { e.fmt_sql(f)?; + writeln!(f)?; } else { if &e.to_string() == alias { write!(f, "expr{i}=")?; diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 282b18730b67f..67a1d7307dda3 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -736,40 +736,42 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ v1sum(t1.v1) ORDER BY [t1 │ -05)│ .v1 ASC NULLS LAST] ROWS │ -06)│ BETWEEN 1 PRECEDING AND │ -07)│ CURRENT ROW │ -08)└─────────────┬─────────────┘ -09)┌─────────────┴─────────────┐ -10)│ BoundedWindowAggExec │ -11)│ -------------------- │ -12)│ mode: Sorted │ -13)│ │ -14)│ select_list: │ -15)│ sum(t1.v1) ORDER BY [t1.v1│ -16)│ ASC NULLS LAST] ROWS │ -17)│ BETWEEN 1 PRECEDING │ -18)│ AND CURRENT ROW │ -19)└─────────────┬─────────────┘ -20)┌─────────────┴─────────────┐ -21)│ SortExec │ -22)│ -------------------- │ -23)│ v1@0 ASC NULLS LAST │ -24)└─────────────┬─────────────┘ -25)┌─────────────┴─────────────┐ -26)│ ProjectionExec │ -27)│ -------------------- │ -28)│ value │ -29)└─────────────┬─────────────┘ -30)┌─────────────┴─────────────┐ -31)│ LazyMemoryExec │ -32)│ -------------------- │ -33)│ batch_generators: │ -34)│ generate_series: start=1, │ -35)│ end=1000, batch_size │ -36)│ =8192 │ -37)└───────────────────────────┘ +04)│ sum(t1.v1) ORDER BY [t1.v1│ +05)│ ASC NULLS LAST] ROWS │ +06)│ BETWEEN 1 PRECEDING │ +07)│ AND CURRENT ROW │ +08)│ │ +09)│ v1 │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ BoundedWindowAggExec │ +13)│ -------------------- │ +14)│ mode: Sorted │ +15)│ │ +16)│ select_list: │ +17)│ sum(t1.v1) ORDER BY [t1.v1│ +18)│ ASC NULLS LAST] ROWS │ +19)│ BETWEEN 1 PRECEDING │ +20)│ AND CURRENT ROW │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ SortExec │ +24)│ -------------------- │ +25)│ v1@0 ASC NULLS LAST │ +26)└─────────────┬─────────────┘ +27)┌─────────────┴─────────────┐ +28)│ ProjectionExec │ +29)│ -------------------- │ +30)│ value │ +31)└─────────────┬─────────────┘ +32)┌─────────────┴─────────────┐ +33)│ LazyMemoryExec │ +34)│ -------------------- │ +35)│ batch_generators: │ +36)│ generate_series: start=1, │ +37)│ end=1000, batch_size │ +38)│ =8192 │ +39)└───────────────────────────┘ query TT explain select @@ -784,10 +786,10 @@ physical_plan 04)│ count(Int64(1)) ROWS │ 05)│ BETWEEN UNBOUNDED │ 06)│ PRECEDING AND UNBOUNDED │ -07)│ FOLLOWINGrow_number() │ -08)│ ROWS BETWEEN │ -09)│ UNBOUNDED │ -10)│ PRECEDING AND │ +07)│ FOLLOWING │ +08)│ │ +09)│ row_number() ROWS BETWEEN │ +10)│ UNBOUNDED PRECEDING AND │ 11)│ UNBOUNDED FOLLOWING │ 12)└─────────────┬─────────────┘ 13)┌─────────────┴─────────────┐ @@ -853,25 +855,29 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ int_colbigint_colsum_col: │ -05)│ CAST(int_col AS Int64) + │ -06)│ bigint_col │ -07)└─────────────┬─────────────┘ -08)┌─────────────┴─────────────┐ -09)│ RepartitionExec │ -10)│ -------------------- │ -11)│ output_partition_count: │ -12)│ 1 │ -13)│ │ -14)│ partitioning_scheme: │ -15)│ RoundRobinBatch(4) │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ DataSourceExec │ -19)│ -------------------- │ -20)│ files: 1 │ -21)│ format: csv │ -22)└───────────────────────────┘ +04)│ bigint_col │ +05)│ │ +06)│ int_col │ +07)│ │ +08)│ sum_col: │ +09)│ CAST(int_col AS Int64) + │ +10)│ bigint_col │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ RepartitionExec │ +14)│ -------------------- │ +15)│ output_partition_count: │ +16)│ 1 │ +17)│ │ +18)│ partitioning_scheme: │ +19)│ RoundRobinBatch(4) │ +20)└─────────────┬─────────────┘ +21)┌─────────────┴─────────────┐ +22)│ DataSourceExec │ +23)│ -------------------- │ +24)│ files: 1 │ +25)│ format: csv │ +26)└───────────────────────────┘ query TT explain select @@ -887,54 +893,56 @@ physical_plan 05)│ .int_col DESC NULLS │ 06)│ FIRST] RANGE BETWEEN │ 07)│ UNBOUNDED PRECEDING AND │ -08)│ CURRENT ROWrow_number() │ -09)│ ORDER BY [table1 │ -10)│ .int_col ASC NULLS │ -11)│ LAST] RANGE BETWEEN │ -12)│ UNBOUNDED PRECEDING │ -13)│ AND CURRENT ROW │ -14)└─────────────┬─────────────┘ -15)┌─────────────┴─────────────┐ -16)│ BoundedWindowAggExec │ -17)│ -------------------- │ -18)│ mode: Sorted │ -19)│ │ -20)│ select_list: │ -21)│ row_number() ORDER BY │ -22)│ [table1.int_col ASC │ -23)│ NULLS LAST] RANGE │ -24)│ BETWEEN UNBOUNDED │ -25)│ PRECEDING AND CURRENT │ -26)│ ROW │ -27)└─────────────┬─────────────┘ -28)┌─────────────┴─────────────┐ -29)│ SortExec │ -30)│ -------------------- │ -31)│ int_col@0 ASC NULLS LAST │ -32)└─────────────┬─────────────┘ -33)┌─────────────┴─────────────┐ -34)│ BoundedWindowAggExec │ -35)│ -------------------- │ -36)│ mode: Sorted │ -37)│ │ -38)│ select_list: │ -39)│ rank() ORDER BY [table1 │ -40)│ .int_col DESC NULLS │ -41)│ FIRST] RANGE BETWEEN │ -42)│ UNBOUNDED PRECEDING AND │ -43)│ CURRENT ROW │ -44)└─────────────┬─────────────┘ -45)┌─────────────┴─────────────┐ -46)│ SortExec │ -47)│ -------------------- │ -48)│ int_col@0 DESC │ -49)└─────────────┬─────────────┘ -50)┌─────────────┴─────────────┐ -51)│ DataSourceExec │ -52)│ -------------------- │ -53)│ files: 1 │ -54)│ format: csv │ -55)└───────────────────────────┘ +08)│ CURRENT ROW │ +09)│ │ +10)│ row_number() ORDER BY │ +11)│ [table1.int_col ASC │ +12)│ NULLS LAST] RANGE │ +13)│ BETWEEN UNBOUNDED │ +14)│ PRECEDING AND CURRENT │ +15)│ ROW │ +16)└─────────────┬─────────────┘ +17)┌─────────────┴─────────────┐ +18)│ BoundedWindowAggExec │ +19)│ -------------------- │ +20)│ mode: Sorted │ +21)│ │ +22)│ select_list: │ +23)│ row_number() ORDER BY │ +24)│ [table1.int_col ASC │ +25)│ NULLS LAST] RANGE │ +26)│ BETWEEN UNBOUNDED │ +27)│ PRECEDING AND CURRENT │ +28)│ ROW │ +29)└─────────────┬─────────────┘ +30)┌─────────────┴─────────────┐ +31)│ SortExec │ +32)│ -------------------- │ +33)│ int_col@0 ASC NULLS LAST │ +34)└─────────────┬─────────────┘ +35)┌─────────────┴─────────────┐ +36)│ BoundedWindowAggExec │ +37)│ -------------------- │ +38)│ mode: Sorted │ +39)│ │ +40)│ select_list: │ +41)│ rank() ORDER BY [table1 │ +42)│ .int_col DESC NULLS │ +43)│ FIRST] RANGE BETWEEN │ +44)│ UNBOUNDED PRECEDING AND │ +45)│ CURRENT ROW │ +46)└─────────────┬─────────────┘ +47)┌─────────────┴─────────────┐ +48)│ SortExec │ +49)│ -------------------- │ +50)│ int_col@0 DESC │ +51)└─────────────┬─────────────┘ +52)┌─────────────┴─────────────┐ +53)│ DataSourceExec │ +54)│ -------------------- │ +55)│ files: 1 │ +56)│ format: csv │ +57)└───────────────────────────┘ # Query with projection on parquet query TT @@ -944,25 +952,29 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ int_colbigint_colsum_col: │ -05)│ CAST(int_col AS Int64) + │ -06)│ bigint_col │ -07)└─────────────┬─────────────┘ -08)┌─────────────┴─────────────┐ -09)│ RepartitionExec │ -10)│ -------------------- │ -11)│ output_partition_count: │ -12)│ 1 │ -13)│ │ -14)│ partitioning_scheme: │ -15)│ RoundRobinBatch(4) │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ DataSourceExec │ -19)│ -------------------- │ -20)│ files: 1 │ -21)│ format: parquet │ -22)└───────────────────────────┘ +04)│ bigint_col │ +05)│ │ +06)│ int_col │ +07)│ │ +08)│ sum_col: │ +09)│ CAST(int_col AS Int64) + │ +10)│ bigint_col │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ RepartitionExec │ +14)│ -------------------- │ +15)│ output_partition_count: │ +16)│ 1 │ +17)│ │ +18)│ partitioning_scheme: │ +19)│ RoundRobinBatch(4) │ +20)└─────────────┬─────────────┘ +21)┌─────────────┴─────────────┐ +22)│ DataSourceExec │ +23)│ -------------------- │ +24)│ files: 1 │ +25)│ format: parquet │ +26)└───────────────────────────┘ # Query with projection on memory @@ -973,17 +985,21 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ int_colbigint_colsum_col: │ -05)│ CAST(int_col AS Int64) + │ -06)│ bigint_col │ -07)└─────────────┬─────────────┘ -08)┌─────────────┴─────────────┐ -09)│ DataSourceExec │ -10)│ -------------------- │ -11)│ bytes: 1560 │ -12)│ format: memory │ -13)│ rows: 1 │ -14)└───────────────────────────┘ +04)│ bigint_col │ +05)│ │ +06)│ int_col │ +07)│ │ +08)│ sum_col: │ +09)│ CAST(int_col AS Int64) + │ +10)│ bigint_col │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ DataSourceExec │ +14)│ -------------------- │ +15)│ bytes: 1560 │ +16)│ format: memory │ +17)│ rows: 1 │ +18)└───────────────────────────┘ # Query with projection on json query TT @@ -993,24 +1009,28 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ int_colbigint_colsum_col: │ -05)│ int_col + bigint_col │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ RepartitionExec │ -09)│ -------------------- │ -10)│ output_partition_count: │ -11)│ 1 │ -12)│ │ -13)│ partitioning_scheme: │ -14)│ RoundRobinBatch(4) │ -15)└─────────────┬─────────────┘ -16)┌─────────────┴─────────────┐ -17)│ DataSourceExec │ -18)│ -------------------- │ -19)│ files: 1 │ -20)│ format: json │ -21)└───────────────────────────┘ +04)│ bigint_col │ +05)│ │ +06)│ int_col │ +07)│ │ +08)│ sum_col: │ +09)│ int_col + bigint_col │ +10)└─────────────┬─────────────┘ +11)┌─────────────┴─────────────┐ +12)│ RepartitionExec │ +13)│ -------------------- │ +14)│ output_partition_count: │ +15)│ 1 │ +16)│ │ +17)│ partitioning_scheme: │ +18)│ RoundRobinBatch(4) │ +19)└─────────────┬─────────────┘ +20)┌─────────────┴─────────────┐ +21)│ DataSourceExec │ +22)│ -------------------- │ +23)│ files: 1 │ +24)│ format: json │ +25)└───────────────────────────┘ # Query with projection on arrow @@ -1021,25 +1041,29 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ int_colbigint_colsum_col: │ -05)│ CAST(int_col AS Int64) + │ -06)│ bigint_col │ -07)└─────────────┬─────────────┘ -08)┌─────────────┴─────────────┐ -09)│ RepartitionExec │ -10)│ -------------------- │ -11)│ output_partition_count: │ -12)│ 1 │ -13)│ │ -14)│ partitioning_scheme: │ -15)│ RoundRobinBatch(4) │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ DataSourceExec │ -19)│ -------------------- │ -20)│ files: 1 │ -21)│ format: arrow │ -22)└───────────────────────────┘ +04)│ bigint_col │ +05)│ │ +06)│ int_col │ +07)│ │ +08)│ sum_col: │ +09)│ CAST(int_col AS Int64) + │ +10)│ bigint_col │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ RepartitionExec │ +14)│ -------------------- │ +15)│ output_partition_count: │ +16)│ 1 │ +17)│ │ +18)│ partitioning_scheme: │ +19)│ RoundRobinBatch(4) │ +20)└─────────────┬─────────────┘ +21)┌─────────────┴─────────────┐ +22)│ DataSourceExec │ +23)│ -------------------- │ +24)│ files: 1 │ +25)│ format: arrow │ +26)└───────────────────────────┘ # Query with PartialSortExec. query TT @@ -1125,28 +1149,34 @@ physical_plan 33)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 34)│ ProjectionExec ││ RepartitionExec │ 35)│ -------------------- ││ -------------------- │ -36)│int_colstring_colbigint_col││ output_partition_count: │ -37)│ date_colCAST(table1 ││ 1 │ -38)│ .string_col AS ││ │ -39)│ Utf8View): ││ partitioning_scheme: │ -40)│ CAST(string_col AS ││ RoundRobinBatch(4) │ -41)│ Utf8View) ││ │ -42)└─────────────┬─────────────┘└─────────────┬─────────────┘ -43)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -44)│ RepartitionExec ││ DataSourceExec │ -45)│ -------------------- ││ -------------------- │ -46)│ output_partition_count: ││ files: 1 │ -47)│ 1 ││ format: parquet │ -48)│ ││ │ -49)│ partitioning_scheme: ││ │ -50)│ RoundRobinBatch(4) ││ │ -51)└─────────────┬─────────────┘└───────────────────────────┘ -52)┌─────────────┴─────────────┐ -53)│ DataSourceExec │ -54)│ -------------------- │ -55)│ files: 1 │ -56)│ format: csv │ -57)└───────────────────────────┘ +36)│ CAST(table1.string_col AS ││ output_partition_count: │ +37)│ Utf8View): ││ 1 │ +38)│ CAST(string_col AS ││ │ +39)│ Utf8View) ││ partitioning_scheme: │ +40)│ ││ RoundRobinBatch(4) │ +41)│ bigint_col ││ │ +42)│ ││ │ +43)│ date_col ││ │ +44)│ ││ │ +45)│ int_col ││ │ +46)│ ││ │ +47)│ string_col ││ │ +48)└─────────────┬─────────────┘└─────────────┬─────────────┘ +49)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +50)│ RepartitionExec ││ DataSourceExec │ +51)│ -------------------- ││ -------------------- │ +52)│ output_partition_count: ││ files: 1 │ +53)│ 1 ││ format: parquet │ +54)│ ││ │ +55)│ partitioning_scheme: ││ │ +56)│ RoundRobinBatch(4) ││ │ +57)└─────────────┬─────────────┘└───────────────────────────┘ +58)┌─────────────┴─────────────┐ +59)│ DataSourceExec │ +60)│ -------------------- │ +61)│ files: 1 │ +62)│ format: csv │ +63)└───────────────────────────┘ # Query with outer hash join. query TT @@ -1190,28 +1220,34 @@ physical_plan 35)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ 36)│ ProjectionExec ││ RepartitionExec │ 37)│ -------------------- ││ -------------------- │ -38)│int_colstring_colbigint_col││ output_partition_count: │ -39)│ date_colCAST(table1 ││ 1 │ -40)│ .string_col AS ││ │ -41)│ Utf8View): ││ partitioning_scheme: │ -42)│ CAST(string_col AS ││ RoundRobinBatch(4) │ -43)│ Utf8View) ││ │ -44)└─────────────┬─────────────┘└─────────────┬─────────────┘ -45)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -46)│ RepartitionExec ││ DataSourceExec │ -47)│ -------------------- ││ -------------------- │ -48)│ output_partition_count: ││ files: 1 │ -49)│ 1 ││ format: parquet │ -50)│ ││ │ -51)│ partitioning_scheme: ││ │ -52)│ RoundRobinBatch(4) ││ │ -53)└─────────────┬─────────────┘└───────────────────────────┘ -54)┌─────────────┴─────────────┐ -55)│ DataSourceExec │ -56)│ -------------------- │ -57)│ files: 1 │ -58)│ format: csv │ -59)└───────────────────────────┘ +38)│ CAST(table1.string_col AS ││ output_partition_count: │ +39)│ Utf8View): ││ 1 │ +40)│ CAST(string_col AS ││ │ +41)│ Utf8View) ││ partitioning_scheme: │ +42)│ ││ RoundRobinBatch(4) │ +43)│ bigint_col ││ │ +44)│ ││ │ +45)│ date_col ││ │ +46)│ ││ │ +47)│ int_col ││ │ +48)│ ││ │ +49)│ string_col ││ │ +50)└─────────────┬─────────────┘└─────────────┬─────────────┘ +51)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +52)│ RepartitionExec ││ DataSourceExec │ +53)│ -------------------- ││ -------------------- │ +54)│ output_partition_count: ││ files: 1 │ +55)│ 1 ││ format: parquet │ +56)│ ││ │ +57)│ partitioning_scheme: ││ │ +58)│ RoundRobinBatch(4) ││ │ +59)└─────────────┬─────────────┘└───────────────────────────┘ +60)┌─────────────┴─────────────┐ +61)│ DataSourceExec │ +62)│ -------------------- │ +63)│ files: 1 │ +64)│ format: csv │ +65)└───────────────────────────┘ # Query with nested loop join. query TT From 7b9a4c929fc00e3fecf0f5f44834f4b6c233add4 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 14 Mar 2025 15:28:07 -0400 Subject: [PATCH 15/16] Update plan --- .../sqllogictest/test_files/explain_tree.slt | 137 +++++++++--------- 1 file changed, 68 insertions(+), 69 deletions(-) diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 9d5e2509e37d0..d52546af72eb9 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -1400,75 +1400,74 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(*): │ -05)│ count(Int64(1))@1 │ -06)└─────────────┬─────────────┘ -07)┌─────────────┴─────────────┐ -08)│ AggregateExec │ -09)│ -------------------- │ -10)│ aggr: count(Int64(1)) │ -11)│ │ -12)│ group_by: │ -13)│ name@0 as name │ -14)│ │ -15)│ mode: │ -16)│ SinglePartitioned │ -17)└─────────────┬─────────────┘ -18)┌─────────────┴─────────────┐ -19)│ InterleaveExec ├──────────────┐ -20)└─────────────┬─────────────┘ │ -21)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -22)│ AggregateExec ││ AggregateExec │ -23)│ -------------------- ││ -------------------- │ -24)│ aggr ││ aggr │ -25)│ ││ │ -26)│ group_by: ││ group_by: │ -27)│ name@0 as name ││ name@0 as name │ -28)│ ││ │ -29)│ mode: ││ mode: │ -30)│ FinalPartitioned ││ FinalPartitioned │ -31)└─────────────┬─────────────┘└─────────────┬─────────────┘ -32)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -33)│ CoalesceBatchesExec ││ CoalesceBatchesExec │ -34)│ -------------------- ││ -------------------- │ -35)│ target_batch_size: ││ target_batch_size: │ -36)│ 8192 ││ 8192 │ -37)└─────────────┬─────────────┘└─────────────┬─────────────┘ -38)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -39)│ RepartitionExec ││ RepartitionExec │ -40)│ -------------------- ││ -------------------- │ -41)│ output_partition_count: ││ output_partition_count: │ -42)│ 4 ││ 4 │ -43)│ ││ │ -44)│ partitioning_scheme: ││ partitioning_scheme: │ -45)│ Hash([name@0], 4) ││ Hash([name@0], 4) │ -46)└─────────────┬─────────────┘└─────────────┬─────────────┘ -47)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -48)│ RepartitionExec ││ RepartitionExec │ -49)│ -------------------- ││ -------------------- │ -50)│ output_partition_count: ││ output_partition_count: │ -51)│ 1 ││ 1 │ -52)│ ││ │ -53)│ partitioning_scheme: ││ partitioning_scheme: │ -54)│ RoundRobinBatch(4) ││ RoundRobinBatch(4) │ -55)└─────────────┬─────────────┘└─────────────┬─────────────┘ -56)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -57)│ AggregateExec ││ AggregateExec │ -58)│ -------------------- ││ -------------------- │ -59)│ aggr ││ aggr │ -60)│ ││ │ -61)│ group_by: ││ group_by: │ -62)│ name@0 as name ││ name@0 as name │ -63)│ ││ │ -64)│ mode: Partial ││ mode: Partial │ -65)└─────────────┬─────────────┘└─────────────┬─────────────┘ -66)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -67)│ DataSourceExec ││ DataSourceExec │ -68)│ -------------------- ││ -------------------- │ -69)│ bytes: 1320 ││ bytes: 1312 │ -70)│ format: memory ││ format: memory │ -71)│ rows: 1 ││ rows: 1 │ -72)└───────────────────────────┘└───────────────────────────┘ +04)│ count(Int64(1)) │ +05)└─────────────┬─────────────┘ +06)┌─────────────┴─────────────┐ +07)│ AggregateExec │ +08)│ -------------------- │ +09)│ aggr: count(Int64(1)) │ +10)│ │ +11)│ group_by: │ +12)│ name@0 as name │ +13)│ │ +14)│ mode: │ +15)│ SinglePartitioned │ +16)└─────────────┬─────────────┘ +17)┌─────────────┴─────────────┐ +18)│ InterleaveExec ├──────────────┐ +19)└─────────────┬─────────────┘ │ +20)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +21)│ AggregateExec ││ AggregateExec │ +22)│ -------------------- ││ -------------------- │ +23)│ aggr ││ aggr │ +24)│ ││ │ +25)│ group_by: ││ group_by: │ +26)│ name@0 as name ││ name@0 as name │ +27)│ ││ │ +28)│ mode: ││ mode: │ +29)│ FinalPartitioned ││ FinalPartitioned │ +30)└─────────────┬─────────────┘└─────────────┬─────────────┘ +31)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +32)│ CoalesceBatchesExec ││ CoalesceBatchesExec │ +33)│ -------------------- ││ -------------------- │ +34)│ target_batch_size: ││ target_batch_size: │ +35)│ 8192 ││ 8192 │ +36)└─────────────┬─────────────┘└─────────────┬─────────────┘ +37)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +38)│ RepartitionExec ││ RepartitionExec │ +39)│ -------------------- ││ -------------------- │ +40)│ output_partition_count: ││ output_partition_count: │ +41)│ 4 ││ 4 │ +42)│ ││ │ +43)│ partitioning_scheme: ││ partitioning_scheme: │ +44)│ Hash([name@0], 4) ││ Hash([name@0], 4) │ +45)└─────────────┬─────────────┘└─────────────┬─────────────┘ +46)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +47)│ RepartitionExec ││ RepartitionExec │ +48)│ -------------------- ││ -------------------- │ +49)│ output_partition_count: ││ output_partition_count: │ +50)│ 1 ││ 1 │ +51)│ ││ │ +52)│ partitioning_scheme: ││ partitioning_scheme: │ +53)│ RoundRobinBatch(4) ││ RoundRobinBatch(4) │ +54)└─────────────┬─────────────┘└─────────────┬─────────────┘ +55)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +56)│ AggregateExec ││ AggregateExec │ +57)│ -------------------- ││ -------------------- │ +58)│ aggr ││ aggr │ +59)│ ││ │ +60)│ group_by: ││ group_by: │ +61)│ name@0 as name ││ name@0 as name │ +62)│ ││ │ +63)│ mode: Partial ││ mode: Partial │ +64)└─────────────┬─────────────┘└─────────────┬─────────────┘ +65)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +66)│ DataSourceExec ││ DataSourceExec │ +67)│ -------------------- ││ -------------------- │ +68)│ bytes: 1320 ││ bytes: 1312 │ +69)│ format: memory ││ format: memory │ +70)│ rows: 1 ││ rows: 1 │ +71)└───────────────────────────┘└───────────────────────────┘ # cleanup statement ok From 3bee9b960c7b202c91e62b611d3cdfc26ccfed25 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 14 Mar 2025 15:52:48 -0400 Subject: [PATCH 16/16] Rename sql_formatter, add doc and examples --- .../physical-expr-common/src/physical_expr.rs | 59 +- .../physical-expr/src/expressions/binary.rs | 14 +- .../physical-expr/src/expressions/case.rs | 4 +- .../physical-expr/src/expressions/cast.rs | 7 +- .../physical-expr/src/expressions/in_list.rs | 10 +- .../src/expressions/is_not_null.rs | 4 +- .../physical-expr/src/expressions/is_null.rs | 4 +- .../physical-expr/src/expressions/like.rs | 4 +- .../physical-expr/src/expressions/literal.rs | 5 +- .../physical-expr/src/expressions/negative.rs | 4 +- .../physical-expr/src/expressions/not.rs | 8 +- .../physical-expr/src/expressions/try_cast.rs | 6 +- datafusion/physical-expr/src/utils/mod.rs | 2 - .../physical-expr/src/utils/sql_formatter.rs | 35 - datafusion/physical-plan/src/projection.rs | 15 +- .../sqllogictest/test_files/explain_tree.slt | 832 +++++++++--------- 16 files changed, 525 insertions(+), 488 deletions(-) delete mode 100644 datafusion/physical-expr/src/utils/sql_formatter.rs diff --git a/datafusion/physical-expr-common/src/physical_expr.rs b/datafusion/physical-expr-common/src/physical_expr.rs index 22a35dbcab27f..43f214607f9fc 100644 --- a/datafusion/physical-expr-common/src/physical_expr.rs +++ b/datafusion/physical-expr-common/src/physical_expr.rs @@ -16,6 +16,7 @@ // under the License. use std::any::Any; +use std::fmt; use std::fmt::{Debug, Display, Formatter}; use std::hash::{Hash, Hasher}; use std::sync::Arc; @@ -57,7 +58,7 @@ pub type PhysicalExprRef = Arc; /// There are three ways to format `PhysicalExpr` as a string: /// * [`Debug`]: Standard Rust debugging format (e.g. `Constant { value: ... }`) /// * [`Display`]: Detailed SQL-like format that shows expression structure (e.g. (`Utf8 ("foobar")`). This is often used for debugging and tests -/// * [`Self::fmt_sql`]: SQL-like human readable format (e.g. ('foobar')`) +/// * [`Self::fmt_sql`]: SQL-like human readable format (e.g. ('foobar')`), See also [`sql_fmt`] /// /// [`SessionContext::create_physical_expr`]: https://docs.rs/datafusion/latest/datafusion/execution/context/struct.SessionContext.html#method.create_physical_expr /// [`PhysicalPlanner`]: https://docs.rs/datafusion/latest/datafusion/physical_planner/trait.PhysicalPlanner.html @@ -279,7 +280,9 @@ pub trait PhysicalExpr: Send + Sync + Display + Debug + DynEq + DynHash { /// expense of details. Use `Display` or `Debug` for more detailed /// representation. /// - fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result; + /// See the [`fmt_sql`] function for an example of printing `PhysicalExpr`s as SQL. + /// + fn fmt_sql(&self, f: &mut Formatter<'_>) -> fmt::Result; } /// [`PhysicalExpr`] can't be constrained by [`Eq`] directly because it must remain object @@ -377,7 +380,7 @@ where I: Iterator + Clone, I::Item: Display, { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { let mut iter = self.0.clone(); write!(f, "[")?; if let Some(expr) = iter.next() { @@ -393,3 +396,53 @@ where DisplayWrapper(exprs.into_iter()) } + +/// Prints a [`PhysicalExpr`] in a SQL-like format +/// +/// # Example +/// ``` +/// # // The boiler plate needed to create a `PhysicalExpr` for the example +/// # use std::any::Any; +/// # use std::fmt::Formatter; +/// # use std::sync::Arc; +/// # use arrow::array::RecordBatch; +/// # use arrow::datatypes::{DataType, Schema}; +/// # use datafusion_common::Result; +/// # use datafusion_expr_common::columnar_value::ColumnarValue; +/// # use datafusion_physical_expr_common::physical_expr::{fmt_sql, DynEq, PhysicalExpr}; +/// # #[derive(Debug, Hash, PartialOrd, PartialEq)] +/// # struct MyExpr {}; +/// # impl PhysicalExpr for MyExpr {fn as_any(&self) -> &dyn Any { unimplemented!() } +/// # fn data_type(&self, input_schema: &Schema) -> Result { unimplemented!() } +/// # fn nullable(&self, input_schema: &Schema) -> Result { unimplemented!() } +/// # fn evaluate(&self, batch: &RecordBatch) -> Result { unimplemented!() } +/// # fn children(&self) -> Vec<&Arc>{ unimplemented!() } +/// # fn with_new_children(self: Arc, children: Vec>) -> Result> { unimplemented!() } +/// # fn fmt_sql(&self, f: &mut Formatter<'_>) -> std::fmt::Result { write!(f, "CASE a > b THEN 1 ELSE 0 END") } +/// # } +/// # impl std::fmt::Display for MyExpr {fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { unimplemented!() } } +/// # impl DynEq for MyExpr {fn dyn_eq(&self, other: &dyn Any) -> bool { unimplemented!() } } +/// # fn make_physical_expr() -> Arc { Arc::new(MyExpr{}) } +/// let expr: Arc = make_physical_expr(); +/// // wrap the expression in `sql_fmt` which can be used with +/// // `format!`, `to_string()`, etc +/// let expr_as_sql = fmt_sql(expr.as_ref()); +/// assert_eq!( +/// "The SQL: CASE a > b THEN 1 ELSE 0 END", +/// format!("The SQL: {expr_as_sql}") +/// ); +/// ``` +pub fn fmt_sql(expr: &dyn PhysicalExpr) -> impl Display + '_ { + struct Wrapper<'a> { + expr: &'a dyn PhysicalExpr, + } + + impl Display for Wrapper<'_> { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + self.expr.fmt_sql(f)?; + Ok(()) + } + } + + Wrapper { expr } +} diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index a22a19f6b1996..872773b06fa6f 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -793,12 +793,10 @@ pub fn similar_to( #[cfg(test)] mod tests { use super::*; - use crate::{ - expressions::{col, lit, try_cast, Column, Literal}, - utils::sql_formatter, - }; + use crate::expressions::{col, lit, try_cast, Column, Literal}; use datafusion_common::plan_datafusion_err; + use datafusion_physical_expr_common::physical_expr::fmt_sql; /// Performs a binary operation, applying any type coercion necessary fn binary_op( @@ -4718,7 +4716,7 @@ mod tests { )?; let display_string = simple_expr.to_string(); assert_eq!(display_string, "a@0 + b@1"); - let sql_string = sql_formatter(&simple_expr).to_string(); + let sql_string = fmt_sql(&simple_expr).to_string(); assert_eq!(sql_string, "a + b"); // Test nested expressions with different operator precedence @@ -4735,7 +4733,7 @@ mod tests { )?; let display_string = nested_expr.to_string(); assert_eq!(display_string, "(a@0 + b@1) * b@1"); - let sql_string = sql_formatter(&nested_expr).to_string(); + let sql_string = fmt_sql(&nested_expr).to_string(); assert_eq!(sql_string, "(a + b) * b"); // Test nested expressions with same operator precedence @@ -4752,7 +4750,7 @@ mod tests { )?; let display_string = nested_same_prec.to_string(); assert_eq!(display_string, "a@0 + b@1 + b@1"); - let sql_string = sql_formatter(&nested_same_prec).to_string(); + let sql_string = fmt_sql(&nested_same_prec).to_string(); assert_eq!(sql_string, "a + b + b"); // Test with literals @@ -4764,7 +4762,7 @@ mod tests { )?; let display_string = lit_expr.to_string(); assert_eq!(display_string, "a@0 = 42"); - let sql_string = sql_formatter(&lit_expr).to_string(); + let sql_string = fmt_sql(&lit_expr).to_string(); assert_eq!(sql_string, "a = 42"); Ok(()) diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index e2f3bd053e923..67fab3912c6a4 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -598,7 +598,6 @@ mod tests { use super::*; use crate::expressions::{binary, cast, col, lit, BinaryExpr}; - use crate::utils::sql_formatter; use arrow::buffer::Buffer; use arrow::datatypes::DataType::Float64; use arrow::datatypes::*; @@ -607,6 +606,7 @@ mod tests { use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_expr::type_coercion::binary::comparison_coercion; use datafusion_expr::Operator; + use datafusion_physical_expr_common::physical_expr::fmt_sql; #[test] fn case_with_expr() -> Result<()> { @@ -1425,7 +1425,7 @@ mod tests { "CASE WHEN a@0 = foo THEN 123.3 ELSE TRY_CAST(999 AS Float64) END" ); - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); assert_eq!( sql_string, "CASE WHEN a = foo THEN 123.3 ELSE TRY_CAST(999 AS Float64) END" diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index 72222321c75f7..a6766687a881a 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -240,7 +240,7 @@ pub fn cast( mod tests { use super::*; - use crate::{expressions::column::col, utils::sql_formatter}; + use crate::expressions::column::col; use arrow::{ array::{ @@ -251,6 +251,7 @@ mod tests { datatypes::*, }; use datafusion_common::assert_contains; + use datafusion_physical_expr_common::physical_expr::fmt_sql; // runs an end-to-end test of physical type cast // 1. construct a record batch with a column "a" of type A @@ -783,7 +784,7 @@ mod tests { let expr = cast(col("a", &schema)?, &schema, Int64)?; let display_string = expr.to_string(); assert_eq!(display_string, "CAST(a@0 AS Int64)"); - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); assert_eq!(sql_string, "CAST(a AS Int64)"); // Test string casting @@ -791,7 +792,7 @@ mod tests { let expr = cast(col("b", &schema)?, &schema, Int32)?; let display_string = expr.to_string(); assert_eq!(display_string, "CAST(b@0 AS Int32)"); - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); assert_eq!(sql_string, "CAST(b AS Int32)"); Ok(()) diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 80ac94f79b495..469f7bbee3173 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -467,9 +467,9 @@ mod tests { use super::*; use crate::expressions; use crate::expressions::{col, lit, try_cast}; - use crate::utils::sql_formatter; use datafusion_common::plan_err; use datafusion_expr::type_coercion::binary::comparison_coercion; + use datafusion_physical_expr_common::physical_expr::fmt_sql; type InListCastResult = (Arc, Vec>); @@ -1448,7 +1448,7 @@ mod tests { // Test: a IN ('a', 'b') let list = vec![lit("a"), lit("b")]; let expr = in_list(Arc::clone(&col_a), list, &false, &schema)?; - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); let display_string = expr.to_string(); assert_eq!(sql_string, "a IN (a, b)"); assert_eq!(display_string, "Use a@0 IN (SET) ([Literal { value: Utf8(\"a\") }, Literal { value: Utf8(\"b\") }])"); @@ -1456,7 +1456,7 @@ mod tests { // Test: a NOT IN ('a', 'b') let list = vec![lit("a"), lit("b")]; let expr = in_list(Arc::clone(&col_a), list, &true, &schema)?; - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); let display_string = expr.to_string(); assert_eq!(sql_string, "a NOT IN (a, b)"); assert_eq!(display_string, "a@0 NOT IN (SET) ([Literal { value: Utf8(\"a\") }, Literal { value: Utf8(\"b\") }])"); @@ -1464,7 +1464,7 @@ mod tests { // Test: a IN ('a', 'b', NULL) let list = vec![lit("a"), lit("b"), lit(ScalarValue::Utf8(None))]; let expr = in_list(Arc::clone(&col_a), list, &false, &schema)?; - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); let display_string = expr.to_string(); assert_eq!(sql_string, "a IN (a, b, NULL)"); assert_eq!(display_string, "Use a@0 IN (SET) ([Literal { value: Utf8(\"a\") }, Literal { value: Utf8(\"b\") }, Literal { value: Utf8(NULL) }])"); @@ -1472,7 +1472,7 @@ mod tests { // Test: a NOT IN ('a', 'b', NULL) let list = vec![lit("a"), lit("b"), lit(ScalarValue::Utf8(None))]; let expr = in_list(Arc::clone(&col_a), list, &true, &schema)?; - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); let display_string = expr.to_string(); assert_eq!(sql_string, "a NOT IN (a, b, NULL)"); assert_eq!(display_string, "a@0 NOT IN (SET) ([Literal { value: Utf8(\"a\") }, Literal { value: Utf8(\"b\") }, Literal { value: Utf8(NULL) }])"); diff --git a/datafusion/physical-expr/src/expressions/is_not_null.rs b/datafusion/physical-expr/src/expressions/is_not_null.rs index dd25ce756a06e..0619e72488581 100644 --- a/datafusion/physical-expr/src/expressions/is_not_null.rs +++ b/datafusion/physical-expr/src/expressions/is_not_null.rs @@ -120,13 +120,13 @@ pub fn is_not_null(arg: Arc) -> Result> mod tests { use super::*; use crate::expressions::col; - use crate::utils::sql_formatter; use arrow::array::{ Array, BooleanArray, Float64Array, Int32Array, StringArray, UnionArray, }; use arrow::buffer::ScalarBuffer; use arrow::datatypes::*; use datafusion_common::cast::as_boolean_array; + use datafusion_physical_expr_common::physical_expr::fmt_sql; #[test] fn is_not_null_op() -> Result<()> { @@ -213,7 +213,7 @@ mod tests { let expr = is_not_null(col("my_union", &schema).unwrap()).unwrap(); let display_string = expr.to_string(); assert_eq!(display_string, "my_union@0 IS NOT NULL"); - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); assert_eq!(sql_string, "my_union IS NOT NULL"); Ok(()) diff --git a/datafusion/physical-expr/src/expressions/is_null.rs b/datafusion/physical-expr/src/expressions/is_null.rs index 5df5e9b47b2e8..4c6081f35cad7 100644 --- a/datafusion/physical-expr/src/expressions/is_null.rs +++ b/datafusion/physical-expr/src/expressions/is_null.rs @@ -119,13 +119,13 @@ pub fn is_null(arg: Arc) -> Result> { mod tests { use super::*; use crate::expressions::col; - use crate::utils::sql_formatter; use arrow::array::{ Array, BooleanArray, Float64Array, Int32Array, StringArray, UnionArray, }; use arrow::buffer::ScalarBuffer; use arrow::datatypes::*; use datafusion_common::cast::as_boolean_array; + use datafusion_physical_expr_common::physical_expr::fmt_sql; #[test] fn is_null_op() -> Result<()> { @@ -224,7 +224,7 @@ mod tests { let expr = is_null(col("a", &schema)?).unwrap(); let display_string = expr.to_string(); assert_eq!(display_string, "a@0 IS NULL"); - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); assert_eq!(sql_string, "a IS NULL"); Ok(()) diff --git a/datafusion/physical-expr/src/expressions/like.rs b/datafusion/physical-expr/src/expressions/like.rs index 02755f9c6d67f..ebf9882665ba0 100644 --- a/datafusion/physical-expr/src/expressions/like.rs +++ b/datafusion/physical-expr/src/expressions/like.rs @@ -188,10 +188,10 @@ pub fn like( mod test { use super::*; use crate::expressions::col; - use crate::utils::sql_formatter; use arrow::array::*; use arrow::datatypes::Field; use datafusion_common::cast::as_boolean_array; + use datafusion_physical_expr_common::physical_expr::fmt_sql; macro_rules! test_like { ($A_VEC:expr, $B_VEC:expr, $VEC:expr, $NULLABLE: expr, $NEGATED:expr, $CASE_INSENSITIVE:expr,) => {{ @@ -284,7 +284,7 @@ mod test { assert_eq!(display_string, "a@0 LIKE b@1"); // fmt_sql format - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); assert_eq!(sql_string, "a LIKE b"); Ok(()) diff --git a/datafusion/physical-expr/src/expressions/literal.rs b/datafusion/physical-expr/src/expressions/literal.rs index 7ad0aeb7c73b2..0d0c0ecc62c79 100644 --- a/datafusion/physical-expr/src/expressions/literal.rs +++ b/datafusion/physical-expr/src/expressions/literal.rs @@ -109,13 +109,12 @@ pub fn lit(value: T) -> Arc { #[cfg(test)] mod tests { - use crate::utils::sql_formatter; - use super::*; use arrow::array::Int32Array; use arrow::datatypes::*; use datafusion_common::cast::as_int32_array; + use datafusion_physical_expr_common::physical_expr::fmt_sql; #[test] fn literal_i32() -> Result<()> { @@ -149,7 +148,7 @@ mod tests { let expr = lit(42i32); let display_string = expr.to_string(); assert_eq!(display_string, "42"); - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); assert_eq!(sql_string, "42"); Ok(()) diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index 72f721397b1f2..33a1bae14d420 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -201,7 +201,6 @@ pub fn negative( mod tests { use super::*; use crate::expressions::{col, Column}; - use crate::utils::sql_formatter; use arrow::array::*; use arrow::datatypes::DataType::{Float32, Float64, Int16, Int32, Int64, Int8}; @@ -209,6 +208,7 @@ mod tests { use datafusion_common::cast::as_primitive_array; use datafusion_common::{DataFusionError, ScalarValue}; + use datafusion_physical_expr_common::physical_expr::fmt_sql; use paste::paste; macro_rules! test_array_negative_op { @@ -392,7 +392,7 @@ mod tests { let expr = NegativeExpr::new(Arc::new(Column::new("a", 0))); let display_string = expr.to_string(); assert_eq!(display_string, "(- a@0)"); - let sql_string = sql_formatter(&expr).to_string(); + let sql_string = fmt_sql(&expr).to_string(); assert_eq!(sql_string, "(- a)"); Ok(()) diff --git a/datafusion/physical-expr/src/expressions/not.rs b/datafusion/physical-expr/src/expressions/not.rs index e370661d0faf8..8a3348b43d20b 100644 --- a/datafusion/physical-expr/src/expressions/not.rs +++ b/datafusion/physical-expr/src/expressions/not.rs @@ -192,12 +192,10 @@ mod tests { use std::sync::LazyLock; use super::*; - use crate::{ - expressions::{col, Column}, - utils::sql_formatter, - }; + use crate::expressions::{col, Column}; use arrow::{array::BooleanArray, datatypes::*}; + use datafusion_physical_expr_common::physical_expr::fmt_sql; #[test] fn neg_op() -> Result<()> { @@ -339,7 +337,7 @@ mod tests { let display_string = expr.to_string(); assert_eq!(display_string, "NOT a@0"); - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); assert_eq!(sql_string, "NOT a"); Ok(()) diff --git a/datafusion/physical-expr/src/expressions/try_cast.rs b/datafusion/physical-expr/src/expressions/try_cast.rs index 6933cb2e2638f..e49815cd8b644 100644 --- a/datafusion/physical-expr/src/expressions/try_cast.rs +++ b/datafusion/physical-expr/src/expressions/try_cast.rs @@ -154,7 +154,6 @@ pub fn try_cast( mod tests { use super::*; use crate::expressions::col; - use crate::utils::sql_formatter; use arrow::array::{ Decimal128Array, Decimal128Builder, StringArray, Time64NanosecondArray, }; @@ -165,6 +164,7 @@ mod tests { }, datatypes::*, }; + use datafusion_physical_expr_common::physical_expr::fmt_sql; // runs an end-to-end test of physical type cast // 1. construct a record batch with a column "a" of type A @@ -589,7 +589,7 @@ mod tests { let expr = try_cast(col("a", &schema)?, &schema, DataType::Int64)?; let display_string = expr.to_string(); assert_eq!(display_string, "TRY_CAST(a@0 AS Int64)"); - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); assert_eq!(sql_string, "TRY_CAST(a AS Int64)"); // Test string casting @@ -597,7 +597,7 @@ mod tests { let expr = try_cast(col("b", &schema)?, &schema, DataType::Int32)?; let display_string = expr.to_string(); assert_eq!(display_string, "TRY_CAST(b@0 AS Int32)"); - let sql_string = sql_formatter(expr.as_ref()).to_string(); + let sql_string = fmt_sql(expr.as_ref()).to_string(); assert_eq!(sql_string, "TRY_CAST(b AS Int32)"); Ok(()) diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index b4a918ef6f617..7e4c7f0e10ba8 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -16,9 +16,7 @@ // under the License. mod guarantee; -mod sql_formatter; pub use guarantee::{Guarantee, LiteralGuarantee}; -pub use sql_formatter::sql_formatter; use std::borrow::Borrow; use std::sync::Arc; diff --git a/datafusion/physical-expr/src/utils/sql_formatter.rs b/datafusion/physical-expr/src/utils/sql_formatter.rs deleted file mode 100644 index 495ee919383c9..0000000000000 --- a/datafusion/physical-expr/src/utils/sql_formatter.rs +++ /dev/null @@ -1,35 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::fmt::{self, Formatter}; - -use datafusion_physical_expr_common::physical_expr::PhysicalExpr; - -pub fn sql_formatter(expr: &dyn PhysicalExpr) -> impl fmt::Display + '_ { - struct Wrapper<'a> { - expr: &'a dyn PhysicalExpr, - } - - impl fmt::Display for Wrapper<'_> { - fn fmt(&self, f: &mut Formatter) -> fmt::Result { - self.expr.fmt_sql(f)?; - Ok(()) - } - } - - Wrapper { expr } -} diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index b36188426a979..1d3e23ea90974 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -48,6 +48,7 @@ use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::PhysicalExprRef; +use datafusion_physical_expr_common::physical_expr::fmt_sql; use futures::stream::{Stream, StreamExt}; use itertools::Itertools; use log::trace; @@ -169,17 +170,11 @@ impl DisplayAs for ProjectionExec { } DisplayFormatType::TreeRender => { for (i, (e, alias)) in self.expr().iter().enumerate() { - if e.as_any().downcast_ref::().is_some() { - e.fmt_sql(f)?; - writeln!(f)?; + let expr_sql = fmt_sql(e.as_ref()); + if &e.to_string() == alias { + writeln!(f, "expr{i}={expr_sql}")?; } else { - if &e.to_string() == alias { - write!(f, "expr{i}=")?; - } else { - write!(f, "{alias}=")?; - } - e.fmt_sql(f)?; - writeln!(f)?; + writeln!(f, "{alias}={expr_sql}")?; } } diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index d52546af72eb9..a9444da6c16d2 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -704,26 +704,29 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(Int64(1)) ROWS │ -05)│ BETWEEN UNBOUNDED │ -06)│ PRECEDING AND UNBOUNDED │ -07)│ FOLLOWING │ -08)└─────────────┬─────────────┘ -09)┌─────────────┴─────────────┐ -10)│ WindowAggExec │ -11)│ -------------------- │ -12)│ select_list: │ -13)│ count(Int64(1)) ROWS │ -14)│ BETWEEN UNBOUNDED │ -15)│ PRECEDING AND UNBOUNDED │ -16)│ FOLLOWING │ -17)└─────────────┬─────────────┘ -18)┌─────────────┴─────────────┐ -19)│ DataSourceExec │ -20)│ -------------------- │ -21)│ files: 1 │ -22)│ format: csv │ -23)└───────────────────────────┘ +04)│ count(*) ROWS BETWEEN │ +05)│ UNBOUNDED PRECEDING │ +06)│ AND UNBOUNDED FOLLOWING: │ +07)│ count(Int64(1)) ROWS │ +08)│ BETWEEN UNBOUNDED │ +09)│ PRECEDING AND UNBOUNDED │ +10)│ FOLLOWING │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ WindowAggExec │ +14)│ -------------------- │ +15)│ select_list: │ +16)│ count(Int64(1)) ROWS │ +17)│ BETWEEN UNBOUNDED │ +18)│ PRECEDING AND UNBOUNDED │ +19)│ FOLLOWING │ +20)└─────────────┬─────────────┘ +21)┌─────────────┴─────────────┐ +22)│ DataSourceExec │ +23)│ -------------------- │ +24)│ files: 1 │ +25)│ format: csv │ +26)└───────────────────────────┘ # Query with bounded window agg. query TT @@ -736,42 +739,43 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ sum(t1.v1) ORDER BY [t1.v1│ -05)│ ASC NULLS LAST] ROWS │ -06)│ BETWEEN 1 PRECEDING │ -07)│ AND CURRENT ROW │ -08)│ │ -09)│ v1 │ -10)└─────────────┬─────────────┘ -11)┌─────────────┴─────────────┐ -12)│ BoundedWindowAggExec │ -13)│ -------------------- │ -14)│ mode: Sorted │ -15)│ │ -16)│ select_list: │ -17)│ sum(t1.v1) ORDER BY [t1.v1│ -18)│ ASC NULLS LAST] ROWS │ -19)│ BETWEEN 1 PRECEDING │ -20)│ AND CURRENT ROW │ -21)└─────────────┬─────────────┘ -22)┌─────────────┴─────────────┐ -23)│ SortExec │ -24)│ -------------------- │ -25)│ v1@0 ASC NULLS LAST │ -26)└─────────────┬─────────────┘ -27)┌─────────────┴─────────────┐ -28)│ ProjectionExec │ -29)│ -------------------- │ -30)│ value │ -31)└─────────────┬─────────────┘ -32)┌─────────────┴─────────────┐ -33)│ LazyMemoryExec │ -34)│ -------------------- │ -35)│ batch_generators: │ -36)│ generate_series: start=1, │ -37)│ end=1000, batch_size │ -38)│ =8192 │ -39)└───────────────────────────┘ +04)│ rolling_sum: │ +05)│ sum(t1.v1) ORDER BY [t1.v1│ +06)│ ASC NULLS LAST] ROWS │ +07)│ BETWEEN 1 PRECEDING │ +08)│ AND CURRENT ROW │ +09)│ │ +10)│ v1: v1 │ +11)└─────────────┬─────────────┘ +12)┌─────────────┴─────────────┐ +13)│ BoundedWindowAggExec │ +14)│ -------------------- │ +15)│ mode: Sorted │ +16)│ │ +17)│ select_list: │ +18)│ sum(t1.v1) ORDER BY [t1.v1│ +19)│ ASC NULLS LAST] ROWS │ +20)│ BETWEEN 1 PRECEDING │ +21)│ AND CURRENT ROW │ +22)└─────────────┬─────────────┘ +23)┌─────────────┴─────────────┐ +24)│ SortExec │ +25)│ -------------------- │ +26)│ v1@0 ASC NULLS LAST │ +27)└─────────────┬─────────────┘ +28)┌─────────────┴─────────────┐ +29)│ ProjectionExec │ +30)│ -------------------- │ +31)│ v1: value │ +32)└─────────────┬─────────────┘ +33)┌─────────────┴─────────────┐ +34)│ LazyMemoryExec │ +35)│ -------------------- │ +36)│ batch_generators: │ +37)│ generate_series: start=1, │ +38)│ end=1000, batch_size │ +39)│ =8192 │ +40)└───────────────────────────┘ query TT explain select @@ -783,33 +787,39 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(Int64(1)) ROWS │ -05)│ BETWEEN UNBOUNDED │ -06)│ PRECEDING AND UNBOUNDED │ -07)│ FOLLOWING │ -08)│ │ -09)│ row_number() ROWS BETWEEN │ -10)│ UNBOUNDED PRECEDING AND │ -11)│ UNBOUNDED FOLLOWING │ -12)└─────────────┬─────────────┘ -13)┌─────────────┴─────────────┐ -14)│ WindowAggExec │ -15)│ -------------------- │ -16)│ select_list: │ -17)│ count(Int64(1)) ROWS │ -18)│ BETWEEN UNBOUNDED │ -19)│ PRECEDING AND UNBOUNDED │ -20)│ FOLLOWING, row_number() │ -21)│ ROWS BETWEEN UNBOUNDED │ -22)│ PRECEDING AND UNBOUNDED │ -23)│ FOLLOWING │ -24)└─────────────┬─────────────┘ -25)┌─────────────┴─────────────┐ -26)│ DataSourceExec │ -27)│ -------------------- │ -28)│ files: 1 │ -29)│ format: csv │ -30)└───────────────────────────┘ +04)│ count(*) ROWS BETWEEN │ +05)│ UNBOUNDED PRECEDING │ +06)│ AND UNBOUNDED FOLLOWING: │ +07)│ count(Int64(1)) ROWS │ +08)│ BETWEEN UNBOUNDED │ +09)│ PRECEDING AND UNBOUNDED │ +10)│ FOLLOWING │ +11)│ │ +12)│ row_number() ROWS BETWEEN │ +13)│ UNBOUNDED PRECEDING AND │ +14)│ UNBOUNDED FOLLOWING: │ +15)│ row_number() ROWS BETWEEN │ +16)│ UNBOUNDED PRECEDING AND │ +17)│ UNBOUNDED FOLLOWING │ +18)└─────────────┬─────────────┘ +19)┌─────────────┴─────────────┐ +20)│ WindowAggExec │ +21)│ -------------------- │ +22)│ select_list: │ +23)│ count(Int64(1)) ROWS │ +24)│ BETWEEN UNBOUNDED │ +25)│ PRECEDING AND UNBOUNDED │ +26)│ FOLLOWING, row_number() │ +27)│ ROWS BETWEEN UNBOUNDED │ +28)│ PRECEDING AND UNBOUNDED │ +29)│ FOLLOWING │ +30)└─────────────┬─────────────┘ +31)┌─────────────┴─────────────┐ +32)│ DataSourceExec │ +33)│ -------------------- │ +34)│ files: 1 │ +35)│ format: csv │ +36)└───────────────────────────┘ # Query for sort. query TT @@ -855,29 +865,30 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col │ -05)│ │ -06)│ int_col │ -07)│ │ -08)│ sum_col: │ -09)│ CAST(int_col AS Int64) + │ -10)│ bigint_col │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ RepartitionExec │ -14)│ -------------------- │ -15)│ output_partition_count: │ -16)│ 1 │ -17)│ │ -18)│ partitioning_scheme: │ -19)│ RoundRobinBatch(4) │ -20)└─────────────┬─────────────┘ -21)┌─────────────┴─────────────┐ -22)│ DataSourceExec │ -23)│ -------------------- │ -24)│ files: 1 │ -25)│ format: csv │ -26)└───────────────────────────┘ +04)│ bigint_col: │ +05)│ bigint_col │ +06)│ │ +07)│ int_col: int_col │ +08)│ │ +09)│ sum_col: │ +10)│ CAST(int_col AS Int64) + │ +11)│ bigint_col │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ output_partition_count: │ +17)│ 1 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ DataSourceExec │ +24)│ -------------------- │ +25)│ files: 1 │ +26)│ format: csv │ +27)└───────────────────────────┘ query TT explain select @@ -893,56 +904,67 @@ physical_plan 05)│ .int_col DESC NULLS │ 06)│ FIRST] RANGE BETWEEN │ 07)│ UNBOUNDED PRECEDING AND │ -08)│ CURRENT ROW │ -09)│ │ -10)│ row_number() ORDER BY │ -11)│ [table1.int_col ASC │ -12)│ NULLS LAST] RANGE │ -13)│ BETWEEN UNBOUNDED │ -14)│ PRECEDING AND CURRENT │ -15)│ ROW │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ BoundedWindowAggExec │ -19)│ -------------------- │ -20)│ mode: Sorted │ -21)│ │ -22)│ select_list: │ -23)│ row_number() ORDER BY │ -24)│ [table1.int_col ASC │ -25)│ NULLS LAST] RANGE │ -26)│ BETWEEN UNBOUNDED │ -27)│ PRECEDING AND CURRENT │ -28)│ ROW │ -29)└─────────────┬─────────────┘ -30)┌─────────────┴─────────────┐ -31)│ SortExec │ -32)│ -------------------- │ -33)│ int_col@0 ASC NULLS LAST │ -34)└─────────────┬─────────────┘ -35)┌─────────────┴─────────────┐ -36)│ BoundedWindowAggExec │ -37)│ -------------------- │ -38)│ mode: Sorted │ -39)│ │ -40)│ select_list: │ -41)│ rank() ORDER BY [table1 │ -42)│ .int_col DESC NULLS │ -43)│ FIRST] RANGE BETWEEN │ -44)│ UNBOUNDED PRECEDING AND │ -45)│ CURRENT ROW │ -46)└─────────────┬─────────────┘ -47)┌─────────────┴─────────────┐ -48)│ SortExec │ -49)│ -------------------- │ -50)│ int_col@0 DESC │ -51)└─────────────┬─────────────┘ -52)┌─────────────┴─────────────┐ -53)│ DataSourceExec │ -54)│ -------------------- │ -55)│ files: 1 │ -56)│ format: csv │ -57)└───────────────────────────┘ +08)│ CURRENT ROW: │ +09)│ rank() ORDER BY [table1 │ +10)│ .int_col DESC NULLS │ +11)│ FIRST] RANGE BETWEEN │ +12)│ UNBOUNDED PRECEDING AND │ +13)│ CURRENT ROW │ +14)│ │ +15)│ row_number() ORDER BY │ +16)│ [table1.int_col ASC │ +17)│ NULLS LAST] RANGE │ +18)│ BETWEEN UNBOUNDED │ +19)│ PRECEDING AND CURRENT │ +20)│ ROW: │ +21)│ row_number() ORDER BY │ +22)│ [table1.int_col ASC │ +23)│ NULLS LAST] RANGE │ +24)│ BETWEEN UNBOUNDED │ +25)│ PRECEDING AND CURRENT │ +26)│ ROW │ +27)└─────────────┬─────────────┘ +28)┌─────────────┴─────────────┐ +29)│ BoundedWindowAggExec │ +30)│ -------------------- │ +31)│ mode: Sorted │ +32)│ │ +33)│ select_list: │ +34)│ row_number() ORDER BY │ +35)│ [table1.int_col ASC │ +36)│ NULLS LAST] RANGE │ +37)│ BETWEEN UNBOUNDED │ +38)│ PRECEDING AND CURRENT │ +39)│ ROW │ +40)└─────────────┬─────────────┘ +41)┌─────────────┴─────────────┐ +42)│ SortExec │ +43)│ -------------------- │ +44)│ int_col@0 ASC NULLS LAST │ +45)└─────────────┬─────────────┘ +46)┌─────────────┴─────────────┐ +47)│ BoundedWindowAggExec │ +48)│ -------------------- │ +49)│ mode: Sorted │ +50)│ │ +51)│ select_list: │ +52)│ rank() ORDER BY [table1 │ +53)│ .int_col DESC NULLS │ +54)│ FIRST] RANGE BETWEEN │ +55)│ UNBOUNDED PRECEDING AND │ +56)│ CURRENT ROW │ +57)└─────────────┬─────────────┘ +58)┌─────────────┴─────────────┐ +59)│ SortExec │ +60)│ -------------------- │ +61)│ int_col@0 DESC │ +62)└─────────────┬─────────────┘ +63)┌─────────────┴─────────────┐ +64)│ DataSourceExec │ +65)│ -------------------- │ +66)│ files: 1 │ +67)│ format: csv │ +68)└───────────────────────────┘ # Query with projection on parquet query TT @@ -952,29 +974,30 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col │ -05)│ │ -06)│ int_col │ -07)│ │ -08)│ sum_col: │ -09)│ CAST(int_col AS Int64) + │ -10)│ bigint_col │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ RepartitionExec │ -14)│ -------------------- │ -15)│ output_partition_count: │ -16)│ 1 │ -17)│ │ -18)│ partitioning_scheme: │ -19)│ RoundRobinBatch(4) │ -20)└─────────────┬─────────────┘ -21)┌─────────────┴─────────────┐ -22)│ DataSourceExec │ -23)│ -------------------- │ -24)│ files: 1 │ -25)│ format: parquet │ -26)└───────────────────────────┘ +04)│ bigint_col: │ +05)│ bigint_col │ +06)│ │ +07)│ int_col: int_col │ +08)│ │ +09)│ sum_col: │ +10)│ CAST(int_col AS Int64) + │ +11)│ bigint_col │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ output_partition_count: │ +17)│ 1 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ DataSourceExec │ +24)│ -------------------- │ +25)│ files: 1 │ +26)│ format: parquet │ +27)└───────────────────────────┘ # Query with projection on memory @@ -985,21 +1008,22 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col │ -05)│ │ -06)│ int_col │ -07)│ │ -08)│ sum_col: │ -09)│ CAST(int_col AS Int64) + │ -10)│ bigint_col │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ DataSourceExec │ -14)│ -------------------- │ -15)│ bytes: 1560 │ -16)│ format: memory │ -17)│ rows: 1 │ -18)└───────────────────────────┘ +04)│ bigint_col: │ +05)│ bigint_col │ +06)│ │ +07)│ int_col: int_col │ +08)│ │ +09)│ sum_col: │ +10)│ CAST(int_col AS Int64) + │ +11)│ bigint_col │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ DataSourceExec │ +15)│ -------------------- │ +16)│ bytes: 1560 │ +17)│ format: memory │ +18)│ rows: 1 │ +19)└───────────────────────────┘ # Query with projection on json query TT @@ -1009,45 +1033,13 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ bigint_col │ -05)│ │ -06)│ int_col │ -07)│ │ -08)│ sum_col: │ -09)│ int_col + bigint_col │ -10)└─────────────┬─────────────┘ -11)┌─────────────┴─────────────┐ -12)│ RepartitionExec │ -13)│ -------------------- │ -14)│ output_partition_count: │ -15)│ 1 │ -16)│ │ -17)│ partitioning_scheme: │ -18)│ RoundRobinBatch(4) │ -19)└─────────────┬─────────────┘ -20)┌─────────────┴─────────────┐ -21)│ DataSourceExec │ -22)│ -------------------- │ -23)│ files: 1 │ -24)│ format: json │ -25)└───────────────────────────┘ - - -# Query with projection on arrow -query TT -explain SELECT int_col, bigint_col, int_col+bigint_col AS sum_col FROM table5; ----- -physical_plan -01)┌───────────────────────────┐ -02)│ ProjectionExec │ -03)│ -------------------- │ -04)│ bigint_col │ -05)│ │ -06)│ int_col │ -07)│ │ -08)│ sum_col: │ -09)│ CAST(int_col AS Int64) + │ -10)│ bigint_col │ +04)│ bigint_col: │ +05)│ bigint_col │ +06)│ │ +07)│ int_col: int_col │ +08)│ │ +09)│ sum_col: │ +10)│ int_col + bigint_col │ 11)└─────────────┬─────────────┘ 12)┌─────────────┴─────────────┐ 13)│ RepartitionExec │ @@ -1062,9 +1054,43 @@ physical_plan 22)│ DataSourceExec │ 23)│ -------------------- │ 24)│ files: 1 │ -25)│ format: arrow │ +25)│ format: json │ 26)└───────────────────────────┘ + +# Query with projection on arrow +query TT +explain SELECT int_col, bigint_col, int_col+bigint_col AS sum_col FROM table5; +---- +physical_plan +01)┌───────────────────────────┐ +02)│ ProjectionExec │ +03)│ -------------------- │ +04)│ bigint_col: │ +05)│ bigint_col │ +06)│ │ +07)│ int_col: int_col │ +08)│ │ +09)│ sum_col: │ +10)│ CAST(int_col AS Int64) + │ +11)│ bigint_col │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ RepartitionExec │ +15)│ -------------------- │ +16)│ output_partition_count: │ +17)│ 1 │ +18)│ │ +19)│ partitioning_scheme: │ +20)│ RoundRobinBatch(4) │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ DataSourceExec │ +24)│ -------------------- │ +25)│ files: 1 │ +26)│ format: arrow │ +27)└───────────────────────────┘ + # Query with PartialSortExec. query TT EXPLAIN SELECT * @@ -1154,29 +1180,30 @@ physical_plan 38)│ CAST(string_col AS ││ │ 39)│ Utf8View) ││ partitioning_scheme: │ 40)│ ││ RoundRobinBatch(4) │ -41)│ bigint_col ││ │ -42)│ ││ │ -43)│ date_col ││ │ -44)│ ││ │ -45)│ int_col ││ │ +41)│ bigint_col: ││ │ +42)│ bigint_col ││ │ +43)│ ││ │ +44)│ date_col: date_col ││ │ +45)│ int_col: int_col ││ │ 46)│ ││ │ -47)│ string_col ││ │ -48)└─────────────┬─────────────┘└─────────────┬─────────────┘ -49)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -50)│ RepartitionExec ││ DataSourceExec │ -51)│ -------------------- ││ -------------------- │ -52)│ output_partition_count: ││ files: 1 │ -53)│ 1 ││ format: parquet │ -54)│ ││ │ -55)│ partitioning_scheme: ││ │ -56)│ RoundRobinBatch(4) ││ │ -57)└─────────────┬─────────────┘└───────────────────────────┘ -58)┌─────────────┴─────────────┐ -59)│ DataSourceExec │ -60)│ -------------------- │ -61)│ files: 1 │ -62)│ format: csv │ -63)└───────────────────────────┘ +47)│ string_col: ││ │ +48)│ string_col ││ │ +49)└─────────────┬─────────────┘└─────────────┬─────────────┘ +50)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +51)│ RepartitionExec ││ DataSourceExec │ +52)│ -------------------- ││ -------------------- │ +53)│ output_partition_count: ││ files: 1 │ +54)│ 1 ││ format: parquet │ +55)│ ││ │ +56)│ partitioning_scheme: ││ │ +57)│ RoundRobinBatch(4) ││ │ +58)└─────────────┬─────────────┘└───────────────────────────┘ +59)┌─────────────┴─────────────┐ +60)│ DataSourceExec │ +61)│ -------------------- │ +62)│ files: 1 │ +63)│ format: csv │ +64)└───────────────────────────┘ # Query with outer hash join. query TT @@ -1225,29 +1252,30 @@ physical_plan 40)│ CAST(string_col AS ││ │ 41)│ Utf8View) ││ partitioning_scheme: │ 42)│ ││ RoundRobinBatch(4) │ -43)│ bigint_col ││ │ -44)│ ││ │ -45)│ date_col ││ │ -46)│ ││ │ -47)│ int_col ││ │ +43)│ bigint_col: ││ │ +44)│ bigint_col ││ │ +45)│ ││ │ +46)│ date_col: date_col ││ │ +47)│ int_col: int_col ││ │ 48)│ ││ │ -49)│ string_col ││ │ -50)└─────────────┬─────────────┘└─────────────┬─────────────┘ -51)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -52)│ RepartitionExec ││ DataSourceExec │ -53)│ -------------------- ││ -------------------- │ -54)│ output_partition_count: ││ files: 1 │ -55)│ 1 ││ format: parquet │ -56)│ ││ │ -57)│ partitioning_scheme: ││ │ -58)│ RoundRobinBatch(4) ││ │ -59)└─────────────┬─────────────┘└───────────────────────────┘ -60)┌─────────────┴─────────────┐ -61)│ DataSourceExec │ -62)│ -------------------- │ -63)│ files: 1 │ -64)│ format: csv │ -65)└───────────────────────────┘ +49)│ string_col: ││ │ +50)│ string_col ││ │ +51)└─────────────┬─────────────┘└─────────────┬─────────────┘ +52)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +53)│ RepartitionExec ││ DataSourceExec │ +54)│ -------------------- ││ -------------------- │ +55)│ output_partition_count: ││ files: 1 │ +56)│ 1 ││ format: parquet │ +57)│ ││ │ +58)│ partitioning_scheme: ││ │ +59)│ RoundRobinBatch(4) ││ │ +60)└─────────────┬─────────────┘└───────────────────────────┘ +61)┌─────────────┴─────────────┐ +62)│ DataSourceExec │ +63)│ -------------------- │ +64)│ files: 1 │ +65)│ format: csv │ +66)└───────────────────────────┘ # Query with nested loop join. query TT @@ -1400,74 +1428,75 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(Int64(1)) │ -05)└─────────────┬─────────────┘ -06)┌─────────────┴─────────────┐ -07)│ AggregateExec │ -08)│ -------------------- │ -09)│ aggr: count(Int64(1)) │ -10)│ │ -11)│ group_by: │ -12)│ name@0 as name │ -13)│ │ -14)│ mode: │ -15)│ SinglePartitioned │ -16)└─────────────┬─────────────┘ -17)┌─────────────┴─────────────┐ -18)│ InterleaveExec ├──────────────┐ -19)└─────────────┬─────────────┘ │ -20)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -21)│ AggregateExec ││ AggregateExec │ -22)│ -------------------- ││ -------------------- │ -23)│ aggr ││ aggr │ -24)│ ││ │ -25)│ group_by: ││ group_by: │ -26)│ name@0 as name ││ name@0 as name │ -27)│ ││ │ -28)│ mode: ││ mode: │ -29)│ FinalPartitioned ││ FinalPartitioned │ -30)└─────────────┬─────────────┘└─────────────┬─────────────┘ -31)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -32)│ CoalesceBatchesExec ││ CoalesceBatchesExec │ -33)│ -------------------- ││ -------------------- │ -34)│ target_batch_size: ││ target_batch_size: │ -35)│ 8192 ││ 8192 │ -36)└─────────────┬─────────────┘└─────────────┬─────────────┘ -37)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -38)│ RepartitionExec ││ RepartitionExec │ -39)│ -------------------- ││ -------------------- │ -40)│ output_partition_count: ││ output_partition_count: │ -41)│ 4 ││ 4 │ -42)│ ││ │ -43)│ partitioning_scheme: ││ partitioning_scheme: │ -44)│ Hash([name@0], 4) ││ Hash([name@0], 4) │ -45)└─────────────┬─────────────┘└─────────────┬─────────────┘ -46)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -47)│ RepartitionExec ││ RepartitionExec │ -48)│ -------------------- ││ -------------------- │ -49)│ output_partition_count: ││ output_partition_count: │ -50)│ 1 ││ 1 │ -51)│ ││ │ -52)│ partitioning_scheme: ││ partitioning_scheme: │ -53)│ RoundRobinBatch(4) ││ RoundRobinBatch(4) │ -54)└─────────────┬─────────────┘└─────────────┬─────────────┘ -55)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -56)│ AggregateExec ││ AggregateExec │ -57)│ -------------------- ││ -------------------- │ -58)│ aggr ││ aggr │ -59)│ ││ │ -60)│ group_by: ││ group_by: │ -61)│ name@0 as name ││ name@0 as name │ -62)│ ││ │ -63)│ mode: Partial ││ mode: Partial │ -64)└─────────────┬─────────────┘└─────────────┬─────────────┘ -65)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -66)│ DataSourceExec ││ DataSourceExec │ -67)│ -------------------- ││ -------------------- │ -68)│ bytes: 1320 ││ bytes: 1312 │ -69)│ format: memory ││ format: memory │ -70)│ rows: 1 ││ rows: 1 │ -71)└───────────────────────────┘└───────────────────────────┘ +04)│ count(*): │ +05)│ count(Int64(1)) │ +06)└─────────────┬─────────────┘ +07)┌─────────────┴─────────────┐ +08)│ AggregateExec │ +09)│ -------------------- │ +10)│ aggr: count(Int64(1)) │ +11)│ │ +12)│ group_by: │ +13)│ name@0 as name │ +14)│ │ +15)│ mode: │ +16)│ SinglePartitioned │ +17)└─────────────┬─────────────┘ +18)┌─────────────┴─────────────┐ +19)│ InterleaveExec ├──────────────┐ +20)└─────────────┬─────────────┘ │ +21)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +22)│ AggregateExec ││ AggregateExec │ +23)│ -------------------- ││ -------------------- │ +24)│ aggr ││ aggr │ +25)│ ││ │ +26)│ group_by: ││ group_by: │ +27)│ name@0 as name ││ name@0 as name │ +28)│ ││ │ +29)│ mode: ││ mode: │ +30)│ FinalPartitioned ││ FinalPartitioned │ +31)└─────────────┬─────────────┘└─────────────┬─────────────┘ +32)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +33)│ CoalesceBatchesExec ││ CoalesceBatchesExec │ +34)│ -------------------- ││ -------------------- │ +35)│ target_batch_size: ││ target_batch_size: │ +36)│ 8192 ││ 8192 │ +37)└─────────────┬─────────────┘└─────────────┬─────────────┘ +38)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +39)│ RepartitionExec ││ RepartitionExec │ +40)│ -------------------- ││ -------------------- │ +41)│ output_partition_count: ││ output_partition_count: │ +42)│ 4 ││ 4 │ +43)│ ││ │ +44)│ partitioning_scheme: ││ partitioning_scheme: │ +45)│ Hash([name@0], 4) ││ Hash([name@0], 4) │ +46)└─────────────┬─────────────┘└─────────────┬─────────────┘ +47)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +48)│ RepartitionExec ││ RepartitionExec │ +49)│ -------------------- ││ -------------------- │ +50)│ output_partition_count: ││ output_partition_count: │ +51)│ 1 ││ 1 │ +52)│ ││ │ +53)│ partitioning_scheme: ││ partitioning_scheme: │ +54)│ RoundRobinBatch(4) ││ RoundRobinBatch(4) │ +55)└─────────────┬─────────────┘└─────────────┬─────────────┘ +56)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +57)│ AggregateExec ││ AggregateExec │ +58)│ -------------------- ││ -------------------- │ +59)│ aggr ││ aggr │ +60)│ ││ │ +61)│ group_by: ││ group_by: │ +62)│ name@0 as name ││ name@0 as name │ +63)│ ││ │ +64)│ mode: Partial ││ mode: Partial │ +65)└─────────────┬─────────────┘└─────────────┬─────────────┘ +66)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +67)│ DataSourceExec ││ DataSourceExec │ +68)│ -------------------- ││ -------------------- │ +69)│ bytes: 1320 ││ bytes: 1312 │ +70)│ format: memory ││ format: memory │ +71)│ rows: 1 ││ rows: 1 │ +72)└───────────────────────────┘└───────────────────────────┘ # cleanup statement ok @@ -1834,61 +1863,62 @@ physical_plan 01)┌───────────────────────────┐ 02)│ ProjectionExec │ 03)│ -------------------- │ -04)│ count(Int64(1)) │ -05)└─────────────┬─────────────┘ -06)┌─────────────┴─────────────┐ -07)│ AggregateExec │ -08)│ -------------------- │ -09)│ aggr: count(Int64(1)) │ -10)│ mode: Final │ -11)└─────────────┬─────────────┘ -12)┌─────────────┴─────────────┐ -13)│ CoalescePartitionsExec │ -14)└─────────────┬─────────────┘ -15)┌─────────────┴─────────────┐ -16)│ AggregateExec │ -17)│ -------------------- │ -18)│ aggr: count(Int64(1)) │ -19)│ mode: Partial │ -20)└─────────────┬─────────────┘ -21)┌─────────────┴─────────────┐ -22)│ RepartitionExec │ -23)│ -------------------- │ -24)│ output_partition_count: │ -25)│ 1 │ -26)│ │ -27)│ partitioning_scheme: │ -28)│ RoundRobinBatch(4) │ -29)└─────────────┬─────────────┘ -30)┌─────────────┴─────────────┐ -31)│ ProjectionExec │ -32)└─────────────┬─────────────┘ -33)┌─────────────┴─────────────┐ -34)│ GlobalLimitExec │ -35)│ -------------------- │ -36)│ limit: 3 │ -37)│ skip: 6 │ -38)└─────────────┬─────────────┘ -39)┌─────────────┴─────────────┐ -40)│ CoalesceBatchesExec │ -41)│ -------------------- │ -42)│ limit: 9 │ -43)│ │ -44)│ target_batch_size: │ -45)│ 8192 │ -46)└─────────────┬─────────────┘ -47)┌─────────────┴─────────────┐ -48)│ FilterExec │ -49)│ -------------------- │ -50)│ predicate: a@0 > 3 │ -51)└─────────────┬─────────────┘ -52)┌─────────────┴─────────────┐ -53)│ DataSourceExec │ -54)│ -------------------- │ -55)│ bytes: 160 │ -56)│ format: memory │ -57)│ rows: 1 │ -58)└───────────────────────────┘ +04)│ count(*): │ +05)│ count(Int64(1)) │ +06)└─────────────┬─────────────┘ +07)┌─────────────┴─────────────┐ +08)│ AggregateExec │ +09)│ -------------------- │ +10)│ aggr: count(Int64(1)) │ +11)│ mode: Final │ +12)└─────────────┬─────────────┘ +13)┌─────────────┴─────────────┐ +14)│ CoalescePartitionsExec │ +15)└─────────────┬─────────────┘ +16)┌─────────────┴─────────────┐ +17)│ AggregateExec │ +18)│ -------------------- │ +19)│ aggr: count(Int64(1)) │ +20)│ mode: Partial │ +21)└─────────────┬─────────────┘ +22)┌─────────────┴─────────────┐ +23)│ RepartitionExec │ +24)│ -------------------- │ +25)│ output_partition_count: │ +26)│ 1 │ +27)│ │ +28)│ partitioning_scheme: │ +29)│ RoundRobinBatch(4) │ +30)└─────────────┬─────────────┘ +31)┌─────────────┴─────────────┐ +32)│ ProjectionExec │ +33)└─────────────┬─────────────┘ +34)┌─────────────┴─────────────┐ +35)│ GlobalLimitExec │ +36)│ -------------------- │ +37)│ limit: 3 │ +38)│ skip: 6 │ +39)└─────────────┬─────────────┘ +40)┌─────────────┴─────────────┐ +41)│ CoalesceBatchesExec │ +42)│ -------------------- │ +43)│ limit: 9 │ +44)│ │ +45)│ target_batch_size: │ +46)│ 8192 │ +47)└─────────────┬─────────────┘ +48)┌─────────────┴─────────────┐ +49)│ FilterExec │ +50)│ -------------------- │ +51)│ predicate: a@0 > 3 │ +52)└─────────────┬─────────────┘ +53)┌─────────────┴─────────────┐ +54)│ DataSourceExec │ +55)│ -------------------- │ +56)│ bytes: 160 │ +57)│ format: memory │ +58)│ rows: 1 │ +59)└───────────────────────────┘ # Test explain tree for LazyMemoryExec query TT