From 5f424d393f32b40d213c8d13695c01cd600845ca Mon Sep 17 00:00:00 2001 From: Justus Flerlage Date: Fri, 20 Sep 2024 22:27:31 +0200 Subject: [PATCH 01/25] Fix unparsing OFFSET (#12539) --- datafusion/sql/src/unparser/plan.rs | 15 ++++++++++++++ datafusion/sql/tests/cases/plan_to_sql.rs | 24 +++++++++++++++++++++-- 2 files changed, 37 insertions(+), 2 deletions(-) diff --git a/datafusion/sql/src/unparser/plan.rs b/datafusion/sql/src/unparser/plan.rs index bc1e94375b31..ad162a9222e6 100644 --- a/datafusion/sql/src/unparser/plan.rs +++ b/datafusion/sql/src/unparser/plan.rs @@ -322,6 +322,21 @@ impl Unparser<'_> { )))); } + if limit.skip > 0 { + let Some(query) = query.as_mut() else { + return internal_err!( + "Offset operator only valid in a statement context." + ); + }; + query.offset(Some(ast::Offset { + rows: ast::OffsetRows::None, + value: ast::Expr::Value(ast::Value::Number( + limit.skip.to_string(), + false, + )), + })); + } + self.select_to_sql_recursively( limit.input.as_ref(), query, diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index 02771bce6d9f..5ef70d2abe5e 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -622,8 +622,11 @@ fn test_pretty_roundtrip() -> Result<()> { Ok(()) } -fn sql_round_trip(query: &str, expect: &str) { - let statement = Parser::new(&GenericDialect {}) +fn sql_round_trip(dialect: D, query: &str, expect: &str) +where + D: Dialect, +{ + let statement = Parser::new(&dialect) .try_with_sql(query) .unwrap() .parse_statement() @@ -817,6 +820,7 @@ fn test_table_scan_pushdown() -> Result<()> { #[test] fn test_interval_lhs_eq() { sql_round_trip( + GenericDialect {}, "select interval '2 seconds' = interval '2 seconds'", "SELECT (INTERVAL '2.000000000 SECS' = INTERVAL '2.000000000 SECS')", ); @@ -825,7 +829,23 @@ fn test_interval_lhs_eq() { #[test] fn test_interval_lhs_lt() { sql_round_trip( + GenericDialect {}, "select interval '2 seconds' < interval '2 seconds'", "SELECT (INTERVAL '2.000000000 SECS' < INTERVAL '2.000000000 SECS')", ); } + +#[test] +fn test_without_offset() { + sql_round_trip(MySqlDialect {}, "select 1", "SELECT 1"); +} + +#[test] +fn test_with_offset0() { + sql_round_trip(MySqlDialect {}, "select 1 offset 0", "SELECT 1"); +} + +#[test] +fn test_with_offset95() { + sql_round_trip(MySqlDialect {}, "select 1 offset 95", "SELECT 1 OFFSET 95"); +} From 21ec3328428d4f27aa84abf7bd0c58332edcaed5 Mon Sep 17 00:00:00 2001 From: Nick Cameron Date: Sat, 21 Sep 2024 08:29:13 +1200 Subject: [PATCH 02/25] support EXTRACT on intervals and durations (#12514) Signed-off-by: Nick Cameron --- .../functions/src/datetime/date_part.rs | 47 ++++- datafusion/sqllogictest/test_files/expr.slt | 183 ++++++++++++++++++ 2 files changed, 224 insertions(+), 6 deletions(-) diff --git a/datafusion/functions/src/datetime/date_part.rs b/datafusion/functions/src/datetime/date_part.rs index 8ee82d872651..b6a9a1c7e9db 100644 --- a/datafusion/functions/src/datetime/date_part.rs +++ b/datafusion/functions/src/datetime/date_part.rs @@ -23,8 +23,10 @@ use arrow::array::{Array, ArrayRef, Float64Array}; use arrow::compute::kernels::cast_utils::IntervalUnit; use arrow::compute::{binary, cast, date_part, DatePart}; use arrow::datatypes::DataType::{ - Date32, Date64, Float64, Time32, Time64, Timestamp, Utf8, Utf8View, + Date32, Date64, Duration, Float64, Interval, Time32, Time64, Timestamp, Utf8, + Utf8View, }; +use arrow::datatypes::IntervalUnit::{DayTime, MonthDayNano, YearMonth}; use arrow::datatypes::TimeUnit::{Microsecond, Millisecond, Nanosecond, Second}; use arrow::datatypes::{DataType, TimeUnit}; @@ -109,6 +111,20 @@ impl DatePartFunc { Exact(vec![Utf8View, Time64(Microsecond)]), Exact(vec![Utf8, Time64(Nanosecond)]), Exact(vec![Utf8View, Time64(Nanosecond)]), + Exact(vec![Utf8, Interval(YearMonth)]), + Exact(vec![Utf8View, Interval(YearMonth)]), + Exact(vec![Utf8, Interval(DayTime)]), + Exact(vec![Utf8View, Interval(DayTime)]), + Exact(vec![Utf8, Interval(MonthDayNano)]), + Exact(vec![Utf8View, Interval(MonthDayNano)]), + Exact(vec![Utf8, Duration(Second)]), + Exact(vec![Utf8View, Duration(Second)]), + Exact(vec![Utf8, Duration(Millisecond)]), + Exact(vec![Utf8View, Duration(Millisecond)]), + Exact(vec![Utf8, Duration(Microsecond)]), + Exact(vec![Utf8View, Duration(Microsecond)]), + Exact(vec![Utf8, Duration(Nanosecond)]), + Exact(vec![Utf8View, Duration(Nanosecond)]), ], Volatility::Immutable, ), @@ -224,10 +240,28 @@ fn seconds(array: &dyn Array, unit: TimeUnit) -> Result { let subsecs = date_part(array, DatePart::Nanosecond)?; let subsecs = as_int32_array(subsecs.as_ref())?; - let r: Float64Array = binary(secs, subsecs, |secs, subsecs| { - (secs as f64 + (subsecs as f64 / 1_000_000_000_f64)) * sf - })?; - Ok(Arc::new(r)) + // Special case where there are no nulls. + if subsecs.null_count() == 0 { + let r: Float64Array = binary(secs, subsecs, |secs, subsecs| { + (secs as f64 + ((subsecs % 1_000_000_000) as f64 / 1_000_000_000_f64)) * sf + })?; + Ok(Arc::new(r)) + } else { + // Nulls in secs are preserved, nulls in subsecs are treated as zero to account for the case + // where the number of nanoseconds overflows. + let r: Float64Array = secs + .iter() + .zip(subsecs) + .map(|(secs, subsecs)| { + secs.map(|secs| { + let subsecs = subsecs.unwrap_or(0); + (secs as f64 + ((subsecs % 1_000_000_000) as f64 / 1_000_000_000_f64)) + * sf + }) + }) + .collect(); + Ok(Arc::new(r)) + } } fn epoch(array: &dyn Array) -> Result { @@ -256,7 +290,8 @@ fn epoch(array: &dyn Array) -> Result { Time64(Nanosecond) => { as_time64_nanosecond_array(array)?.unary(|x| x as f64 / 1_000_000_000_f64) } - d => return exec_err!("Can not convert {d:?} to epoch"), + Interval(_) | Duration(_) => return seconds(array, Second), + d => return exec_err!("Cannot convert {d:?} to epoch"), }; Ok(Arc::new(f)) } diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index e8d8329d34e1..182afff7a693 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -1472,6 +1472,189 @@ SELECT extract(epoch from arrow_cast('1969-12-31', 'Date64')) ---- -86400 +# test_extract_interval + +query R +SELECT extract(year from arrow_cast('10 years', 'Interval(YearMonth)')) +---- +10 + +query R +SELECT extract(month from arrow_cast('10 years', 'Interval(YearMonth)')) +---- +0 + +query R +SELECT extract(year from arrow_cast('10 months', 'Interval(YearMonth)')) +---- +0 + +query R +SELECT extract(month from arrow_cast('10 months', 'Interval(YearMonth)')) +---- +10 + +query R +SELECT extract(year from arrow_cast('20 months', 'Interval(YearMonth)')) +---- +1 + +query R +SELECT extract(month from arrow_cast('20 months', 'Interval(YearMonth)')) +---- +8 + +query error DataFusion error: Arrow error: Compute error: Year does not support: Interval\(DayTime\) +SELECT extract(year from arrow_cast('10 days', 'Interval(DayTime)')) + +query error DataFusion error: Arrow error: Compute error: Month does not support: Interval\(DayTime\) +SELECT extract(month from arrow_cast('10 days', 'Interval(DayTime)')) + +query R +SELECT extract(day from arrow_cast('10 days', 'Interval(DayTime)')) +---- +10 + +query R +SELECT extract(day from arrow_cast('14400 minutes', 'Interval(DayTime)')) +---- +0 + +query R +SELECT extract(minute from arrow_cast('14400 minutes', 'Interval(DayTime)')) +---- +14400 + +query R +SELECT extract(second from arrow_cast('5.1 seconds', 'Interval(DayTime)')) +---- +5 + +query R +SELECT extract(second from arrow_cast('14400 minutes', 'Interval(DayTime)')) +---- +864000 + +query R +SELECT extract(second from arrow_cast('2 months', 'Interval(MonthDayNano)')) +---- +0 + +query R +SELECT extract(second from arrow_cast('2 days', 'Interval(MonthDayNano)')) +---- +0 + +query R +SELECT extract(second from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) +---- +2 + +query R +SELECT extract(seconds from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) +---- +2 + +query R +SELECT extract(epoch from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) +---- +2 + +query R +SELECT extract(milliseconds from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) +---- +2000 + +query R +SELECT extract(second from arrow_cast('2030 milliseconds', 'Interval(MonthDayNano)')) +---- +2.03 + +query R +SELECT extract(second from arrow_cast(NULL, 'Interval(MonthDayNano)')) +---- +NULL + +statement ok +create table t (id int, i interval) as values + (0, interval '5 months 1 day 10 nanoseconds'), + (1, interval '1 year 3 months'), + (2, interval '3 days 2 milliseconds'), + (3, interval '2 seconds'), + (4, interval '8 months'), + (5, NULL); + +query IRR rowsort +select + id, + extract(second from i), + extract(month from i) +from t +order by id; +---- +0 0.00000001 5 +1 0 15 +2 0.002 0 +3 2 0 +4 0 8 +5 NULL NULL + +statement ok +drop table t; + +# test_extract_duration + +query R +SELECT extract(second from arrow_cast(2, 'Duration(Second)')) +---- +2 + +query R +SELECT extract(seconds from arrow_cast(2, 'Duration(Second)')) +---- +2 + +query R +SELECT extract(epoch from arrow_cast(2, 'Duration(Second)')) +---- +2 + +query R +SELECT extract(millisecond from arrow_cast(2, 'Duration(Second)')) +---- +2000 + +query R +SELECT extract(second from arrow_cast(2, 'Duration(Millisecond)')) +---- +0.002 + +query R +SELECT extract(second from arrow_cast(2002, 'Duration(Millisecond)')) +---- +2.002 + +query R +SELECT extract(millisecond from arrow_cast(2002, 'Duration(Millisecond)')) +---- +2002 + +query R +SELECT extract(day from arrow_cast(864000, 'Duration(Second)')) +---- +10 + +query error DataFusion error: Arrow error: Compute error: Month does not support: Duration\(Second\) +SELECT extract(month from arrow_cast(864000, 'Duration(Second)')) + +query error DataFusion error: Arrow error: Compute error: Year does not support: Duration\(Second\) +SELECT extract(year from arrow_cast(864000, 'Duration(Second)')) + +query R +SELECT extract(day from arrow_cast(NULL, 'Duration(Second)')) +---- +NULL + # test_extract_date_part_func query B From e5c0c0ca74e626178caf32c578c0580c0d4f1fe6 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Sat, 21 Sep 2024 18:20:46 +0800 Subject: [PATCH 03/25] feat: Support adding a single new table factory to SessionStateBuilder (#12563) --- .../core/src/execution/session_state.rs | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index f656fae4016f..a46e5c36156c 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -1193,6 +1193,18 @@ impl SessionStateBuilder { self } + /// Add a [`TableProviderFactory`] to the map of factories + pub fn with_table_factory( + mut self, + key: String, + table_factory: Arc, + ) -> Self { + let mut table_factories = self.table_factories.unwrap_or_default(); + table_factories.insert(key, table_factory); + self.table_factories = Some(table_factories); + self + } + /// Set the map of [`TableProviderFactory`]s pub fn with_table_factories( mut self, @@ -1929,4 +1941,22 @@ mod tests { Optimizer::default().rules.len() + 1 ); } + + #[test] + fn test_with_table_factories() -> Result<()> { + use crate::test_util::TestTableFactory; + + let state = SessionStateBuilder::new().build(); + let table_factories = state.table_factories(); + assert!(table_factories.is_empty()); + + let table_factory = Arc::new(TestTableFactory {}); + let state = SessionStateBuilder::new() + .with_table_factory("employee".to_string(), table_factory) + .build(); + let table_factories = state.table_factories(); + assert_eq!(table_factories.len(), 1); + assert!(table_factories.contains_key("employee")); + Ok(()) + } } From d9cb6e677ec437a9c6ff2cc0900f902804da39bb Mon Sep 17 00:00:00 2001 From: ngli-me <107162634+ngli-me@users.noreply.github.com> Date: Sat, 21 Sep 2024 06:54:23 -0400 Subject: [PATCH 04/25] Implement PartialOrd for Expr and sub fields/structs without using hash values (#12481) * Derive PartialOrd for WindowFrameBound, and added partial_cmp for ScalarUDF based on the equals fn. * Derived PartialOrd where possible for structs not using DFSchema/Schema. Otherwise, implemented PartialOrd, ignoring the schema field. * Added additional tests to verify partial ord for Expr, DdlStatement, and LogicalPlan. * Formatting. * Added PartialOrd implementations where necessary, otherwise derived for structs implementing UserDefinedLogicalNodeCore. * Added Comparable versions for structs with more than 3 comparable fields. * Added additional comments, specifying which fields caused a manual implementation of `PartialOrd` to be necessary. --------- Co-authored-by: nglime --- datafusion/common/src/display/mod.rs | 4 +- .../common/src/functional_dependencies.rs | 4 +- datafusion/common/src/join_type.rs | 4 +- datafusion/core/src/physical_planner.rs | 9 + .../tests/user_defined/user_defined_plan.rs | 2 +- datafusion/expr/src/expr.rs | 71 ++-- datafusion/expr/src/logical_plan/ddl.rs | 223 +++++++++++- datafusion/expr/src/logical_plan/dml.rs | 34 +- datafusion/expr/src/logical_plan/extension.rs | 19 +- datafusion/expr/src/logical_plan/plan.rs | 334 +++++++++++++++++- datafusion/expr/src/logical_plan/statement.rs | 41 ++- datafusion/expr/src/udf.rs | 11 + datafusion/expr/src/window_frame.rs | 4 +- datafusion/optimizer/src/analyzer/subquery.rs | 7 + .../optimizer/src/optimize_projections/mod.rs | 28 ++ datafusion/optimizer/src/push_down_filter.rs | 8 + datafusion/optimizer/src/test/user_defined.rs | 2 +- .../tests/cases/roundtrip_logical_plan.rs | 2 +- .../tests/cases/roundtrip_logical_plan.rs | 16 + 19 files changed, 744 insertions(+), 79 deletions(-) diff --git a/datafusion/common/src/display/mod.rs b/datafusion/common/src/display/mod.rs index 2345c0e4c4fc..c12e7419e4b6 100644 --- a/datafusion/common/src/display/mod.rs +++ b/datafusion/common/src/display/mod.rs @@ -27,7 +27,7 @@ use std::{ /// Represents which type of plan, when storing multiple /// for use in EXPLAIN plans -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub enum PlanType { /// The initial LogicalPlan provided to DataFusion InitialLogicalPlan, @@ -96,7 +96,7 @@ impl Display for PlanType { } /// Represents some sort of execution plan, in String form -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub struct StringifiedPlan { /// An identifier of what type of plan this string represents pub plan_type: PlanType, diff --git a/datafusion/common/src/functional_dependencies.rs b/datafusion/common/src/functional_dependencies.rs index 666ea73027b3..90f4e6e7e3d1 100644 --- a/datafusion/common/src/functional_dependencies.rs +++ b/datafusion/common/src/functional_dependencies.rs @@ -30,7 +30,7 @@ use crate::{DFSchema, DFSchemaRef, DataFusionError, JoinType, Result}; use sqlparser::ast::TableConstraint; /// This object defines a constraint on a table. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub enum Constraint { /// Columns with the given indices form a composite primary key (they are /// jointly unique and not nullable): @@ -40,7 +40,7 @@ pub enum Constraint { } /// This object encapsulates a list of functional constraints: -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub struct Constraints { inner: Vec, } diff --git a/datafusion/common/src/join_type.rs b/datafusion/common/src/join_type.rs index 0a00a57ba45f..fbdae1c50a83 100644 --- a/datafusion/common/src/join_type.rs +++ b/datafusion/common/src/join_type.rs @@ -26,7 +26,7 @@ use crate::error::_not_impl_err; use crate::{DataFusionError, Result}; /// Join type -#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Hash)] pub enum JoinType { /// Inner Join Inner, @@ -88,7 +88,7 @@ impl FromStr for JoinType { } /// Join constraint -#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Hash)] pub enum JoinConstraint { /// Join ON On, diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 2010a5c66412..84d285fc2509 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1974,6 +1974,7 @@ fn tuple_err(value: (Result, Result)) -> Result<(T, R)> { #[cfg(test)] mod tests { use std::any::Any; + use std::cmp::Ordering; use std::fmt::{self, Debug}; use std::ops::{BitAnd, Not}; @@ -2528,6 +2529,14 @@ mod tests { } } + // Implementation needed for `UserDefinedLogicalNodeCore`, since the only field is + // a schema, we can't derive `PartialOrd`, and we can't compare these. + impl PartialOrd for NoOpExtensionNode { + fn partial_cmp(&self, _other: &Self) -> Option { + None + } + } + impl UserDefinedLogicalNodeCore for NoOpExtensionNode { fn name(&self) -> &str { "NoOp" diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 56edeab443c7..101e676484d2 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -387,7 +387,7 @@ impl OptimizerRule for TopKOptimizerRule { } } -#[derive(PartialEq, Eq, Hash)] +#[derive(PartialEq, Eq, PartialOrd, Hash)] struct TopKPlanNode { k: usize, input: LogicalPlan, diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index db0bfd6b1bc2..8cb759b8811e 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -27,11 +27,11 @@ use std::sync::Arc; use crate::expr_fn::binary_expr; use crate::logical_plan::Subquery; use crate::utils::expr_to_columns; +use crate::Volatility; use crate::{ built_in_window_function, udaf, BuiltInWindowFunction, ExprSchemable, Operator, Signature, WindowFrame, WindowUDF, }; -use crate::{window_frame, Volatility}; use arrow::datatypes::{DataType, FieldRef}; use datafusion_common::tree_node::{ @@ -193,7 +193,7 @@ use sqlparser::ast::{ /// } /// // The return value controls whether to continue visiting the tree /// Ok(TreeNodeRecursion::Continue) -/// }).unwrap();; +/// }).unwrap(); /// // All subtrees have been visited and literals found /// assert_eq!(scalars.len(), 2); /// assert!(scalars.contains(&ScalarValue::Int32(Some(5)))); @@ -223,7 +223,7 @@ use sqlparser::ast::{ /// assert!(rewritten.transformed); /// // to 42 = 5 AND b = 6 /// assert_eq!(rewritten.data, lit(42).eq(lit(5)).and(col("b").eq(lit(6)))); -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub enum Expr { /// An expression with a specific name. Alias(Alias), @@ -354,7 +354,7 @@ impl<'a> From<(Option<&'a TableReference>, &'a FieldRef)> for Expr { } /// UNNEST expression. -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct Unnest { pub expr: Box, } @@ -374,7 +374,7 @@ impl Unnest { } /// Alias expression -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct Alias { pub expr: Box, pub relation: Option, @@ -397,7 +397,7 @@ impl Alias { } /// Binary expression -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct BinaryExpr { /// Left-hand side of the expression pub left: Box, @@ -448,7 +448,7 @@ impl Display for BinaryExpr { } /// CASE expression -#[derive(Clone, Debug, PartialEq, Eq, Hash)] +#[derive(Clone, Debug, PartialEq, Eq, PartialOrd, Hash)] pub struct Case { /// Optional base expression that can be compared to literal values in the "when" expressions pub expr: Option>, @@ -474,7 +474,7 @@ impl Case { } /// LIKE expression -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct Like { pub negated: bool, pub expr: Box, @@ -504,7 +504,7 @@ impl Like { } /// BETWEEN expression -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct Between { /// The value to compare pub expr: Box, @@ -529,7 +529,7 @@ impl Between { } /// ScalarFunction expression invokes a built-in scalar function -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct ScalarFunction { /// The function pub func: Arc, @@ -567,7 +567,7 @@ pub enum GetFieldAccess { } /// Cast expression -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct Cast { /// The expression being cast pub expr: Box, @@ -583,7 +583,7 @@ impl Cast { } /// TryCast Expression -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct TryCast { /// The expression being cast pub expr: Box, @@ -599,7 +599,7 @@ impl TryCast { } /// SORT expression -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct Sort { /// The expression to sort on pub expr: Expr, @@ -651,7 +651,7 @@ impl Display for Sort { /// See also [`ExprFunctionExt`] to set these fields on `Expr` /// /// [`ExprFunctionExt`]: crate::expr_fn::ExprFunctionExt -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct AggregateFunction { /// Name of the function pub func: Arc, @@ -789,7 +789,7 @@ impl From> for WindowFunctionDefinition { /// .build() /// .unwrap(); /// ``` -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct WindowFunction { /// Name of the function pub fun: WindowFunctionDefinition, @@ -800,7 +800,7 @@ pub struct WindowFunction { /// List of order by expressions pub order_by: Vec, /// Window frame - pub window_frame: window_frame::WindowFrame, + pub window_frame: WindowFrame, /// Specifies how NULL value is treated: ignore or respect pub null_treatment: Option, } @@ -840,7 +840,7 @@ pub fn find_df_window_func(name: &str) -> Option { } /// EXISTS expression -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct Exists { /// subquery that will produce a single column of data pub subquery: Subquery, @@ -888,7 +888,7 @@ impl AggregateUDF { } /// InList expression -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct InList { /// The expression to compare pub expr: Box, @@ -910,7 +910,7 @@ impl InList { } /// IN subquery -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct InSubquery { /// The expression to compare pub expr: Box, @@ -935,7 +935,7 @@ impl InSubquery { /// /// The type of these parameters is inferred using [`Expr::infer_placeholder_types`] /// or can be specified directly using `PREPARE` statements. -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct Placeholder { /// The identifier of the parameter, including the leading `$` (e.g, `"$1"` or `"$foo"`) pub id: String, @@ -956,7 +956,7 @@ impl Placeholder { /// for Postgres definition. /// See /// for Apache Spark definition. -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub enum GroupingSet { /// Rollup grouping sets Rollup(Vec), @@ -989,7 +989,7 @@ impl GroupingSet { } /// Additional options for wildcards, e.g. Snowflake `EXCLUDE`/`RENAME` and Bigquery `EXCEPT`. -#[derive(Clone, PartialEq, Eq, Hash, Debug, Default)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug, Default)] pub struct WildcardOptions { /// `[ILIKE...]`. /// Snowflake syntax: @@ -1045,7 +1045,7 @@ impl Display for WildcardOptions { } /// The planned expressions for `REPLACE` -#[derive(Clone, PartialEq, Eq, Hash, Debug, Default)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug, Default)] pub struct PlannedReplaceSelectItem { /// The original ast nodes pub items: Vec, @@ -1071,18 +1071,6 @@ impl PlannedReplaceSelectItem { } } -/// Fixed seed for the hashing so that Ords are consistent across runs -const SEED: ahash::RandomState = ahash::RandomState::with_seeds(0, 0, 0, 0); - -impl PartialOrd for Expr { - fn partial_cmp(&self, other: &Self) -> Option { - let s = SEED.hash_one(self); - let o = SEED.hash_one(other); - - Some(s.cmp(&o)) - } -} - impl Expr { #[deprecated(since = "40.0.0", note = "use schema_name instead")] pub fn display_name(&self) -> Result { @@ -2432,20 +2420,15 @@ mod test { #[test] fn test_partial_ord() { - // Test validates that partial ord is defined for Expr using hashes, not + // Test validates that partial ord is defined for Expr, not // intended to exhaustively test all possibilities let exp1 = col("a") + lit(1); let exp2 = col("a") + lit(2); let exp3 = !(col("a") + lit(2)); - // Since comparisons are done using hash value of the expression - // expr < expr2 may return false, or true. There is no guaranteed result. - // The only guarantee is "<" operator should have the opposite result of ">=" operator - let greater_or_equal = exp1 >= exp2; - assert_eq!(exp1 < exp2, !greater_or_equal); - - let greater_or_equal = exp3 >= exp2; - assert_eq!(exp3 < exp2, !greater_or_equal); + assert!(exp1 < exp2); + assert!(exp3 > exp2); + assert!(exp1 < exp3) } #[test] diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index 3fc43200efe6..9aaa5c98037a 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +use crate::{Expr, LogicalPlan, SortExpr, Volatility}; +use std::cmp::Ordering; use std::collections::HashMap; use std::sync::Arc; use std::{ @@ -22,15 +24,13 @@ use std::{ hash::{Hash, Hasher}, }; -use crate::{Expr, LogicalPlan, SortExpr, Volatility}; - use crate::expr::Sort; use arrow::datatypes::DataType; use datafusion_common::{Constraints, DFSchemaRef, SchemaReference, TableReference}; use sqlparser::ast::Ident; /// Various types of DDL (CREATE / DROP) catalog manipulation -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub enum DdlStatement { /// Creates an external table. CreateExternalTable(CreateExternalTable), @@ -232,8 +232,59 @@ impl Hash for CreateExternalTable { } } +// Manual implementation needed because of `schema`, `options`, and `column_defaults` fields. +// Comparison excludes these fields. +impl PartialOrd for CreateExternalTable { + fn partial_cmp(&self, other: &Self) -> Option { + #[derive(PartialEq, PartialOrd)] + struct ComparableCreateExternalTable<'a> { + /// The table name + pub name: &'a TableReference, + /// The physical location + pub location: &'a String, + /// The file type of physical file + pub file_type: &'a String, + /// Partition Columns + pub table_partition_cols: &'a Vec, + /// Option to not error if table already exists + pub if_not_exists: &'a bool, + /// SQL used to create the table, if available + pub definition: &'a Option, + /// Order expressions supplied by user + pub order_exprs: &'a Vec>, + /// Whether the table is an infinite streams + pub unbounded: &'a bool, + /// The list of constraints in the schema, such as primary key, unique, etc. + pub constraints: &'a Constraints, + } + let comparable_self = ComparableCreateExternalTable { + name: &self.name, + location: &self.location, + file_type: &self.file_type, + table_partition_cols: &self.table_partition_cols, + if_not_exists: &self.if_not_exists, + definition: &self.definition, + order_exprs: &self.order_exprs, + unbounded: &self.unbounded, + constraints: &self.constraints, + }; + let comparable_other = ComparableCreateExternalTable { + name: &other.name, + location: &other.location, + file_type: &other.file_type, + table_partition_cols: &other.table_partition_cols, + if_not_exists: &other.if_not_exists, + definition: &other.definition, + order_exprs: &other.order_exprs, + unbounded: &other.unbounded, + constraints: &other.constraints, + }; + comparable_self.partial_cmp(&comparable_other) + } +} + /// Creates an in memory table. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub struct CreateMemoryTable { /// The table name pub name: TableReference, @@ -250,7 +301,7 @@ pub struct CreateMemoryTable { } /// Creates a view. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, PartialOrd, Eq, Hash)] pub struct CreateView { /// The table name pub name: TableReference, @@ -273,6 +324,16 @@ pub struct CreateCatalog { pub schema: DFSchemaRef, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for CreateCatalog { + fn partial_cmp(&self, other: &Self) -> Option { + match self.catalog_name.partial_cmp(&other.catalog_name) { + Some(Ordering::Equal) => self.if_not_exists.partial_cmp(&other.if_not_exists), + cmp => cmp, + } + } +} + /// Creates a schema. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct CreateCatalogSchema { @@ -284,6 +345,16 @@ pub struct CreateCatalogSchema { pub schema: DFSchemaRef, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for CreateCatalogSchema { + fn partial_cmp(&self, other: &Self) -> Option { + match self.schema_name.partial_cmp(&other.schema_name) { + Some(Ordering::Equal) => self.if_not_exists.partial_cmp(&other.if_not_exists), + cmp => cmp, + } + } +} + /// Drops a table. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct DropTable { @@ -295,6 +366,16 @@ pub struct DropTable { pub schema: DFSchemaRef, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for DropTable { + fn partial_cmp(&self, other: &Self) -> Option { + match self.name.partial_cmp(&other.name) { + Some(Ordering::Equal) => self.if_exists.partial_cmp(&other.if_exists), + cmp => cmp, + } + } +} + /// Drops a view. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct DropView { @@ -306,6 +387,16 @@ pub struct DropView { pub schema: DFSchemaRef, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for DropView { + fn partial_cmp(&self, other: &Self) -> Option { + match self.name.partial_cmp(&other.name) { + Some(Ordering::Equal) => self.if_exists.partial_cmp(&other.if_exists), + cmp => cmp, + } + } +} + /// Drops a schema #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct DropCatalogSchema { @@ -319,6 +410,19 @@ pub struct DropCatalogSchema { pub schema: DFSchemaRef, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for DropCatalogSchema { + fn partial_cmp(&self, other: &Self) -> Option { + match self.name.partial_cmp(&other.name) { + Some(Ordering::Equal) => match self.if_exists.partial_cmp(&other.if_exists) { + Some(Ordering::Equal) => self.cascade.partial_cmp(&other.cascade), + cmp => cmp, + }, + cmp => cmp, + } + } +} + /// Arguments passed to `CREATE FUNCTION` /// /// Note this meant to be the same as from sqlparser's [`sqlparser::ast::Statement::CreateFunction`] @@ -336,7 +440,40 @@ pub struct CreateFunction { /// Dummy schema pub schema: DFSchemaRef, } -#[derive(Clone, PartialEq, Eq, Hash, Debug)] + +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for CreateFunction { + fn partial_cmp(&self, other: &Self) -> Option { + #[derive(PartialEq, PartialOrd)] + struct ComparableCreateFunction<'a> { + pub or_replace: &'a bool, + pub temporary: &'a bool, + pub name: &'a String, + pub args: &'a Option>, + pub return_type: &'a Option, + pub params: &'a CreateFunctionBody, + } + let comparable_self = ComparableCreateFunction { + or_replace: &self.or_replace, + temporary: &self.temporary, + name: &self.name, + args: &self.args, + return_type: &self.return_type, + params: &self.params, + }; + let comparable_other = ComparableCreateFunction { + or_replace: &other.or_replace, + temporary: &other.temporary, + name: &other.name, + args: &other.args, + return_type: &other.return_type, + params: &other.params, + }; + comparable_self.partial_cmp(&comparable_other) + } +} + +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct OperateFunctionArg { // TODO: figure out how to support mode // pub mode: Option, @@ -344,7 +481,7 @@ pub struct OperateFunctionArg { pub data_type: DataType, pub default_expr: Option, } -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub struct CreateFunctionBody { /// LANGUAGE lang_name pub language: Option, @@ -361,6 +498,15 @@ pub struct DropFunction { pub schema: DFSchemaRef, } +impl PartialOrd for DropFunction { + fn partial_cmp(&self, other: &Self) -> Option { + match self.name.partial_cmp(&other.name) { + Some(Ordering::Equal) => self.if_exists.partial_cmp(&other.if_exists), + cmp => cmp, + } + } +} + #[derive(Clone, PartialEq, Eq, Hash, Debug)] pub struct CreateIndex { pub name: Option, @@ -371,3 +517,66 @@ pub struct CreateIndex { pub if_not_exists: bool, pub schema: DFSchemaRef, } + +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for CreateIndex { + fn partial_cmp(&self, other: &Self) -> Option { + #[derive(PartialEq, PartialOrd)] + struct ComparableCreateIndex<'a> { + pub name: &'a Option, + pub table: &'a TableReference, + pub using: &'a Option, + pub columns: &'a Vec, + pub unique: &'a bool, + pub if_not_exists: &'a bool, + } + let comparable_self = ComparableCreateIndex { + name: &self.name, + table: &self.table, + using: &self.using, + columns: &self.columns, + unique: &self.unique, + if_not_exists: &self.if_not_exists, + }; + let comparable_other = ComparableCreateIndex { + name: &other.name, + table: &other.table, + using: &other.using, + columns: &other.columns, + unique: &other.unique, + if_not_exists: &other.if_not_exists, + }; + comparable_self.partial_cmp(&comparable_other) + } +} + +#[cfg(test)] +mod test { + use crate::{CreateCatalog, DdlStatement, DropView}; + use datafusion_common::{DFSchema, DFSchemaRef, TableReference}; + use std::cmp::Ordering; + + #[test] + fn test_partial_ord() { + let catalog = DdlStatement::CreateCatalog(CreateCatalog { + catalog_name: "name".to_string(), + if_not_exists: false, + schema: DFSchemaRef::new(DFSchema::empty()), + }); + let catalog_2 = DdlStatement::CreateCatalog(CreateCatalog { + catalog_name: "name".to_string(), + if_not_exists: true, + schema: DFSchemaRef::new(DFSchema::empty()), + }); + + assert_eq!(catalog.partial_cmp(&catalog_2), Some(Ordering::Less)); + + let drop_view = DdlStatement::DropView(DropView { + name: TableReference::from("table"), + if_exists: false, + schema: DFSchemaRef::new(DFSchema::empty()), + }); + + assert_eq!(drop_view.partial_cmp(&catalog), Some(Ordering::Greater)); + } +} diff --git a/datafusion/expr/src/logical_plan/dml.rs b/datafusion/expr/src/logical_plan/dml.rs index 025bb7b289dc..c2ed9dc0781c 100644 --- a/datafusion/expr/src/logical_plan/dml.rs +++ b/datafusion/expr/src/logical_plan/dml.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::cmp::Ordering; use std::collections::HashMap; use std::fmt::{self, Debug, Display, Formatter}; use std::hash::{Hash, Hasher}; @@ -63,6 +64,23 @@ impl PartialEq for CopyTo { // Implement Eq (no need for additional logic over PartialEq) impl Eq for CopyTo {} +// Manual implementation needed because of `file_type` and `options` fields. +// Comparison excludes these field. +impl PartialOrd for CopyTo { + fn partial_cmp(&self, other: &Self) -> Option { + match self.input.partial_cmp(&other.input) { + Some(Ordering::Equal) => match self.output_url.partial_cmp(&other.output_url) + { + Some(Ordering::Equal) => { + self.partition_by.partial_cmp(&other.partition_by) + } + cmp => cmp, + }, + cmp => cmp, + } + } +} + // Implement Hash manually impl Hash for CopyTo { fn hash(&self, state: &mut H) { @@ -112,7 +130,21 @@ impl DmlStatement { } } -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +// Manual implementation needed because of `table_schema` and `output_schema` fields. +// Comparison excludes these fields. +impl PartialOrd for DmlStatement { + fn partial_cmp(&self, other: &Self) -> Option { + match self.table_name.partial_cmp(&other.table_name) { + Some(Ordering::Equal) => match self.op.partial_cmp(&other.op) { + Some(Ordering::Equal) => self.input.partial_cmp(&other.input), + cmp => cmp, + }, + cmp => cmp, + } + } +} + +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub enum WriteOp { InsertOverwrite, InsertInto, diff --git a/datafusion/expr/src/logical_plan/extension.rs b/datafusion/expr/src/logical_plan/extension.rs index 5514ec2925d9..d49c85fb6fd6 100644 --- a/datafusion/expr/src/logical_plan/extension.rs +++ b/datafusion/expr/src/logical_plan/extension.rs @@ -18,6 +18,7 @@ //! This module defines the interface for logical nodes use crate::{Expr, LogicalPlan}; use datafusion_common::{DFSchema, DFSchemaRef, Result}; +use std::cmp::Ordering; use std::hash::{Hash, Hasher}; use std::{any::Any, collections::HashSet, fmt, sync::Arc}; @@ -193,6 +194,7 @@ pub trait UserDefinedLogicalNode: fmt::Debug + Send + Sync { /// Note: [`UserDefinedLogicalNode`] is not constrained by [`Eq`] /// directly because it must remain object safe. fn dyn_eq(&self, other: &dyn UserDefinedLogicalNode) -> bool; + fn dyn_ord(&self, other: &dyn UserDefinedLogicalNode) -> Option; } impl Hash for dyn UserDefinedLogicalNode { @@ -201,12 +203,18 @@ impl Hash for dyn UserDefinedLogicalNode { } } -impl std::cmp::PartialEq for dyn UserDefinedLogicalNode { +impl PartialEq for dyn UserDefinedLogicalNode { fn eq(&self, other: &Self) -> bool { self.dyn_eq(other) } } +impl PartialOrd for dyn UserDefinedLogicalNode { + fn partial_cmp(&self, other: &Self) -> Option { + self.dyn_ord(other) + } +} + impl Eq for dyn UserDefinedLogicalNode {} /// This trait facilitates implementation of the [`UserDefinedLogicalNode`]. @@ -215,7 +223,7 @@ impl Eq for dyn UserDefinedLogicalNode {} /// [user_defined_plan.rs](https://github.com/apache/datafusion/blob/main/datafusion/core/tests/user_defined/user_defined_plan.rs) /// file for an example of how to use this extension API. pub trait UserDefinedLogicalNodeCore: - fmt::Debug + Eq + Hash + Sized + Send + Sync + 'static + fmt::Debug + Eq + PartialOrd + Hash + Sized + Send + Sync + 'static { /// Return the plan's name. fn name(&self) -> &str; @@ -346,6 +354,13 @@ impl UserDefinedLogicalNode for T { None => false, } } + + fn dyn_ord(&self, other: &dyn UserDefinedLogicalNode) -> Option { + other + .as_any() + .downcast_ref::() + .and_then(|other| self.partial_cmp(other)) + } } fn get_all_columns_from_schema(schema: &DFSchema) -> HashSet { diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index b3f9b26fa46e..6a88382061cd 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -17,6 +17,7 @@ //! Logical plan types +use std::cmp::Ordering; use std::collections::{HashMap, HashSet}; use std::fmt::{self, Debug, Display, Formatter}; use std::hash::{Hash, Hasher}; @@ -190,7 +191,7 @@ pub use datafusion_common::{JoinConstraint, JoinType}; /// # } /// ``` /// -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub enum LogicalPlan { /// Evaluates an arbitrary list of expressions (essentially a /// SELECT with an expression list) on its input. @@ -2012,6 +2013,13 @@ pub struct EmptyRelation { pub schema: DFSchemaRef, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for EmptyRelation { + fn partial_cmp(&self, other: &Self) -> Option { + self.produce_one_row.partial_cmp(&other.produce_one_row) + } +} + /// A variadic query operation, Recursive CTE. /// /// # Recursive Query Evaluation @@ -2034,7 +2042,7 @@ pub struct EmptyRelation { /// intermediate table, then empty the intermediate table. /// /// [Postgres Docs]: https://www.postgresql.org/docs/current/queries-with.html#QUERIES-WITH-RECURSIVE -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub struct RecursiveQuery { /// Name of the query pub name: String, @@ -2059,6 +2067,13 @@ pub struct Values { pub values: Vec>, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for Values { + fn partial_cmp(&self, other: &Self) -> Option { + self.values.partial_cmp(&other.values) + } +} + /// Evaluates an arbitrary list of expressions (essentially a /// SELECT with an expression list) on its input. #[derive(Clone, PartialEq, Eq, Hash, Debug)] @@ -2073,6 +2088,16 @@ pub struct Projection { pub schema: DFSchemaRef, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for Projection { + fn partial_cmp(&self, other: &Self) -> Option { + match self.expr.partial_cmp(&other.expr) { + Some(Ordering::Equal) => self.input.partial_cmp(&other.input), + cmp => cmp, + } + } +} + impl Projection { /// Create a new Projection pub fn try_new(expr: Vec, input: Arc) -> Result { @@ -2172,6 +2197,16 @@ impl SubqueryAlias { } } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for SubqueryAlias { + fn partial_cmp(&self, other: &Self) -> Option { + match self.input.partial_cmp(&other.input) { + Some(Ordering::Equal) => self.alias.partial_cmp(&other.alias), + cmp => cmp, + } + } +} + /// Filters rows from its input that do not match an /// expression (essentially a WHERE clause with a predicate /// expression). @@ -2183,7 +2218,7 @@ impl SubqueryAlias { /// /// Filter should not be created directly but instead use `try_new()` /// and that these fields are only pub to support pattern matching -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] #[non_exhaustive] pub struct Filter { /// The predicate expression, which must have Boolean type. @@ -2409,6 +2444,16 @@ impl Window { } } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for Window { + fn partial_cmp(&self, other: &Self) -> Option { + match self.input.partial_cmp(&other.input) { + Some(Ordering::Equal) => self.window_expr.partial_cmp(&other.window_expr), + cmp => cmp, + } + } +} + /// Produces rows from a table provider by reference or from the context #[derive(Clone)] pub struct TableScan { @@ -2451,6 +2496,37 @@ impl PartialEq for TableScan { impl Eq for TableScan {} +// Manual implementation needed because of `source` and `projected_schema` fields. +// Comparison excludes these field. +impl PartialOrd for TableScan { + fn partial_cmp(&self, other: &Self) -> Option { + #[derive(PartialEq, PartialOrd)] + struct ComparableTableScan<'a> { + /// The name of the table + pub table_name: &'a TableReference, + /// Optional column indices to use as a projection + pub projection: &'a Option>, + /// Optional expressions to be used as filters by the table provider + pub filters: &'a Vec, + /// Optional number of rows to read + pub fetch: &'a Option, + } + let comparable_self = ComparableTableScan { + table_name: &self.table_name, + projection: &self.projection, + filters: &self.filters, + fetch: &self.fetch, + }; + let comparable_other = ComparableTableScan { + table_name: &other.table_name, + projection: &other.projection, + filters: &other.filters, + fetch: &other.fetch, + }; + comparable_self.partial_cmp(&comparable_other) + } +} + impl Hash for TableScan { fn hash(&self, state: &mut H) { self.table_name.hash(state); @@ -2526,8 +2602,18 @@ pub struct CrossJoin { pub schema: DFSchemaRef, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for CrossJoin { + fn partial_cmp(&self, other: &Self) -> Option { + match self.left.partial_cmp(&other.left) { + Some(Ordering::Equal) => self.right.partial_cmp(&other.right), + cmp => cmp, + } + } +} + /// Repartition the plan based on a partitioning scheme. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub struct Repartition { /// The incoming logical plan pub input: Arc, @@ -2544,9 +2630,16 @@ pub struct Union { pub schema: DFSchemaRef, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for Union { + fn partial_cmp(&self, other: &Self) -> Option { + self.inputs.partial_cmp(&other.inputs) + } +} + /// Prepare a statement but do not execute it. Prepare statements can have 0 or more /// `Expr::Placeholder` expressions that are filled in during execution -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub struct Prepare { /// The name of the statement pub name: String, @@ -2586,6 +2679,15 @@ pub struct DescribeTable { pub output_schema: DFSchemaRef, } +// Manual implementation of `PartialOrd`, returning none since there are no comparable types in +// `DescribeTable`. This allows `LogicalPlan` to derive `PartialOrd`. +impl PartialOrd for DescribeTable { + fn partial_cmp(&self, _other: &Self) -> Option { + // There is no relevant comparison for schemas + None + } +} + /// Produces a relation with string representations of /// various parts of the plan #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -2602,6 +2704,36 @@ pub struct Explain { pub logical_optimization_succeeded: bool, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for Explain { + fn partial_cmp(&self, other: &Self) -> Option { + #[derive(PartialEq, PartialOrd)] + struct ComparableExplain<'a> { + /// Should extra (detailed, intermediate plans) be included? + pub verbose: &'a bool, + /// The logical plan that is being EXPLAIN'd + pub plan: &'a Arc, + /// Represent the various stages plans have gone through + pub stringified_plans: &'a Vec, + /// Used by physical planner to check if should proceed with planning + pub logical_optimization_succeeded: &'a bool, + } + let comparable_self = ComparableExplain { + verbose: &self.verbose, + plan: &self.plan, + stringified_plans: &self.stringified_plans, + logical_optimization_succeeded: &self.logical_optimization_succeeded, + }; + let comparable_other = ComparableExplain { + verbose: &other.verbose, + plan: &other.plan, + stringified_plans: &other.stringified_plans, + logical_optimization_succeeded: &other.logical_optimization_succeeded, + }; + comparable_self.partial_cmp(&comparable_other) + } +} + /// Runs the actual plan, and then prints the physical plan with /// with execution metrics. #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -2614,6 +2746,16 @@ pub struct Analyze { pub schema: DFSchemaRef, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for Analyze { + fn partial_cmp(&self, other: &Self) -> Option { + match self.verbose.partial_cmp(&other.verbose) { + Some(Ordering::Equal) => self.input.partial_cmp(&other.input), + cmp => cmp, + } + } +} + /// Extension operator defined outside of DataFusion // TODO(clippy): This clippy `allow` should be removed if // the manual `PartialEq` is removed in favor of a derive. @@ -2634,8 +2776,14 @@ impl PartialEq for Extension { } } +impl PartialOrd for Extension { + fn partial_cmp(&self, other: &Self) -> Option { + self.node.partial_cmp(&other.node) + } +} + /// Produces the first `n` tuples from its input and discards the rest. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub struct Limit { /// Number of rows to skip before fetch pub skip: usize, @@ -2647,7 +2795,7 @@ pub struct Limit { } /// Removes duplicate rows from the input -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub enum Distinct { /// Plain `DISTINCT` referencing all selection expressions All(Arc), @@ -2745,6 +2893,38 @@ impl DistinctOn { } } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for DistinctOn { + fn partial_cmp(&self, other: &Self) -> Option { + #[derive(PartialEq, PartialOrd)] + struct ComparableDistinctOn<'a> { + /// The `DISTINCT ON` clause expression list + pub on_expr: &'a Vec, + /// The selected projection expression list + pub select_expr: &'a Vec, + /// The `ORDER BY` clause, whose initial expressions must match those of the `ON` clause when + /// present. Note that those matching expressions actually wrap the `ON` expressions with + /// additional info pertaining to the sorting procedure (i.e. ASC/DESC, and NULLS FIRST/LAST). + pub sort_expr: &'a Option>, + /// The logical plan that is being DISTINCT'd + pub input: &'a Arc, + } + let comparable_self = ComparableDistinctOn { + on_expr: &self.on_expr, + select_expr: &self.select_expr, + sort_expr: &self.sort_expr, + input: &self.input, + }; + let comparable_other = ComparableDistinctOn { + on_expr: &other.on_expr, + select_expr: &other.select_expr, + sort_expr: &other.sort_expr, + input: &other.input, + }; + comparable_self.partial_cmp(&comparable_other) + } +} + /// Aggregates its input based on a set of grouping and aggregate /// expressions (e.g. SUM). #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -2849,6 +3029,21 @@ impl Aggregate { } } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for Aggregate { + fn partial_cmp(&self, other: &Self) -> Option { + match self.input.partial_cmp(&other.input) { + Some(Ordering::Equal) => { + match self.group_expr.partial_cmp(&other.group_expr) { + Some(Ordering::Equal) => self.aggr_expr.partial_cmp(&other.aggr_expr), + cmp => cmp, + } + } + cmp => cmp, + } + } +} + /// Checks whether any expression in `group_expr` contains `Expr::GroupingSet`. fn contains_grouping_set(group_expr: &[Expr]) -> bool { group_expr @@ -2947,7 +3142,7 @@ fn calc_func_dependencies_for_project( } /// Sorts its input according to a list of sort expressions. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub struct Sort { /// The sort expressions pub expr: Vec, @@ -3013,8 +3208,50 @@ impl Join { } } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for Join { + fn partial_cmp(&self, other: &Self) -> Option { + #[derive(PartialEq, PartialOrd)] + struct ComparableJoin<'a> { + /// Left input + pub left: &'a Arc, + /// Right input + pub right: &'a Arc, + /// Equijoin clause expressed as pairs of (left, right) join expressions + pub on: &'a Vec<(Expr, Expr)>, + /// Filters applied during join (non-equi conditions) + pub filter: &'a Option, + /// Join type + pub join_type: &'a JoinType, + /// Join constraint + pub join_constraint: &'a JoinConstraint, + /// If null_equals_null is true, null == null else null != null + pub null_equals_null: &'a bool, + } + let comparable_self = ComparableJoin { + left: &self.left, + right: &self.right, + on: &self.on, + filter: &self.filter, + join_type: &self.join_type, + join_constraint: &self.join_constraint, + null_equals_null: &self.null_equals_null, + }; + let comparable_other = ComparableJoin { + left: &other.left, + right: &other.right, + on: &other.on, + filter: &other.filter, + join_type: &other.join_type, + join_constraint: &other.join_constraint, + null_equals_null: &other.null_equals_null, + }; + comparable_self.partial_cmp(&comparable_other) + } +} + /// Subquery -#[derive(Clone, PartialEq, Eq, Hash)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash)] pub struct Subquery { /// The subquery pub subquery: Arc, @@ -3050,7 +3287,7 @@ impl Debug for Subquery { /// See [`Partitioning`] for more details on partitioning /// /// [`Partitioning`]: https://docs.rs/datafusion/latest/datafusion/physical_expr/enum.Partitioning.html# -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub enum Partitioning { /// Allocate batches using a round-robin algorithm and the specified number of partitions RoundRobinBatch(usize), @@ -3084,6 +3321,47 @@ pub struct Unnest { pub options: UnnestOptions, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for Unnest { + fn partial_cmp(&self, other: &Self) -> Option { + #[derive(PartialEq, PartialOrd)] + struct ComparableUnnest<'a> { + /// The incoming logical plan + pub input: &'a Arc, + /// Columns to run unnest on, can be a list of (List/Struct) columns + pub exec_columns: &'a Vec, + /// refer to the indices(in the input schema) of columns + /// that have type list to run unnest on + pub list_type_columns: &'a Vec, + /// refer to the indices (in the input schema) of columns + /// that have type struct to run unnest on + pub struct_type_columns: &'a Vec, + /// Having items aligned with the output columns + /// representing which column in the input schema each output column depends on + pub dependency_indices: &'a Vec, + /// Options + pub options: &'a UnnestOptions, + } + let comparable_self = ComparableUnnest { + input: &self.input, + exec_columns: &self.exec_columns, + list_type_columns: &self.list_type_columns, + struct_type_columns: &self.struct_type_columns, + dependency_indices: &self.dependency_indices, + options: &self.options, + }; + let comparable_other = ComparableUnnest { + input: &other.input, + exec_columns: &other.exec_columns, + list_type_columns: &other.list_type_columns, + struct_type_columns: &other.struct_type_columns, + dependency_indices: &other.dependency_indices, + options: &other.options, + }; + comparable_self.partial_cmp(&comparable_other) + } +} + #[cfg(test)] mod tests { @@ -3696,4 +3974,40 @@ digraph { let actual = format!("{}", plan.display_indent()); assert_eq!(expected.to_string(), actual) } + + #[test] + fn test_plan_partial_ord() { + let empty_relation = LogicalPlan::EmptyRelation(EmptyRelation { + produce_one_row: false, + schema: Arc::new(DFSchema::empty()), + }); + + let describe_table = LogicalPlan::DescribeTable(DescribeTable { + schema: Arc::new(Schema::new(vec![Field::new( + "foo", + DataType::Int32, + false, + )])), + output_schema: DFSchemaRef::new(DFSchema::empty()), + }); + + let describe_table_clone = LogicalPlan::DescribeTable(DescribeTable { + schema: Arc::new(Schema::new(vec![Field::new( + "foo", + DataType::Int32, + false, + )])), + output_schema: DFSchemaRef::new(DFSchema::empty()), + }); + + assert_eq!( + empty_relation.partial_cmp(&describe_table), + Some(Ordering::Less) + ); + assert_eq!( + describe_table.partial_cmp(&empty_relation), + Some(Ordering::Greater) + ); + assert_eq!(describe_table.partial_cmp(&describe_table_clone), None); + } } diff --git a/datafusion/expr/src/logical_plan/statement.rs b/datafusion/expr/src/logical_plan/statement.rs index 21ff8dbd8eec..ed06375157c9 100644 --- a/datafusion/expr/src/logical_plan/statement.rs +++ b/datafusion/expr/src/logical_plan/statement.rs @@ -16,6 +16,7 @@ // under the License. use datafusion_common::DFSchemaRef; +use std::cmp::Ordering; use std::fmt::{self, Display}; /// Various types of Statements. @@ -25,7 +26,7 @@ use std::fmt::{self, Display}; /// While DataFusion does not offer support transactions, it provides /// [`LogicalPlan`](crate::LogicalPlan) support to assist building /// database systems using DataFusion -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub enum Statement { // Begin a transaction TransactionStart(TransactionStart), @@ -92,21 +93,21 @@ impl Statement { } /// Indicates if a transaction was committed or aborted -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub enum TransactionConclusion { Commit, Rollback, } /// Indicates if this transaction is allowed to write -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub enum TransactionAccessMode { ReadOnly, ReadWrite, } /// Indicates ANSI transaction isolation level -#[derive(Clone, PartialEq, Eq, Hash, Debug)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash, Debug)] pub enum TransactionIsolationLevel { ReadUncommitted, ReadCommitted, @@ -125,6 +126,18 @@ pub struct TransactionStart { pub schema: DFSchemaRef, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for TransactionStart { + fn partial_cmp(&self, other: &Self) -> Option { + match self.access_mode.partial_cmp(&other.access_mode) { + Some(Ordering::Equal) => { + self.isolation_level.partial_cmp(&other.isolation_level) + } + cmp => cmp, + } + } +} + /// Indicator that any current transaction should be terminated #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct TransactionEnd { @@ -136,6 +149,16 @@ pub struct TransactionEnd { pub schema: DFSchemaRef, } +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for TransactionEnd { + fn partial_cmp(&self, other: &Self) -> Option { + match self.conclusion.partial_cmp(&other.conclusion) { + Some(Ordering::Equal) => self.chain.partial_cmp(&other.chain), + cmp => cmp, + } + } +} + /// Set a Variable's value -- value in /// [`ConfigOptions`](datafusion_common::config::ConfigOptions) #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -147,3 +170,13 @@ pub struct SetVariable { /// Dummy schema pub schema: DFSchemaRef, } + +// Manual implementation needed because of `schema` field. Comparison excludes this field. +impl PartialOrd for SetVariable { + fn partial_cmp(&self, other: &Self) -> Option { + match self.variable.partial_cmp(&other.value) { + Some(Ordering::Equal) => self.value.partial_cmp(&other.value), + cmp => cmp, + } + } +} diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index 598607ae5e61..938e1181d85d 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -25,6 +25,7 @@ use arrow::datatypes::DataType; use datafusion_common::{not_impl_err, ExprSchema, Result}; use datafusion_expr_common::interval_arithmetic::Interval; use std::any::Any; +use std::cmp::Ordering; use std::fmt::Debug; use std::hash::{DefaultHasher, Hash, Hasher}; use std::sync::Arc; @@ -62,6 +63,16 @@ impl PartialEq for ScalarUDF { } } +// Manual implementation based on `ScalarUDFImpl::equals` +impl PartialOrd for ScalarUDF { + fn partial_cmp(&self, other: &Self) -> Option { + match self.name().partial_cmp(other.name()) { + Some(Ordering::Equal) => self.signature().partial_cmp(other.signature()), + cmp => cmp, + } + } +} + impl Eq for ScalarUDF {} impl Hash for ScalarUDF { diff --git a/datafusion/expr/src/window_frame.rs b/datafusion/expr/src/window_frame.rs index 6c935cdcd121..b2e8268aa332 100644 --- a/datafusion/expr/src/window_frame.rs +++ b/datafusion/expr/src/window_frame.rs @@ -36,7 +36,7 @@ use sqlparser::parser::ParserError::ParserError; /// window function. The ending frame boundary can be omitted if the `BETWEEN` /// and `AND` keywords that surround the starting frame boundary are also omitted, /// in which case the ending frame boundary defaults to `CURRENT ROW`. -#[derive(Clone, PartialEq, Eq, Hash)] +#[derive(Clone, PartialEq, Eq, PartialOrd, Hash)] pub struct WindowFrame { /// Frame type - either `ROWS`, `RANGE` or `GROUPS` pub units: WindowFrameUnits, @@ -300,7 +300,7 @@ impl WindowFrame { /// 4. `` FOLLOWING /// 5. UNBOUNDED FOLLOWING /// -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub enum WindowFrameBound { /// 1. UNBOUNDED PRECEDING /// The frame boundary is the first row in the partition. diff --git a/datafusion/optimizer/src/analyzer/subquery.rs b/datafusion/optimizer/src/analyzer/subquery.rs index 996dc238853e..c771f31a58b2 100644 --- a/datafusion/optimizer/src/analyzer/subquery.rs +++ b/datafusion/optimizer/src/analyzer/subquery.rs @@ -336,6 +336,7 @@ fn check_mixed_out_refer_in_window(window: &Window) -> Result<()> { #[cfg(test)] mod test { + use std::cmp::Ordering; use std::sync::Arc; use datafusion_common::{DFSchema, DFSchemaRef}; @@ -348,6 +349,12 @@ mod test { empty_schema: DFSchemaRef, } + impl PartialOrd for MockUserDefinedLogicalPlan { + fn partial_cmp(&self, _other: &Self) -> Option { + None + } + } + impl UserDefinedLogicalNodeCore for MockUserDefinedLogicalPlan { fn name(&self) -> &str { "MockUserDefinedLogicalPlan" diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index 65db164c6e55..8c61d45cc883 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -785,6 +785,7 @@ fn is_projection_unnecessary(input: &LogicalPlan, proj_exprs: &[Expr]) -> Result #[cfg(test)] mod tests { + use std::cmp::Ordering; use std::collections::HashMap; use std::fmt::Formatter; use std::ops::Add; @@ -844,6 +845,16 @@ mod tests { } } + // Manual implementation needed because of `schema` field. Comparison excludes this field. + impl PartialOrd for NoOpUserDefined { + fn partial_cmp(&self, other: &Self) -> Option { + match self.exprs.partial_cmp(&other.exprs) { + Some(Ordering::Equal) => self.input.partial_cmp(&other.input), + cmp => cmp, + } + } + } + impl UserDefinedLogicalNodeCore for NoOpUserDefined { fn name(&self) -> &str { "NoOpUserDefined" @@ -910,6 +921,23 @@ mod tests { } } + // Manual implementation needed because of `schema` field. Comparison excludes this field. + impl PartialOrd for UserDefinedCrossJoin { + fn partial_cmp(&self, other: &Self) -> Option { + match self.exprs.partial_cmp(&other.exprs) { + Some(Ordering::Equal) => { + match self.left_child.partial_cmp(&other.left_child) { + Some(Ordering::Equal) => { + self.right_child.partial_cmp(&other.right_child) + } + cmp => cmp, + } + } + cmp => cmp, + } + } + } + impl UserDefinedLogicalNodeCore for UserDefinedCrossJoin { fn name(&self) -> &str { "UserDefinedCrossJoin" diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 6f0a64b85cb6..a306ff7d2dbc 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -1195,6 +1195,7 @@ fn contain(e: &Expr, check_map: &HashMap) -> bool { #[cfg(test)] mod tests { use std::any::Any; + use std::cmp::Ordering; use std::fmt::{Debug, Formatter}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; @@ -1451,6 +1452,13 @@ mod tests { schema: DFSchemaRef, } + // Manual implementation needed because of `schema` field. Comparison excludes this field. + impl PartialOrd for NoopPlan { + fn partial_cmp(&self, other: &Self) -> Option { + self.input.partial_cmp(&other.input) + } + } + impl UserDefinedLogicalNodeCore for NoopPlan { fn name(&self) -> &str { "NoopPlan" diff --git a/datafusion/optimizer/src/test/user_defined.rs b/datafusion/optimizer/src/test/user_defined.rs index d040fa2baefd..814cd0c0cd0a 100644 --- a/datafusion/optimizer/src/test/user_defined.rs +++ b/datafusion/optimizer/src/test/user_defined.rs @@ -33,7 +33,7 @@ pub fn new(input: LogicalPlan) -> LogicalPlan { LogicalPlan::Extension(Extension { node }) } -#[derive(PartialEq, Eq, Hash)] +#[derive(PartialEq, Eq, PartialOrd, Hash)] struct TestUserDefinedPlanNode { input: LogicalPlan, } diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 71c8dbe6ec50..133c38ab8cb0 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -978,7 +978,7 @@ pub mod proto { } } -#[derive(PartialEq, Eq, Hash)] +#[derive(PartialEq, Eq, PartialOrd, Hash)] struct TopKPlanNode { k: usize, input: LogicalPlan, diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index cc353ab36d97..ea85092f7a6c 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -21,6 +21,7 @@ use datafusion::scalar::ScalarValue; use datafusion_substrait::logical_plan::{ consumer::from_substrait_plan, producer::to_substrait_plan, }; +use std::cmp::Ordering; use datafusion::arrow::datatypes::{DataType, Field, IntervalUnit, Schema, TimeUnit}; use datafusion::common::{not_impl_err, plan_err, DFSchema, DFSchemaRef}; @@ -84,6 +85,17 @@ struct MockUserDefinedLogicalPlan { empty_schema: DFSchemaRef, } +// `PartialOrd` needed for `UserDefinedLogicalNodeCore`, manual implementation necessary due to +// the `empty_schema` field. +impl PartialOrd for MockUserDefinedLogicalPlan { + fn partial_cmp(&self, other: &Self) -> Option { + match self.validation_bytes.partial_cmp(&other.validation_bytes) { + Some(Ordering::Equal) => self.inputs.partial_cmp(&other.inputs), + cmp => cmp, + } + } +} + impl UserDefinedLogicalNode for MockUserDefinedLogicalPlan { fn as_any(&self) -> &dyn std::any::Any { self @@ -132,6 +144,10 @@ impl UserDefinedLogicalNode for MockUserDefinedLogicalPlan { fn dyn_eq(&self, _: &dyn UserDefinedLogicalNode) -> bool { unimplemented!() } + + fn dyn_ord(&self, _: &dyn UserDefinedLogicalNode) -> Option { + unimplemented!() + } } impl MockUserDefinedLogicalPlan { From e1b992a7878e78e8a63b7e24425c665727bda493 Mon Sep 17 00:00:00 2001 From: jcsherin Date: Sat, 21 Sep 2024 16:53:08 +0530 Subject: [PATCH 05/25] Add `field` trait method to `WindowUDFImpl`, remove `return_type`/`nullable` (#12374) * Adds new library `functions-window-common` * Adds `FieldArgs` struct for field of final result * Adds `field` method to `WindowUDFImpl` trait * Minor: fixes formatting * Fixes: udwf doc test * Fixes: implements missing trait items * Updates `datafusion-cli` dependencies * Fixes: formatting of `Cargo.toml` files * Fixes: implementation of `field` in udwf example * Pass `FieldArgs` argument to `field` * Use `field` in place of `return_type` for udwf * Update `field` in udwf implementations * Fixes: implementation of `field` in udwf example * Revert unrelated change * Mark `return_type` for udwf as unreachable * Delete code * Uses schema name of udwf to construct `FieldArgs` * Adds deprecated notice to `return_type` trait method * Add doc comments to `field` trait method * Reify `input_types` when creating the udwf window expression * Rename name field to `schema_name` in `FieldArgs` * Make `FieldArgs` opaque * Minor refactor * Removes `nullable` trait method from `WindowUDFImpl` * Add doc comments * Rename to `WindowUDFResultArgs` * Minor: fixes formatting * Copy edits for doc comments * Renames field to `function_name` * Rename struct to `WindowUDFFieldArgs` * Add comments for unreachable code * Copy edit for `WindowUDFImpl::field` trait method * Renames module * Fix warning: unused doc comment * Minor: rename bindings * Minor refactor * Minor: copy edit * Fixes: use `Expr::qualified_name` for window function name * Fixes: apply previous fix to `Expr::nullable` * Refactor: reuse type coercion for window functions * Fixes: clippy errors * Adds name parameter to `WindowFunctionDefinition::return_type` * Removes `return_type` field from `SimpleWindowUDF` * Add doc comment for helper method * Rewrite doc comments * Minor: remove empty comment * Remove `WindowUDFImpl::return_type` * Fixes doc test --- Cargo.toml | 2 + datafusion-cli/Cargo.lock | 10 ++ datafusion-examples/examples/advanced_udwf.rs | 11 +- .../examples/simplify_udwf_expression.rs | 12 +- datafusion/core/Cargo.toml | 1 + .../user_defined_window_functions.rs | 14 +- datafusion/expr/Cargo.toml | 1 + datafusion/expr/src/expr.rs | 32 ++-- datafusion/expr/src/expr_fn.rs | 13 +- datafusion/expr/src/expr_schema.rs | 140 +++++++++++------- datafusion/expr/src/function.rs | 2 + datafusion/expr/src/udwf.rs | 68 ++++----- datafusion/functions-window-common/Cargo.toml | 41 +++++ datafusion/functions-window-common/README.md | 26 ++++ .../functions-window-common/src/field.rs | 64 ++++++++ datafusion/functions-window-common/src/lib.rs | 21 +++ datafusion/functions-window/Cargo.toml | 1 + datafusion/functions-window/src/row_number.rs | 11 +- datafusion/optimizer/Cargo.toml | 1 + .../simplify_expressions/expr_simplifier.rs | 9 +- datafusion/physical-plan/Cargo.toml | 1 + datafusion/physical-plan/src/windows/mod.rs | 19 +-- datafusion/proto/Cargo.toml | 1 + .../tests/cases/roundtrip_logical_plan.rs | 24 +-- 24 files changed, 357 insertions(+), 168 deletions(-) create mode 100644 datafusion/functions-window-common/Cargo.toml create mode 100644 datafusion/functions-window-common/README.md create mode 100644 datafusion/functions-window-common/src/field.rs create mode 100644 datafusion/functions-window-common/src/lib.rs diff --git a/Cargo.toml b/Cargo.toml index c80297a1f5a6..e8cd52315a51 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -31,6 +31,7 @@ members = [ "datafusion/functions-aggregate-common", "datafusion/functions-nested", "datafusion/functions-window", + "datafusion/functions-window-common", "datafusion/optimizer", "datafusion/physical-expr", "datafusion/physical-expr-common", @@ -103,6 +104,7 @@ datafusion-functions-aggregate = { path = "datafusion/functions-aggregate", vers datafusion-functions-aggregate-common = { path = "datafusion/functions-aggregate-common", version = "42.0.0" } datafusion-functions-nested = { path = "datafusion/functions-nested", version = "42.0.0" } datafusion-functions-window = { path = "datafusion/functions-window", version = "42.0.0" } +datafusion-functions-window-common = { path = "datafusion/functions-window-common", version = "42.0.0" } datafusion-optimizer = { path = "datafusion/optimizer", version = "42.0.0", default-features = false } datafusion-physical-expr = { path = "datafusion/physical-expr", version = "42.0.0", default-features = false } datafusion-physical-expr-common = { path = "datafusion/physical-expr-common", version = "42.0.0", default-features = false } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 15a2b14ec35f..fbe7d5c04b9b 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1343,6 +1343,7 @@ dependencies = [ "datafusion-common", "datafusion-expr-common", "datafusion-functions-aggregate-common", + "datafusion-functions-window-common", "datafusion-physical-expr-common", "paste", "serde_json", @@ -1443,10 +1444,18 @@ version = "42.0.0" dependencies = [ "datafusion-common", "datafusion-expr", + "datafusion-functions-window-common", "datafusion-physical-expr-common", "log", ] +[[package]] +name = "datafusion-functions-window-common" +version = "42.0.0" +dependencies = [ + "datafusion-common", +] + [[package]] name = "datafusion-optimizer" version = "42.0.0" @@ -1537,6 +1546,7 @@ dependencies = [ "datafusion-expr", "datafusion-functions-aggregate", "datafusion-functions-aggregate-common", + "datafusion-functions-window-common", "datafusion-physical-expr", "datafusion-physical-expr-common", "futures", diff --git a/datafusion-examples/examples/advanced_udwf.rs b/datafusion-examples/examples/advanced_udwf.rs index ec0318a561b9..fd1b84070cf6 100644 --- a/datafusion-examples/examples/advanced_udwf.rs +++ b/datafusion-examples/examples/advanced_udwf.rs @@ -22,9 +22,11 @@ use arrow::{ array::{ArrayRef, AsArray, Float64Array}, datatypes::Float64Type, }; +use arrow_schema::Field; use datafusion::error::Result; use datafusion::prelude::*; use datafusion_common::ScalarValue; +use datafusion_expr::function::WindowUDFFieldArgs; use datafusion_expr::{ PartitionEvaluator, Signature, WindowFrame, WindowUDF, WindowUDFImpl, }; @@ -70,16 +72,15 @@ impl WindowUDFImpl for SmoothItUdf { &self.signature } - /// What is the type of value that will be returned by this function. - fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(DataType::Float64) - } - /// Create a `PartitionEvaluator` to evaluate this function on a new /// partition. fn partition_evaluator(&self) -> Result> { Ok(Box::new(MyPartitionEvaluator::new())) } + + fn field(&self, field_args: WindowUDFFieldArgs) -> Result { + Ok(Field::new(field_args.name(), DataType::Float64, true)) + } } /// This implements the lowest level evaluation for a window function diff --git a/datafusion-examples/examples/simplify_udwf_expression.rs b/datafusion-examples/examples/simplify_udwf_expression.rs index a17e45dba2a3..1ff629eef196 100644 --- a/datafusion-examples/examples/simplify_udwf_expression.rs +++ b/datafusion-examples/examples/simplify_udwf_expression.rs @@ -17,12 +17,12 @@ use std::any::Any; -use arrow_schema::DataType; +use arrow_schema::{DataType, Field}; use datafusion::execution::context::SessionContext; use datafusion::functions_aggregate::average::avg_udaf; use datafusion::{error::Result, execution::options::CsvReadOptions}; -use datafusion_expr::function::WindowFunctionSimplification; +use datafusion_expr::function::{WindowFunctionSimplification, WindowUDFFieldArgs}; use datafusion_expr::{ expr::WindowFunction, simplify::SimplifyInfo, Expr, PartitionEvaluator, Signature, Volatility, WindowUDF, WindowUDFImpl, @@ -60,10 +60,6 @@ impl WindowUDFImpl for SimplifySmoothItUdf { &self.signature } - fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(DataType::Float64) - } - fn partition_evaluator(&self) -> Result> { todo!() } @@ -84,6 +80,10 @@ impl WindowUDFImpl for SimplifySmoothItUdf { Some(Box::new(simplify)) } + + fn field(&self, field_args: WindowUDFFieldArgs) -> Result { + Ok(Field::new(field_args.name(), DataType::Float64, true)) + } } // create local execution context with `cars.csv` registered as a table named `cars` diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 82a799f8585c..01ba90ee5de8 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -145,6 +145,7 @@ bigdecimal = { workspace = true } criterion = { version = "0.5", features = ["async_tokio"] } csv = "1.1.6" ctor = { workspace = true } +datafusion-functions-window-common = { workspace = true } doc-comment = { workspace = true } env_logger = { workspace = true } half = { workspace = true, default-features = true } diff --git a/datafusion/core/tests/user_defined/user_defined_window_functions.rs b/datafusion/core/tests/user_defined/user_defined_window_functions.rs index 3c607301fc98..d96bb23953ae 100644 --- a/datafusion/core/tests/user_defined/user_defined_window_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_window_functions.rs @@ -29,12 +29,13 @@ use std::{ use arrow::array::AsArray; use arrow_array::{ArrayRef, Int64Array, RecordBatch, StringArray}; -use arrow_schema::DataType; +use arrow_schema::{DataType, Field}; use datafusion::{assert_batches_eq, prelude::SessionContext}; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::{ PartitionEvaluator, Signature, Volatility, WindowUDF, WindowUDFImpl, }; +use datafusion_functions_window_common::field::WindowUDFFieldArgs; /// A query with a window function evaluated over the entire partition const UNBOUNDED_WINDOW_QUERY: &str = "SELECT x, y, val, \ @@ -522,7 +523,6 @@ impl OddCounter { #[derive(Debug, Clone)] struct SimpleWindowUDF { signature: Signature, - return_type: DataType, test_state: Arc, aliases: Vec, } @@ -531,10 +531,8 @@ impl OddCounter { fn new(test_state: Arc) -> Self { let signature = Signature::exact(vec![DataType::Float64], Volatility::Immutable); - let return_type = DataType::Int64; Self { signature, - return_type, test_state, aliases: vec!["odd_counter_alias".to_string()], } @@ -554,10 +552,6 @@ impl OddCounter { &self.signature } - fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(self.return_type.clone()) - } - fn partition_evaluator(&self) -> Result> { Ok(Box::new(OddCounter::new(Arc::clone(&self.test_state)))) } @@ -565,6 +559,10 @@ impl OddCounter { fn aliases(&self) -> &[String] { &self.aliases } + + fn field(&self, field_args: WindowUDFFieldArgs) -> Result { + Ok(Field::new(field_args.name(), DataType::Int64, true)) + } } ctx.register_udwf(WindowUDF::from(SimpleWindowUDF::new(test_state))) diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index b5d34d9a3834..55387fea22ee 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -46,6 +46,7 @@ chrono = { workspace = true } datafusion-common = { workspace = true } datafusion-expr-common = { workspace = true } datafusion-functions-aggregate-common = { workspace = true } +datafusion-functions-window-common = { workspace = true } datafusion-physical-expr-common = { workspace = true } paste = "^1.0" serde_json = { workspace = true } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 8cb759b8811e..c14132496203 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -40,6 +40,7 @@ use datafusion_common::tree_node::{ use datafusion_common::{ plan_err, Column, DFSchema, Result, ScalarValue, TableReference, }; +use datafusion_functions_window_common::field::WindowUDFFieldArgs; use sqlparser::ast::{ display_comma_separated, ExceptSelectItem, ExcludeSelectItem, IlikeSelectItem, NullTreatment, RenameSelectItem, ReplaceSelectElement, @@ -706,6 +707,7 @@ impl WindowFunctionDefinition { &self, input_expr_types: &[DataType], _input_expr_nullable: &[bool], + display_name: &str, ) -> Result { match self { WindowFunctionDefinition::BuiltInWindowFunction(fun) => { @@ -714,7 +716,9 @@ impl WindowFunctionDefinition { WindowFunctionDefinition::AggregateUDF(fun) => { fun.return_type(input_expr_types) } - WindowFunctionDefinition::WindowUDF(fun) => fun.return_type(input_expr_types), + WindowFunctionDefinition::WindowUDF(fun) => fun + .field(WindowUDFFieldArgs::new(input_expr_types, display_name)) + .map(|field| field.data_type().clone()), } } @@ -2536,10 +2540,10 @@ mod test { #[test] fn test_first_value_return_type() -> Result<()> { let fun = find_df_window_func("first_value").unwrap(); - let observed = fun.return_type(&[DataType::Utf8], &[true])?; + let observed = fun.return_type(&[DataType::Utf8], &[true], "")?; assert_eq!(DataType::Utf8, observed); - let observed = fun.return_type(&[DataType::UInt64], &[true])?; + let observed = fun.return_type(&[DataType::UInt64], &[true], "")?; assert_eq!(DataType::UInt64, observed); Ok(()) @@ -2548,10 +2552,10 @@ mod test { #[test] fn test_last_value_return_type() -> Result<()> { let fun = find_df_window_func("last_value").unwrap(); - let observed = fun.return_type(&[DataType::Utf8], &[true])?; + let observed = fun.return_type(&[DataType::Utf8], &[true], "")?; assert_eq!(DataType::Utf8, observed); - let observed = fun.return_type(&[DataType::Float64], &[true])?; + let observed = fun.return_type(&[DataType::Float64], &[true], "")?; assert_eq!(DataType::Float64, observed); Ok(()) @@ -2560,10 +2564,10 @@ mod test { #[test] fn test_lead_return_type() -> Result<()> { let fun = find_df_window_func("lead").unwrap(); - let observed = fun.return_type(&[DataType::Utf8], &[true])?; + let observed = fun.return_type(&[DataType::Utf8], &[true], "")?; assert_eq!(DataType::Utf8, observed); - let observed = fun.return_type(&[DataType::Float64], &[true])?; + let observed = fun.return_type(&[DataType::Float64], &[true], "")?; assert_eq!(DataType::Float64, observed); Ok(()) @@ -2572,10 +2576,10 @@ mod test { #[test] fn test_lag_return_type() -> Result<()> { let fun = find_df_window_func("lag").unwrap(); - let observed = fun.return_type(&[DataType::Utf8], &[true])?; + let observed = fun.return_type(&[DataType::Utf8], &[true], "")?; assert_eq!(DataType::Utf8, observed); - let observed = fun.return_type(&[DataType::Float64], &[true])?; + let observed = fun.return_type(&[DataType::Float64], &[true], "")?; assert_eq!(DataType::Float64, observed); Ok(()) @@ -2585,11 +2589,11 @@ mod test { fn test_nth_value_return_type() -> Result<()> { let fun = find_df_window_func("nth_value").unwrap(); let observed = - fun.return_type(&[DataType::Utf8, DataType::UInt64], &[true, true])?; + fun.return_type(&[DataType::Utf8, DataType::UInt64], &[true, true], "")?; assert_eq!(DataType::Utf8, observed); let observed = - fun.return_type(&[DataType::Float64, DataType::UInt64], &[true, true])?; + fun.return_type(&[DataType::Float64, DataType::UInt64], &[true, true], "")?; assert_eq!(DataType::Float64, observed); Ok(()) @@ -2598,7 +2602,7 @@ mod test { #[test] fn test_percent_rank_return_type() -> Result<()> { let fun = find_df_window_func("percent_rank").unwrap(); - let observed = fun.return_type(&[], &[])?; + let observed = fun.return_type(&[], &[], "")?; assert_eq!(DataType::Float64, observed); Ok(()) @@ -2607,7 +2611,7 @@ mod test { #[test] fn test_cume_dist_return_type() -> Result<()> { let fun = find_df_window_func("cume_dist").unwrap(); - let observed = fun.return_type(&[], &[])?; + let observed = fun.return_type(&[], &[], "")?; assert_eq!(DataType::Float64, observed); Ok(()) @@ -2616,7 +2620,7 @@ mod test { #[test] fn test_ntile_return_type() -> Result<()> { let fun = find_df_window_func("ntile").unwrap(); - let observed = fun.return_type(&[DataType::Int16], &[true])?; + let observed = fun.return_type(&[DataType::Int16], &[true], "")?; assert_eq!(DataType::UInt64, observed); Ok(()) diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 5fd3177bc27b..2975e36488dc 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -38,6 +38,7 @@ use arrow::compute::kernels::cast_utils::{ }; use arrow::datatypes::{DataType, Field}; use datafusion_common::{plan_err, Column, Result, ScalarValue, TableReference}; +use datafusion_functions_window_common::field::WindowUDFFieldArgs; use sqlparser::ast::NullTreatment; use std::any::Any; use std::fmt::Debug; @@ -657,13 +658,17 @@ impl WindowUDFImpl for SimpleWindowUDF { &self.signature } - fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(self.return_type.clone()) - } - fn partition_evaluator(&self) -> Result> { (self.partition_evaluator_factory)() } + + fn field(&self, field_args: WindowUDFFieldArgs) -> Result { + Ok(Field::new( + field_args.name(), + self.return_type.clone(), + true, + )) + } } pub fn interval_year_month_lit(value: &str) -> Expr { diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 598d172d30a0..f40ac409dd43 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -31,6 +31,7 @@ use datafusion_common::{ not_impl_err, plan_datafusion_err, plan_err, Column, ExprSchema, Result, TableReference, }; +use datafusion_functions_window_common::field::WindowUDFFieldArgs; use std::collections::HashMap; use std::sync::Arc; @@ -166,49 +167,9 @@ impl ExprSchemable for Expr { // expressiveness of `TypeSignature`), then infer return type Ok(func.return_type_from_exprs(args, schema, &arg_data_types)?) } - Expr::WindowFunction(WindowFunction { fun, args, .. }) => { - let data_types = args - .iter() - .map(|e| e.get_type(schema)) - .collect::>>()?; - let nullability = args - .iter() - .map(|e| e.nullable(schema)) - .collect::>>()?; - match fun { - WindowFunctionDefinition::AggregateUDF(udf) => { - let new_types = data_types_with_aggregate_udf(&data_types, udf) - .map_err(|err| { - plan_datafusion_err!( - "{} {}", - err, - utils::generate_signature_error_msg( - fun.name(), - fun.signature(), - &data_types - ) - ) - })?; - Ok(fun.return_type(&new_types, &nullability)?) - } - WindowFunctionDefinition::WindowUDF(udwf) => { - let new_types = data_types_with_window_udf(&data_types, udwf) - .map_err(|err| { - plan_datafusion_err!( - "{} {}", - err, - utils::generate_signature_error_msg( - fun.name(), - fun.signature(), - &data_types - ) - ) - })?; - Ok(fun.return_type(&new_types, &nullability)?) - } - _ => fun.return_type(&data_types, &nullability), - } - } + Expr::WindowFunction(window_function) => self + .data_type_and_nullable_with_window_function(schema, window_function) + .map(|(return_type, _)| return_type), Expr::AggregateFunction(AggregateFunction { func, args, .. }) => { let data_types = args .iter() @@ -340,20 +301,12 @@ impl ExprSchemable for Expr { Expr::AggregateFunction(AggregateFunction { func, .. }) => { Ok(func.is_nullable()) } - Expr::WindowFunction(WindowFunction { fun, .. }) => match fun { - WindowFunctionDefinition::BuiltInWindowFunction(func) => { - if func.name() == "RANK" - || func.name() == "NTILE" - || func.name() == "CUME_DIST" - { - Ok(false) - } else { - Ok(true) - } - } - WindowFunctionDefinition::AggregateUDF(func) => Ok(func.is_nullable()), - WindowFunctionDefinition::WindowUDF(udwf) => Ok(udwf.nullable()), - }, + Expr::WindowFunction(window_function) => self + .data_type_and_nullable_with_window_function( + input_schema, + window_function, + ) + .map(|(_, nullable)| nullable), Expr::ScalarVariable(_, _) | Expr::TryCast { .. } | Expr::Unnest(_) @@ -450,6 +403,9 @@ impl ExprSchemable for Expr { let right = right.data_type_and_nullable(schema)?; Ok((get_result_type(&left.0, op, &right.0)?, left.1 || right.1)) } + Expr::WindowFunction(window_function) => { + self.data_type_and_nullable_with_window_function(schema, window_function) + } _ => Ok((self.get_type(schema)?, self.nullable(schema)?)), } } @@ -499,6 +455,76 @@ impl ExprSchemable for Expr { } } +impl Expr { + /// Common method for window functions that applies type coercion + /// to all arguments of the window function to check if it matches + /// its signature. + /// + /// If successful, this method returns the data type and + /// nullability of the window function's result. + /// + /// Otherwise, returns an error if there's a type mismatch between + /// the window function's signature and the provided arguments. + fn data_type_and_nullable_with_window_function( + &self, + schema: &dyn ExprSchema, + window_function: &WindowFunction, + ) -> Result<(DataType, bool)> { + let WindowFunction { fun, args, .. } = window_function; + + let data_types = args + .iter() + .map(|e| e.get_type(schema)) + .collect::>>()?; + match fun { + WindowFunctionDefinition::BuiltInWindowFunction(window_fun) => { + let return_type = window_fun.return_type(&data_types)?; + let nullable = + !["RANK", "NTILE", "CUME_DIST"].contains(&window_fun.name()); + Ok((return_type, nullable)) + } + WindowFunctionDefinition::AggregateUDF(udaf) => { + let new_types = data_types_with_aggregate_udf(&data_types, udaf) + .map_err(|err| { + plan_datafusion_err!( + "{} {}", + err, + utils::generate_signature_error_msg( + fun.name(), + fun.signature(), + &data_types + ) + ) + })?; + + let return_type = udaf.return_type(&new_types)?; + let nullable = udaf.is_nullable(); + + Ok((return_type, nullable)) + } + WindowFunctionDefinition::WindowUDF(udwf) => { + let new_types = + data_types_with_window_udf(&data_types, udwf).map_err(|err| { + plan_datafusion_err!( + "{} {}", + err, + utils::generate_signature_error_msg( + fun.name(), + fun.signature(), + &data_types + ) + ) + })?; + let (_, function_name) = self.qualified_name(); + let field_args = WindowUDFFieldArgs::new(&new_types, &function_name); + + udwf.field(field_args) + .map(|field| (field.data_type().clone(), field.is_nullable())) + } + } + } +} + /// cast subquery in InSubquery/ScalarSubquery to a given type. pub fn cast_subquery(subquery: Subquery, cast_to_type: &DataType) -> Result { if subquery.subquery.schema().field(0).data_type() == cast_to_type { diff --git a/datafusion/expr/src/function.rs b/datafusion/expr/src/function.rs index cd7a0c8aa918..9814d16ddfa3 100644 --- a/datafusion/expr/src/function.rs +++ b/datafusion/expr/src/function.rs @@ -27,6 +27,8 @@ pub use datafusion_functions_aggregate_common::accumulator::{ AccumulatorArgs, AccumulatorFactoryFunction, StateFieldsArgs, }; +pub use datafusion_functions_window_common::field::WindowUDFFieldArgs; + #[derive(Debug, Clone, Copy)] pub enum Hint { /// Indicates the argument needs to be padded if it is scalar diff --git a/datafusion/expr/src/udwf.rs b/datafusion/expr/src/udwf.rs index fc4432ffdf90..7cc57523a14d 100644 --- a/datafusion/expr/src/udwf.rs +++ b/datafusion/expr/src/udwf.rs @@ -18,7 +18,6 @@ //! [`WindowUDF`]: User Defined Window Functions use arrow::compute::SortOptions; -use arrow::datatypes::DataType; use std::cmp::Ordering; use std::hash::{DefaultHasher, Hash, Hasher}; use std::{ @@ -27,7 +26,10 @@ use std::{ sync::Arc, }; +use arrow::datatypes::{DataType, Field}; + use datafusion_common::{not_impl_err, Result}; +use datafusion_functions_window_common::field::WindowUDFFieldArgs; use crate::expr::WindowFunction; use crate::{ @@ -139,13 +141,6 @@ impl WindowUDF { self.inner.signature() } - /// Return the type of the function given its input types - /// - /// See [`WindowUDFImpl::return_type`] for more details. - pub fn return_type(&self, args: &[DataType]) -> Result { - self.inner.return_type(args) - } - /// Do the function rewrite /// /// See [`WindowUDFImpl::simplify`] for more details. @@ -158,11 +153,11 @@ impl WindowUDF { self.inner.partition_evaluator() } - /// Returns if column values are nullable for this window function. + /// Returns the field of the final result of evaluating this window function. /// - /// See [`WindowUDFImpl::nullable`] for more details. - pub fn nullable(&self) -> bool { - self.inner.nullable() + /// See [`WindowUDFImpl::field`] for more details. + pub fn field(&self, field_args: WindowUDFFieldArgs) -> Result { + self.inner.field(field_args) } /// Returns custom result ordering introduced by this window function @@ -201,10 +196,11 @@ where /// # Basic Example /// ``` /// # use std::any::Any; -/// # use arrow::datatypes::DataType; +/// # use arrow::datatypes::{DataType, Field}; /// # use datafusion_common::{DataFusionError, plan_err, Result}; /// # use datafusion_expr::{col, Signature, Volatility, PartitionEvaluator, WindowFrame, ExprFunctionExt}; /// # use datafusion_expr::{WindowUDFImpl, WindowUDF}; +/// use datafusion_functions_window_common::field::WindowUDFFieldArgs; /// #[derive(Debug, Clone)] /// struct SmoothIt { /// signature: Signature @@ -223,14 +219,15 @@ where /// fn as_any(&self) -> &dyn Any { self } /// fn name(&self) -> &str { "smooth_it" } /// fn signature(&self) -> &Signature { &self.signature } -/// fn return_type(&self, args: &[DataType]) -> Result { -/// if !matches!(args.get(0), Some(&DataType::Int32)) { -/// return plan_err!("smooth_it only accepts Int32 arguments"); -/// } -/// Ok(DataType::Int32) -/// } /// // The actual implementation would add one to the argument /// fn partition_evaluator(&self) -> Result> { unimplemented!() } +/// fn field(&self, field_args: WindowUDFFieldArgs) -> Result { +/// if let Some(DataType::Int32) = field_args.get_input_type(0) { +/// Ok(Field::new(field_args.name(), DataType::Int32, false)) +/// } else { +/// plan_err!("smooth_it only accepts Int32 arguments") +/// } +/// } /// } /// /// // Create a new WindowUDF from the implementation @@ -259,10 +256,6 @@ pub trait WindowUDFImpl: Debug + Send + Sync { /// types are accepted and the function's Volatility. fn signature(&self) -> &Signature; - /// What [`DataType`] will be returned by this function, given the types of - /// the arguments - fn return_type(&self, arg_types: &[DataType]) -> Result; - /// Invoke the function, returning the [`PartitionEvaluator`] instance fn partition_evaluator(&self) -> Result>; @@ -324,14 +317,8 @@ pub trait WindowUDFImpl: Debug + Send + Sync { hasher.finish() } - /// Allows customizing nullable of column for this window UDF. - /// - /// By default, the final result of evaluating the window UDF is - /// allowed to have null values. But if that is not the case then - /// it can be customized in the window UDF implementation. - fn nullable(&self) -> bool { - true - } + /// The [`Field`] of the final result of evaluating this window function. + fn field(&self, field_args: WindowUDFFieldArgs) -> Result; /// Allows the window UDF to define a custom result ordering. /// @@ -414,10 +401,6 @@ impl WindowUDFImpl for AliasedWindowUDFImpl { self.inner.signature() } - fn return_type(&self, arg_types: &[DataType]) -> Result { - self.inner.return_type(arg_types) - } - fn partition_evaluator(&self) -> Result> { self.inner.partition_evaluator() } @@ -445,8 +428,8 @@ impl WindowUDFImpl for AliasedWindowUDFImpl { hasher.finish() } - fn nullable(&self) -> bool { - self.inner.nullable() + fn field(&self, field_args: WindowUDFFieldArgs) -> Result { + self.inner.field(field_args) } fn sort_options(&self) -> Option { @@ -461,9 +444,10 @@ impl WindowUDFImpl for AliasedWindowUDFImpl { #[cfg(test)] mod test { use crate::{PartitionEvaluator, WindowUDF, WindowUDFImpl}; - use arrow::datatypes::DataType; + use arrow::datatypes::{DataType, Field}; use datafusion_common::Result; use datafusion_expr_common::signature::{Signature, Volatility}; + use datafusion_functions_window_common::field::WindowUDFFieldArgs; use std::any::Any; use std::cmp::Ordering; @@ -495,10 +479,10 @@ mod test { fn signature(&self) -> &Signature { &self.signature } - fn return_type(&self, _args: &[DataType]) -> Result { + fn partition_evaluator(&self) -> Result> { unimplemented!() } - fn partition_evaluator(&self) -> Result> { + fn field(&self, _field_args: WindowUDFFieldArgs) -> Result { unimplemented!() } } @@ -531,10 +515,10 @@ mod test { fn signature(&self) -> &Signature { &self.signature } - fn return_type(&self, _args: &[DataType]) -> Result { + fn partition_evaluator(&self) -> Result> { unimplemented!() } - fn partition_evaluator(&self) -> Result> { + fn field(&self, _field_args: WindowUDFFieldArgs) -> Result { unimplemented!() } } diff --git a/datafusion/functions-window-common/Cargo.toml b/datafusion/functions-window-common/Cargo.toml new file mode 100644 index 000000000000..98b6f8c6dba5 --- /dev/null +++ b/datafusion/functions-window-common/Cargo.toml @@ -0,0 +1,41 @@ +# 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. + +[package] +name = "datafusion-functions-window-common" +description = "Common functions for implementing user-defined window functions for the DataFusion query engine" +keywords = ["datafusion", "logical", "plan", "expressions"] +readme = "README.md" +authors = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +license = { workspace = true } +repository = { workspace = true } +rust-version = { workspace = true } +version = { workspace = true } + +[lints] +workspace = true + +[lib] +name = "datafusion_functions_window_common" +path = "src/lib.rs" + +# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html + +[dependencies] +datafusion-common = { workspace = true } diff --git a/datafusion/functions-window-common/README.md b/datafusion/functions-window-common/README.md new file mode 100644 index 000000000000..de12d25f9731 --- /dev/null +++ b/datafusion/functions-window-common/README.md @@ -0,0 +1,26 @@ + + +# DataFusion Window Function Common Library + +[DataFusion][df] is an extensible query execution framework, written in Rust, that uses Apache Arrow as its in-memory format. + +This crate contains common functions for implementing user-defined window functions. + +[df]: https://crates.io/crates/datafusion diff --git a/datafusion/functions-window-common/src/field.rs b/datafusion/functions-window-common/src/field.rs new file mode 100644 index 000000000000..8011b7b0f05f --- /dev/null +++ b/datafusion/functions-window-common/src/field.rs @@ -0,0 +1,64 @@ +// 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 datafusion_common::arrow::datatypes::DataType; + +/// Metadata for defining the result field from evaluating a +/// user-defined window function. +pub struct WindowUDFFieldArgs<'a> { + /// The data types corresponding to the arguments to the + /// user-defined window function. + input_types: &'a [DataType], + /// The display name of the user-defined window function. + display_name: &'a str, +} + +impl<'a> WindowUDFFieldArgs<'a> { + /// Create an instance of [`WindowUDFFieldArgs`]. + /// + /// # Arguments + /// + /// * `input_types` - The data types corresponding to the + /// arguments to the user-defined window function. + /// * `function_name` - The qualified schema name of the + /// user-defined window function expression. + /// + pub fn new(input_types: &'a [DataType], display_name: &'a str) -> Self { + WindowUDFFieldArgs { + input_types, + display_name, + } + } + + /// Returns the data type of input expressions passed as arguments + /// to the user-defined window function. + pub fn input_types(&self) -> &[DataType] { + self.input_types + } + + /// Returns the name for the field of the final result of evaluating + /// the user-defined window function. + pub fn name(&self) -> &str { + self.display_name + } + + /// Returns `Some(DataType)` of input expression at index, otherwise + /// returns `None` if the index is out of bounds. + pub fn get_input_type(&self, index: usize) -> Option { + self.input_types.get(index).cloned() + } +} diff --git a/datafusion/functions-window-common/src/lib.rs b/datafusion/functions-window-common/src/lib.rs new file mode 100644 index 000000000000..2e4bcbbc83b9 --- /dev/null +++ b/datafusion/functions-window-common/src/lib.rs @@ -0,0 +1,21 @@ +// 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. + +//! Common user-defined window functionality for [DataFusion] +//! +//! [DataFusion]: +pub mod field; diff --git a/datafusion/functions-window/Cargo.toml b/datafusion/functions-window/Cargo.toml index 94dd421284fd..8dcec6bc964b 100644 --- a/datafusion/functions-window/Cargo.toml +++ b/datafusion/functions-window/Cargo.toml @@ -40,6 +40,7 @@ path = "src/lib.rs" [dependencies] datafusion-common = { workspace = true } datafusion-expr = { workspace = true } +datafusion-functions-window-common = { workspace = true } datafusion-physical-expr-common = { workspace = true } log = { workspace = true } diff --git a/datafusion/functions-window/src/row_number.rs b/datafusion/functions-window/src/row_number.rs index 43d2796ad7dc..7f348bf9d2a0 100644 --- a/datafusion/functions-window/src/row_number.rs +++ b/datafusion/functions-window/src/row_number.rs @@ -25,9 +25,12 @@ use datafusion_common::arrow::array::ArrayRef; use datafusion_common::arrow::array::UInt64Array; use datafusion_common::arrow::compute::SortOptions; use datafusion_common::arrow::datatypes::DataType; +use datafusion_common::arrow::datatypes::Field; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::expr::WindowFunction; use datafusion_expr::{Expr, PartitionEvaluator, Signature, Volatility, WindowUDFImpl}; +use datafusion_functions_window_common::field; +use field::WindowUDFFieldArgs; /// Create a [`WindowFunction`](Expr::WindowFunction) expression for /// `row_number` user-defined window function. @@ -84,16 +87,12 @@ impl WindowUDFImpl for RowNumber { &self.signature } - fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(DataType::UInt64) - } - fn partition_evaluator(&self) -> Result> { Ok(Box::::default()) } - fn nullable(&self) -> bool { - false + fn field(&self, field_args: WindowUDFFieldArgs) -> Result { + Ok(Field::new(field_args.name(), DataType::UInt64, false)) } fn sort_options(&self) -> Option { diff --git a/datafusion/optimizer/Cargo.toml b/datafusion/optimizer/Cargo.toml index 1a9e9630c076..337a24ffae20 100644 --- a/datafusion/optimizer/Cargo.toml +++ b/datafusion/optimizer/Cargo.toml @@ -57,5 +57,6 @@ regex-syntax = "0.8.0" arrow-buffer = { workspace = true } ctor = { workspace = true } datafusion-functions-aggregate = { workspace = true } +datafusion-functions-window-common = { workspace = true } datafusion-sql = { workspace = true } env_logger = { workspace = true } diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index fc3921d29615..a78a54a57123 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -1798,6 +1798,7 @@ mod tests { interval_arithmetic::Interval, *, }; + use datafusion_functions_window_common::field::WindowUDFFieldArgs; use std::{ collections::HashMap, ops::{BitAnd, BitOr, BitXor}, @@ -3901,10 +3902,6 @@ mod tests { unimplemented!() } - fn return_type(&self, _arg_types: &[DataType]) -> Result { - unimplemented!("not needed for tests") - } - fn simplify(&self) -> Option { if self.simplify { Some(Box::new(|_, _| Ok(col("result_column")))) @@ -3916,5 +3913,9 @@ mod tests { fn partition_evaluator(&self) -> Result> { unimplemented!("not needed for tests") } + + fn field(&self, _field_args: WindowUDFFieldArgs) -> Result { + unimplemented!("not needed for tests") + } } } diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 24387c5f15ee..c3f1b7eb0e95 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -53,6 +53,7 @@ datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-functions-aggregate = { workspace = true } datafusion-functions-aggregate-common = { workspace = true } +datafusion-functions-window-common = { workspace = true } datafusion-physical-expr = { workspace = true, default-features = true } datafusion-physical-expr-common = { workspace = true } futures = { workspace = true } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 0275cd2441a9..981a8e285166 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -51,6 +51,7 @@ mod utils; mod window_agg_exec; pub use bounded_window_agg_exec::BoundedWindowAggExec; +use datafusion_functions_window_common::field::WindowUDFFieldArgs; use datafusion_physical_expr::expressions::Column; pub use datafusion_physical_expr::window::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowExpr, @@ -73,7 +74,8 @@ pub fn schema_add_window_field( .iter() .map(|e| Arc::clone(e).as_ref().nullable(schema)) .collect::>>()?; - let window_expr_return_type = window_fn.return_type(&data_types, &nullability)?; + let window_expr_return_type = + window_fn.return_type(&data_types, &nullability, fn_name)?; let mut window_fields = schema .fields() .iter() @@ -334,13 +336,11 @@ fn create_udwf_window_expr( .map(|arg| arg.data_type(input_schema)) .collect::>()?; - // figure out the output type - let data_type = fun.return_type(&input_types)?; Ok(Arc::new(WindowUDFExpr { fun: Arc::clone(fun), args: args.to_vec(), + input_types, name, - data_type, })) } @@ -351,8 +351,8 @@ struct WindowUDFExpr { args: Vec>, /// Display name name: String, - /// result type - data_type: DataType, + /// Types of input expressions + input_types: Vec, } impl BuiltInWindowFunctionExpr for WindowUDFExpr { @@ -361,11 +361,8 @@ impl BuiltInWindowFunctionExpr for WindowUDFExpr { } fn field(&self) -> Result { - Ok(Field::new( - &self.name, - self.data_type.clone(), - self.fun.nullable(), - )) + self.fun + .field(WindowUDFFieldArgs::new(&self.input_types, &self.name)) } fn expressions(&self) -> Vec> { diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index ce40129fcf93..d65c6ccaa660 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -60,6 +60,7 @@ serde_json = { workspace = true, optional = true } [dev-dependencies] datafusion-functions = { workspace = true, default-features = true } datafusion-functions-aggregate = { workspace = true } +datafusion-functions-window-common = { workspace = true } doc-comment = { workspace = true } strum = { version = "0.26.1", features = ["derive"] } tokio = { workspace = true, features = ["rt-multi-thread"] } diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 133c38ab8cb0..1f1426164d39 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -75,6 +75,7 @@ use datafusion_functions_aggregate::expr_fn::{ }; use datafusion_functions_aggregate::kurtosis_pop::kurtosis_pop; use datafusion_functions_aggregate::string_agg::string_agg; +use datafusion_functions_window_common::field::WindowUDFFieldArgs; use datafusion_proto::bytes::{ logical_plan_from_bytes, logical_plan_from_bytes_with_extension_codec, logical_plan_to_bytes, logical_plan_to_bytes_with_extension_codec, @@ -2430,20 +2431,21 @@ fn roundtrip_window() { &self.signature } - fn return_type(&self, arg_types: &[DataType]) -> Result { - if arg_types.len() != 1 { - return plan_err!( - "dummy_udwf expects 1 argument, got {}: {:?}", - arg_types.len(), - arg_types - ); - } - Ok(arg_types[0].clone()) - } - fn partition_evaluator(&self) -> Result> { make_partition_evaluator() } + + fn field(&self, field_args: WindowUDFFieldArgs) -> Result { + if let Some(return_type) = field_args.get_input_type(0) { + Ok(Field::new(field_args.name(), return_type, true)) + } else { + plan_err!( + "dummy_udwf expects 1 argument, got {}: {:?}", + field_args.input_types().len(), + field_args.input_types() + ) + } + } } fn make_partition_evaluator() -> Result> { From 515a64ea3e2d238ead2c1230ff3985abc0ce7fca Mon Sep 17 00:00:00 2001 From: WeblWabl Date: Sat, 21 Sep 2024 06:23:26 -0500 Subject: [PATCH 06/25] feat(planner): Allowing setting sort order of parquet files without specifying the schema (#12466) * fix(planner): Allowing setting sort order of parquet files without specifying the schema This PR allows for the following SQL query to be passed without a schema create external table cpu stored as parquet location 'cpu.parquet' with order (time); closes https://github.com/apache/datafusion/issues/7317 * chore: fmt'ing * fix: fmt * fix: remove test that checks for error with schema * Add some more tests * fix: use !asc Co-authored-by: Andrew Lamb * feat: clean up some testing and modify statement when building order by expr --------- Co-authored-by: Andrew Lamb --- .../src/datasource/listing_table_factory.rs | 31 ++++++++++-- datafusion/sql/src/statement.rs | 30 ++++++++++-- datafusion/sql/tests/sql_integration.rs | 7 +++ .../test_files/create_external_table.slt | 47 +++++++++++++++++++ datafusion/sqllogictest/test_files/order.slt | 7 --- 5 files changed, 106 insertions(+), 16 deletions(-) diff --git a/datafusion/core/src/datasource/listing_table_factory.rs b/datafusion/core/src/datasource/listing_table_factory.rs index 591a19aab49b..fed63ec12b49 100644 --- a/datafusion/core/src/datasource/listing_table_factory.rs +++ b/datafusion/core/src/datasource/listing_table_factory.rs @@ -17,6 +17,7 @@ //! Factory for creating ListingTables with default options +use std::collections::HashSet; use std::path::Path; use std::sync::Arc; @@ -27,7 +28,7 @@ use crate::datasource::listing::{ use crate::execution::context::SessionState; use arrow::datatypes::{DataType, SchemaRef}; -use datafusion_common::{arrow_datafusion_err, DataFusionError}; +use datafusion_common::{arrow_datafusion_err, plan_err, DataFusionError, ToDFSchema}; use datafusion_common::{config_datafusion_err, Result}; use datafusion_expr::CreateExternalTable; @@ -113,19 +114,39 @@ impl TableProviderFactory for ListingTableFactory { .with_collect_stat(state.config().collect_statistics()) .with_file_extension(file_extension) .with_target_partitions(state.config().target_partitions()) - .with_table_partition_cols(table_partition_cols) - .with_file_sort_order(cmd.order_exprs.clone()); + .with_table_partition_cols(table_partition_cols); options .validate_partitions(session_state, &table_path) .await?; let resolved_schema = match provided_schema { - None => options.infer_schema(session_state, &table_path).await?, + // We will need to check the table columns against the schema + // this is done so that we can do an ORDER BY for external table creation + // specifically for parquet file format. + // See: https://github.com/apache/datafusion/issues/7317 + None => { + let schema = options.infer_schema(session_state, &table_path).await?; + let df_schema = schema.clone().to_dfschema()?; + let column_refs: HashSet<_> = cmd + .order_exprs + .iter() + .flat_map(|sort| sort.iter()) + .flat_map(|s| s.expr.column_refs()) + .collect(); + + for column in &column_refs { + if !df_schema.has_column(column) { + return plan_err!("Column {column} is not in schema"); + } + } + + schema + } Some(s) => s, }; let config = ListingTableConfig::new(table_path) - .with_listing_options(options) + .with_listing_options(options.with_file_sort_order(cmd.order_exprs.clone())) .with_schema(resolved_schema); let provider = ListingTable::try_new(config)? .with_cache(state.runtime_env().cache_manager.get_file_statistic_cache()); diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index d9719e08052f..29dfe25993f1 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -1136,11 +1136,33 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { schema: &DFSchemaRef, planner_context: &mut PlannerContext, ) -> Result>> { - // Ask user to provide a schema if schema is empty. if !order_exprs.is_empty() && schema.fields().is_empty() { - return plan_err!( - "Provide a schema before specifying the order while creating a table." - ); + let results = order_exprs + .iter() + .map(|lex_order| { + let result = lex_order + .iter() + .map(|order_by_expr| { + let ordered_expr = &order_by_expr.expr; + let ordered_expr = ordered_expr.to_owned(); + let ordered_expr = self + .sql_expr_to_logical_expr( + ordered_expr, + schema, + planner_context, + ) + .unwrap(); + let asc = order_by_expr.asc.unwrap_or(true); + let nulls_first = order_by_expr.nulls_first.unwrap_or(!asc); + + SortExpr::new(ordered_expr, asc, nulls_first) + }) + .collect::>(); + result + }) + .collect::>>(); + + return Ok(results); } let mut all_results = vec![]; diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index bdb84af464f2..5c9655a55606 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -2002,6 +2002,13 @@ fn create_external_table_parquet_no_schema() { quick_test(sql, expected); } +#[test] +fn create_external_table_parquet_no_schema_sort_order() { + let sql = "CREATE EXTERNAL TABLE t STORED AS PARQUET LOCATION 'foo.parquet' WITH ORDER (id)"; + let expected = "CreateExternalTable: Bare { table: \"t\" }"; + quick_test(sql, expected); +} + #[test] fn equijoin_explicit_syntax() { let sql = "SELECT id, order_id \ diff --git a/datafusion/sqllogictest/test_files/create_external_table.slt b/datafusion/sqllogictest/test_files/create_external_table.slt index 3e2412cf021d..12b097c3d5d1 100644 --- a/datafusion/sqllogictest/test_files/create_external_table.slt +++ b/datafusion/sqllogictest/test_files/create_external_table.slt @@ -228,3 +228,50 @@ OPTIONS ( format.delimiter '|', has_header false, compression gzip); + +# Create an external parquet table and infer schema to order by + +# query should succeed +statement ok +CREATE EXTERNAL TABLE t STORED AS parquet LOCATION '../../parquet-testing/data/alltypes_plain.parquet' WITH ORDER (id); + +## Verify that the table is created with a sort order. Explain should show output_ordering=[id@0 ASC] +query TT +EXPLAIN SELECT id FROM t ORDER BY id ASC; +---- +logical_plan +01)Sort: t.id ASC NULLS LAST +02)--TableScan: t projection=[id] +physical_plan ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST] + +## Test a DESC order and verify that output_ordering is ASC from the previous OBRDER BY +query TT +EXPLAIN SELECT id FROM t ORDER BY id DESC; +---- +logical_plan +01)Sort: t.id DESC NULLS FIRST +02)--TableScan: t projection=[id] +physical_plan +01)SortExec: expr=[id@0 DESC], preserve_partitioning=[false] +02)--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 ASC NULLS LAST] + +statement ok +DROP TABLE t; + +# Create table with non default sort order +statement ok +CREATE EXTERNAL TABLE t STORED AS parquet LOCATION '../../parquet-testing/data/alltypes_plain.parquet' WITH ORDER (id DESC NULLS FIRST); + +## Verify that the table is created with a sort order. Explain should show output_ordering=[id@0 DESC NULLS FIRST] +query TT +EXPLAIN SELECT id FROM t; +---- +logical_plan TableScan: t projection=[id] +physical_plan ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id], output_ordering=[id@0 DESC] + +statement ok +DROP TABLE t; + +# query should fail with bad column +statement error DataFusion error: Error during planning: Column foo is not in schema +CREATE EXTERNAL TABLE t STORED AS parquet LOCATION '../../parquet-testing/data/alltypes_plain.parquet' WITH ORDER (foo); diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index 7bb872e5a48f..f53363b6eb38 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -653,13 +653,6 @@ physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/te query error DataFusion error: Error during planning: Column a is not in schema CREATE EXTERNAL TABLE dt (a_id integer, a_str string, a_bool boolean) STORED AS CSV WITH ORDER (a ASC) LOCATION 'file://path/to/table'; - -# Create external table with DDL ordered columns without schema -# When schema is missing the query is expected to fail -query error DataFusion error: Error during planning: Provide a schema before specifying the order while creating a table\. -CREATE EXTERNAL TABLE dt STORED AS CSV WITH ORDER (a ASC) LOCATION 'file://path/to/table'; - - # Sort with duplicate sort expressions # Table is sorted multiple times on the same column name and should not fail statement ok From 244ce5a2a47bca27b086f184f6aab0b63394c2a1 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sat, 21 Sep 2024 19:24:38 +0800 Subject: [PATCH 07/25] Support List type coercion for CASE-WHEN-THEN expression (#12490) * support list type coercion * add planing and sql tests * clippy * support to compare nested type for case-when expression * simplify the macro rules * fix the FixedSizeList type coercion and add tests * add test for THEN-ELSE --- .../expr-common/src/type_coercion/binary.rs | 73 +++++++ .../optimizer/src/analyzer/type_coercion.rs | 180 ++++++++++++++++++ .../physical-expr/src/expressions/case.rs | 10 +- datafusion/sqllogictest/test_files/case.slt | 94 +++++++++ 4 files changed, 355 insertions(+), 2 deletions(-) diff --git a/datafusion/expr-common/src/type_coercion/binary.rs b/datafusion/expr-common/src/type_coercion/binary.rs index d1144ed40759..c1e96a8fa97d 100644 --- a/datafusion/expr-common/src/type_coercion/binary.rs +++ b/datafusion/expr-common/src/type_coercion/binary.rs @@ -1021,6 +1021,22 @@ fn list_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { use arrow::datatypes::DataType::*; match (lhs_type, rhs_type) { (List(_), List(_)) => Some(lhs_type.clone()), + (LargeList(_), List(_)) => Some(lhs_type.clone()), + (List(_), LargeList(_)) => Some(rhs_type.clone()), + (LargeList(_), LargeList(_)) => Some(lhs_type.clone()), + (List(_), FixedSizeList(_, _)) => Some(lhs_type.clone()), + (FixedSizeList(_, _), List(_)) => Some(rhs_type.clone()), + // Coerce to the left side FixedSizeList type if the list lengths are the same, + // otherwise coerce to list with the left type for dynamic length + (FixedSizeList(lf, ls), FixedSizeList(_, rs)) => { + if ls == rs { + Some(lhs_type.clone()) + } else { + Some(List(Arc::clone(lf))) + } + } + (LargeList(_), FixedSizeList(_, _)) => Some(lhs_type.clone()), + (FixedSizeList(_, _), LargeList(_)) => Some(rhs_type.clone()), _ => None, } } @@ -1906,6 +1922,63 @@ mod tests { DataType::Timestamp(TimeUnit::Second, Some("Europe/Brussels".into())) ); + // list + let inner_field = Arc::new(Field::new("item", DataType::Int64, true)); + test_coercion_binary_rule!( + DataType::List(Arc::clone(&inner_field)), + DataType::List(Arc::clone(&inner_field)), + Operator::Eq, + DataType::List(Arc::clone(&inner_field)) + ); + test_coercion_binary_rule!( + DataType::List(Arc::clone(&inner_field)), + DataType::LargeList(Arc::clone(&inner_field)), + Operator::Eq, + DataType::LargeList(Arc::clone(&inner_field)) + ); + test_coercion_binary_rule!( + DataType::LargeList(Arc::clone(&inner_field)), + DataType::List(Arc::clone(&inner_field)), + Operator::Eq, + DataType::LargeList(Arc::clone(&inner_field)) + ); + test_coercion_binary_rule!( + DataType::LargeList(Arc::clone(&inner_field)), + DataType::LargeList(Arc::clone(&inner_field)), + Operator::Eq, + DataType::LargeList(Arc::clone(&inner_field)) + ); + test_coercion_binary_rule!( + DataType::FixedSizeList(Arc::clone(&inner_field), 10), + DataType::FixedSizeList(Arc::clone(&inner_field), 10), + Operator::Eq, + DataType::FixedSizeList(Arc::clone(&inner_field), 10) + ); + test_coercion_binary_rule!( + DataType::FixedSizeList(Arc::clone(&inner_field), 10), + DataType::LargeList(Arc::clone(&inner_field)), + Operator::Eq, + DataType::LargeList(Arc::clone(&inner_field)) + ); + test_coercion_binary_rule!( + DataType::LargeList(Arc::clone(&inner_field)), + DataType::FixedSizeList(Arc::clone(&inner_field), 10), + Operator::Eq, + DataType::LargeList(Arc::clone(&inner_field)) + ); + test_coercion_binary_rule!( + DataType::List(Arc::clone(&inner_field)), + DataType::FixedSizeList(Arc::clone(&inner_field), 10), + Operator::Eq, + DataType::List(Arc::clone(&inner_field)) + ); + test_coercion_binary_rule!( + DataType::FixedSizeList(Arc::clone(&inner_field), 10), + DataType::List(Arc::clone(&inner_field)), + Operator::Eq, + DataType::List(Arc::clone(&inner_field)) + ); + // TODO add other data type Ok(()) } diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 7a8746572cfd..3e31ac1c6375 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -1811,6 +1811,186 @@ mod test { Ok(()) } + macro_rules! test_case_expression { + ($expr:expr, $when_then:expr, $case_when_type:expr, $then_else_type:expr, $schema:expr) => { + let case = Case { + expr: $expr.map(|e| Box::new(col(e))), + when_then_expr: $when_then, + else_expr: None, + }; + + let expected = + cast_helper(case.clone(), &$case_when_type, &$then_else_type, &$schema); + + let actual = coerce_case_expression(case, &$schema)?; + assert_eq!(expected, actual); + }; + } + + #[test] + fn tes_case_when_list() -> Result<()> { + let inner_field = Arc::new(Field::new("item", DataType::Int64, true)); + let schema = Arc::new(DFSchema::from_unqualified_fields( + vec![ + Field::new( + "large_list", + DataType::LargeList(Arc::clone(&inner_field)), + true, + ), + Field::new( + "fixed_list", + DataType::FixedSizeList(Arc::clone(&inner_field), 3), + true, + ), + Field::new("list", DataType::List(inner_field), true), + ] + .into(), + std::collections::HashMap::new(), + )?); + + test_case_expression!( + Some("list"), + vec![(Box::new(col("large_list")), Box::new(lit("1")))], + DataType::LargeList(Arc::new(Field::new("item", DataType::Int64, true))), + DataType::Utf8, + schema + ); + + test_case_expression!( + Some("large_list"), + vec![(Box::new(col("list")), Box::new(lit("1")))], + DataType::LargeList(Arc::new(Field::new("item", DataType::Int64, true))), + DataType::Utf8, + schema + ); + + test_case_expression!( + Some("list"), + vec![(Box::new(col("fixed_list")), Box::new(lit("1")))], + DataType::List(Arc::new(Field::new("item", DataType::Int64, true))), + DataType::Utf8, + schema + ); + + test_case_expression!( + Some("fixed_list"), + vec![(Box::new(col("list")), Box::new(lit("1")))], + DataType::List(Arc::new(Field::new("item", DataType::Int64, true))), + DataType::Utf8, + schema + ); + + test_case_expression!( + Some("fixed_list"), + vec![(Box::new(col("large_list")), Box::new(lit("1")))], + DataType::LargeList(Arc::new(Field::new("item", DataType::Int64, true))), + DataType::Utf8, + schema + ); + + test_case_expression!( + Some("large_list"), + vec![(Box::new(col("fixed_list")), Box::new(lit("1")))], + DataType::LargeList(Arc::new(Field::new("item", DataType::Int64, true))), + DataType::Utf8, + schema + ); + Ok(()) + } + + #[test] + fn test_then_else_list() -> Result<()> { + let inner_field = Arc::new(Field::new("item", DataType::Int64, true)); + let schema = Arc::new(DFSchema::from_unqualified_fields( + vec![ + Field::new("boolean", DataType::Boolean, true), + Field::new( + "large_list", + DataType::LargeList(Arc::clone(&inner_field)), + true, + ), + Field::new( + "fixed_list", + DataType::FixedSizeList(Arc::clone(&inner_field), 3), + true, + ), + Field::new("list", DataType::List(inner_field), true), + ] + .into(), + std::collections::HashMap::new(), + )?); + + // large list and list + test_case_expression!( + None::, + vec![ + (Box::new(col("boolean")), Box::new(col("large_list"))), + (Box::new(col("boolean")), Box::new(col("list"))) + ], + DataType::Boolean, + DataType::LargeList(Arc::new(Field::new("item", DataType::Int64, true))), + schema + ); + + test_case_expression!( + None::, + vec![ + (Box::new(col("boolean")), Box::new(col("list"))), + (Box::new(col("boolean")), Box::new(col("large_list"))) + ], + DataType::Boolean, + DataType::LargeList(Arc::new(Field::new("item", DataType::Int64, true))), + schema + ); + + // fixed list and list + test_case_expression!( + None::, + vec![ + (Box::new(col("boolean")), Box::new(col("fixed_list"))), + (Box::new(col("boolean")), Box::new(col("list"))) + ], + DataType::Boolean, + DataType::List(Arc::new(Field::new("item", DataType::Int64, true))), + schema + ); + + test_case_expression!( + None::, + vec![ + (Box::new(col("boolean")), Box::new(col("list"))), + (Box::new(col("boolean")), Box::new(col("fixed_list"))) + ], + DataType::Boolean, + DataType::List(Arc::new(Field::new("item", DataType::Int64, true))), + schema + ); + + // fixed list and large list + test_case_expression!( + None::, + vec![ + (Box::new(col("boolean")), Box::new(col("fixed_list"))), + (Box::new(col("boolean")), Box::new(col("large_list"))) + ], + DataType::Boolean, + DataType::LargeList(Arc::new(Field::new("item", DataType::Int64, true))), + schema + ); + + test_case_expression!( + None::, + vec![ + (Box::new(col("boolean")), Box::new(col("large_list"))), + (Box::new(col("boolean")), Box::new(col("fixed_list"))) + ], + DataType::Boolean, + DataType::LargeList(Arc::new(Field::new("item", DataType::Int64, true))), + schema + ); + Ok(()) + } + #[test] fn interval_plus_timestamp() -> Result<()> { // SELECT INTERVAL '1' YEAR + '2000-01-01T00:00:00'::timestamp; diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index 712175c9afbe..ffb431b200f2 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -24,7 +24,6 @@ use crate::physical_expr::down_cast_any_ref; use crate::PhysicalExpr; use arrow::array::*; -use arrow::compute::kernels::cmp::eq; use arrow::compute::kernels::zip::zip; use arrow::compute::{and, and_not, is_null, not, nullif, or, prep_null_mask_filter}; use arrow::datatypes::{DataType, Schema}; @@ -33,6 +32,7 @@ use datafusion_common::{exec_err, internal_err, DataFusionError, Result, ScalarV use datafusion_expr::ColumnarValue; use super::{Column, Literal}; +use datafusion_physical_expr_common::datum::compare_with_eq; use itertools::Itertools; type WhenThen = (Arc, Arc); @@ -204,7 +204,13 @@ impl CaseExpr { .evaluate_selection(batch, &remainder)?; let when_value = when_value.into_array(batch.num_rows())?; // build boolean array representing which rows match the "when" value - let when_match = eq(&when_value, &base_value)?; + let when_match = compare_with_eq( + &when_value, + &base_value, + // The types of case and when expressions will be coerced to match. + // We only need to check if the base_value is nested. + base_value.data_type().is_nested(), + )?; // Treat nulls as false let when_match = match when_match.null_count() { 0 => Cow::Borrowed(&when_match), diff --git a/datafusion/sqllogictest/test_files/case.slt b/datafusion/sqllogictest/test_files/case.slt index 70063b88fb19..3c967eed219a 100644 --- a/datafusion/sqllogictest/test_files/case.slt +++ b/datafusion/sqllogictest/test_files/case.slt @@ -108,3 +108,97 @@ SELECT CASE WHEN false THEN 1 ELSE 0 END FROM foo 0 0 0 + +# test then value type coercion +# List(Utf8) will be casted to List(Int64) +query ? +SELECT CASE 1 WHEN 1 THEN ['1', '2', '3'] WHEN 2 THEN [1, 2, 3] ELSE null END; +---- +[1, 2, 3] + +query ? +SELECT CASE 1 WHEN 1 THEN [[1,2], [2,4]] WHEN 2 THEN [['1','2'], ['2','4']] ELSE null END; +---- +[[1, 2], [2, 4]] + +query ? +SELECT CASE 1 WHEN 1 THEN [1,2,3] WHEN 2 THEN arrow_cast([1,2,3], 'LargeList(Int64)') WHEN 3 THEN arrow_cast([1,2,3], 'FixedSizeList(3, Int32)') ELSE null END; +---- +[1, 2, 3] + +query ? +SELECT CASE 1 WHEN 1 THEN [[1,2], [2,4]] WHEN 2 THEN arrow_cast([[1,2], [2,4]], 'LargeList(LargeList(Int64))') WHEN 3 THEN arrow_cast([[1,2], [2,4]], 'FixedSizeList(2, FixedSizeList(2, Int32))') ELSE null END; +---- +[[1, 2], [2, 4]] + +query ? +SELECT CASE 1 WHEN 1 THEN [1,2,3] WHEN 2 THEN arrow_cast(['1','2','3'], 'LargeList(Utf8)') WHEN 3 THEN arrow_cast(['1','2','3'], 'FixedSizeList(3, Utf8)') ELSE null END; +---- +[1, 2, 3] + +query ? +SELECT CASE 1 WHEN 1 THEN [[1,2], [2,4]] WHEN 2 THEN arrow_cast([['1','2'], ['2','4']], 'LargeList(LargeList(Utf8))') WHEN 3 THEN arrow_cast([['1','2'], ['2','4']], 'FixedSizeList(2, FixedSizeList(2, Utf8))') ELSE null END; +---- +[[1, 2], [2, 4]] + +query ? +SELECT CASE 1 WHEN 1 THEN arrow_cast([1,2,3], 'LargeList(Int64)') WHEN 2 THEN arrow_cast(['1','2','3'], 'LargeList(Utf8)') ELSE null END; +---- +[1, 2, 3] + +query ? +SELECT CASE 1 WHEN 1 THEN arrow_cast([1, 2], 'FixedSizeList(2, Int64)') WHEN 2 THEN arrow_cast(['1', '2', '3'], 'FixedSizeList(3, Utf8)') ELSE null END; +---- +[1, 2] + +query error DataFusion error: type_coercion +SELECT CASE 1 WHEN 1 THEN [1,2,3] WHEN 2 THEN 'test' ELSE null END; + +# test case when type coercion +query I +SELECT CASE [1,2,3] WHEN arrow_cast([1,2,3], 'LargeList(Int64)') THEN 1 ELSE 0 END; +---- +1 + +query I +SELECT CASE [1,2,3] WHEN arrow_cast(['1','2','3'], 'LargeList(Int64)') THEN 1 ELSE 0 END; +---- +1 + +query I +SELECT CASE arrow_cast([1,2,3], 'LargeList(Int64)') WHEN [1,2,3] THEN 1 ELSE 0 END; +---- +1 + +query I +SELECT CASE [[1,2],[2,4]] WHEN arrow_cast([[1,2],[2,4]], 'LargeList(LargeList(Int64))') THEN 1 ELSE 0 END; +---- +1 + +query I +SELECT CASE arrow_cast([1,2,3], 'FixedSizeList(3, Int64)') WHEN [1,2,3] THEN 1 ELSE 0 END; +---- +1 + +query error DataFusion error: type_coercion +SELECT CASE [1,2,3] WHEN 'test' THEN 1 ELSE 0 END; + +query I +SELECT CASE arrow_cast([1,2], 'FixedSizeList(2, Int64)') WHEN arrow_cast([1,2,3], 'FixedSizeList(3, Int64)') THEN 1 ELSE 0 END; +---- +0 + +query I +SELECT CASE arrow_cast([1,2], 'FixedSizeList(2, Int64)') WHEN arrow_cast(['1','2','3'], 'FixedSizeList(3, Utf8)') THEN 1 ELSE 0 END; +---- +0 + +query I +SELECT CASE arrow_cast(['1','2'], 'FixedSizeList(2, Utf8)') WHEN arrow_cast([1,2,3], 'FixedSizeList(3, Int64)') THEN 1 ELSE 0 END; +---- +0 + +query I +SELECT CASE arrow_cast([1,2,3], 'FixedSizeList(3, Int64)') WHEN arrow_cast([1,2,3], 'FixedSizeList(3, Int64)') THEN 1 ELSE 0 END; +---- +1 From 3bd41bcd076efd05026f5b6f236fa3fecdfea6bc Mon Sep 17 00:00:00 2001 From: Val Lorentz Date: Sun, 22 Sep 2024 06:29:24 +0200 Subject: [PATCH 08/25] Sort metrics alphabetically in EXPLAIN ANALYZE output (#12568) --- datafusion/physical-plan/src/metrics/mod.rs | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/metrics/mod.rs b/datafusion/physical-plan/src/metrics/mod.rs index 9232865aa09c..ead0ca336938 100644 --- a/datafusion/physical-plan/src/metrics/mod.rs +++ b/datafusion/physical-plan/src/metrics/mod.rs @@ -301,8 +301,12 @@ impl MetricsSet { /// Sort the order of metrics so the "most useful" show up first pub fn sorted_for_display(mut self) -> Self { - self.metrics - .sort_unstable_by_key(|metric| metric.value().display_sort_key()); + self.metrics.sort_unstable_by_key(|metric| { + ( + metric.value().display_sort_key(), + metric.value().name().to_owned(), + ) + }); self } @@ -665,7 +669,9 @@ mod tests { MetricBuilder::new(&metrics).end_timestamp(0); MetricBuilder::new(&metrics).start_timestamp(0); MetricBuilder::new(&metrics).elapsed_compute(0); + MetricBuilder::new(&metrics).counter("the_second_counter", 0); MetricBuilder::new(&metrics).counter("the_counter", 0); + MetricBuilder::new(&metrics).counter("the_third_counter", 0); MetricBuilder::new(&metrics).subset_time("the_time", 0); MetricBuilder::new(&metrics).output_rows(0); let metrics = metrics.clone_inner(); @@ -675,9 +681,9 @@ mod tests { n.join(", ") } - assert_eq!("end_timestamp, start_timestamp, elapsed_compute, the_counter, the_time, output_rows", metric_names(&metrics)); + assert_eq!("end_timestamp, start_timestamp, elapsed_compute, the_second_counter, the_counter, the_third_counter, the_time, output_rows", metric_names(&metrics)); let metrics = metrics.sorted_for_display(); - assert_eq!("output_rows, elapsed_compute, the_counter, the_time, start_timestamp, end_timestamp", metric_names(&metrics)); + assert_eq!("output_rows, elapsed_compute, the_counter, the_second_counter, the_third_counter, the_time, start_timestamp, end_timestamp", metric_names(&metrics)); } } From 300a39b89308ae949a99058660cb69e112e65e3c Mon Sep 17 00:00:00 2001 From: Val Lorentz Date: Sun, 22 Sep 2024 13:14:44 +0200 Subject: [PATCH 09/25] parquet: Make page_index/pushdown metrics consistent with row_group metrics (#12545) * parquet: Make page_index/pushdown metrics consistent with row_group metrics 1. Rename `{pushdown,page_index}_filtered` to `{pushdown,page_index}_pruned` 2. Add `{pushdown,page_index}_matched` The latter makes it clearer in EXPLAIN ANALYZE when the Page Index is not checked because their row groups were already eliminated (with a Bloom Filter or row group statistics). * Add missing metric definitions in the docs Co-authored-by: Andrew Lamb * s/pass/select/ --------- Co-authored-by: Andrew Lamb --- .../physical_plan/parquet/metrics.rs | 28 +++++++++++++------ .../datasource/physical_plan/parquet/mod.rs | 15 ++++++---- .../physical_plan/parquet/page_filter.rs | 17 +++++++++-- .../physical_plan/parquet/row_filter.rs | 24 +++++++++++----- .../core/tests/parquet/filter_pushdown.rs | 20 +++++++------ datafusion/core/tests/parquet/mod.rs | 2 +- docs/source/user-guide/explain-usage.md | 15 ++++++++++ 7 files changed, 90 insertions(+), 31 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/metrics.rs b/datafusion/core/src/datasource/physical_plan/parquet/metrics.rs index 3599a2f5fa51..20dbc94ecf1d 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/metrics.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/metrics.rs @@ -40,11 +40,15 @@ pub struct ParquetFileMetrics { /// Total number of bytes scanned pub bytes_scanned: Count, /// Total rows filtered out by predicates pushed into parquet scan - pub pushdown_rows_filtered: Count, + pub pushdown_rows_pruned: Count, + /// Total rows passed predicates pushed into parquet scan + pub pushdown_rows_matched: Count, /// Total time spent evaluating pushdown filters pub pushdown_eval_time: Time, /// Total rows filtered out by parquet page index - pub page_index_rows_filtered: Count, + pub page_index_rows_pruned: Count, + /// Total rows passed through the parquet page index + pub page_index_rows_matched: Count, /// Total time spent evaluating parquet page index filters pub page_index_eval_time: Time, } @@ -80,16 +84,22 @@ impl ParquetFileMetrics { .with_new_label("filename", filename.to_string()) .counter("bytes_scanned", partition); - let pushdown_rows_filtered = MetricBuilder::new(metrics) + let pushdown_rows_pruned = MetricBuilder::new(metrics) .with_new_label("filename", filename.to_string()) - .counter("pushdown_rows_filtered", partition); + .counter("pushdown_rows_pruned", partition); + let pushdown_rows_matched = MetricBuilder::new(metrics) + .with_new_label("filename", filename.to_string()) + .counter("pushdown_rows_matched", partition); let pushdown_eval_time = MetricBuilder::new(metrics) .with_new_label("filename", filename.to_string()) .subset_time("pushdown_eval_time", partition); - let page_index_rows_filtered = MetricBuilder::new(metrics) + let page_index_rows_pruned = MetricBuilder::new(metrics) + .with_new_label("filename", filename.to_string()) + .counter("page_index_rows_pruned", partition); + let page_index_rows_matched = MetricBuilder::new(metrics) .with_new_label("filename", filename.to_string()) - .counter("page_index_rows_filtered", partition); + .counter("page_index_rows_matched", partition); let page_index_eval_time = MetricBuilder::new(metrics) .with_new_label("filename", filename.to_string()) @@ -102,9 +112,11 @@ impl ParquetFileMetrics { row_groups_matched_statistics, row_groups_pruned_statistics, bytes_scanned, - pushdown_rows_filtered, + pushdown_rows_pruned, + pushdown_rows_matched, pushdown_eval_time, - page_index_rows_filtered, + page_index_rows_pruned, + page_index_rows_matched, page_index_eval_time, } } diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 7d3342db5ccd..7aacf5cf5c5c 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -1173,7 +1173,8 @@ mod tests { assert_batches_sorted_eq!(expected, &rt.batches.unwrap()); let metrics = rt.parquet_exec.metrics().unwrap(); // Note there are were 6 rows in total (across three batches) - assert_eq!(get_value(&metrics, "pushdown_rows_filtered"), 4); + assert_eq!(get_value(&metrics, "pushdown_rows_pruned"), 4); + assert_eq!(get_value(&metrics, "pushdown_rows_matched"), 2); } #[tokio::test] @@ -1325,7 +1326,8 @@ mod tests { assert_batches_sorted_eq!(expected, &rt.batches.unwrap()); let metrics = rt.parquet_exec.metrics().unwrap(); // Note there are were 6 rows in total (across three batches) - assert_eq!(get_value(&metrics, "pushdown_rows_filtered"), 5); + assert_eq!(get_value(&metrics, "pushdown_rows_pruned"), 5); + assert_eq!(get_value(&metrics, "pushdown_rows_matched"), 1); } #[tokio::test] @@ -1399,7 +1401,8 @@ mod tests { // There are 4 rows pruned in each of batch2, batch3, and // batch4 for a total of 12. batch1 had no pruning as c2 was // filled in as null - assert_eq!(get_value(&metrics, "page_index_rows_filtered"), 12); + assert_eq!(get_value(&metrics, "page_index_rows_pruned"), 12); + assert_eq!(get_value(&metrics, "page_index_rows_matched"), 6); } #[tokio::test] @@ -1786,7 +1789,8 @@ mod tests { "+-----+" ]; assert_batches_sorted_eq!(expected, &rt.batches.unwrap()); - assert_eq!(get_value(&metrics, "page_index_rows_filtered"), 4); + assert_eq!(get_value(&metrics, "page_index_rows_pruned"), 4); + assert_eq!(get_value(&metrics, "page_index_rows_matched"), 2); assert!( get_value(&metrics, "page_index_eval_time") > 0, "no eval time in metrics: {metrics:#?}" @@ -1855,7 +1859,8 @@ mod tests { // pushdown predicates have eliminated all 4 bar rows and the // null row for 5 rows total - assert_eq!(get_value(&metrics, "pushdown_rows_filtered"), 5); + assert_eq!(get_value(&metrics, "pushdown_rows_pruned"), 5); + assert_eq!(get_value(&metrics, "pushdown_rows_matched"), 2); assert!( get_value(&metrics, "pushdown_eval_time") > 0, "no eval time in metrics: {metrics:#?}" diff --git a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs index 4e71993b5153..ced07de974f6 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs @@ -178,6 +178,8 @@ impl PagePruningAccessPlanFilter { // track the total number of rows that should be skipped let mut total_skip = 0; + // track the total number of rows that should not be skipped + let mut total_select = 0; // for each row group specified in the access plan let row_group_indexes = access_plan.row_group_indexes(); @@ -242,8 +244,10 @@ impl PagePruningAccessPlanFilter { if let Some(overall_selection) = overall_selection { if overall_selection.selects_any() { let rows_skipped = rows_skipped(&overall_selection); - trace!("Overall selection from predicate skipped {rows_skipped}: {overall_selection:?}"); + let rows_selected = rows_selected(&overall_selection); + trace!("Overall selection from predicate skipped {rows_skipped}, selected {rows_selected}: {overall_selection:?}"); total_skip += rows_skipped; + total_select += rows_selected; access_plan.scan_selection(row_group_index, overall_selection) } else { // Selection skips all rows, so skip the entire row group @@ -258,7 +262,8 @@ impl PagePruningAccessPlanFilter { } } - file_metrics.page_index_rows_filtered.add(total_skip); + file_metrics.page_index_rows_pruned.add(total_skip); + file_metrics.page_index_rows_matched.add(total_select); access_plan } @@ -276,6 +281,14 @@ fn rows_skipped(selection: &RowSelection) -> usize { .fold(0, |acc, x| if x.skip { acc + x.row_count } else { acc }) } +/// returns the number of rows not skipped in the selection +/// TODO should this be upstreamed to RowSelection? +fn rows_selected(selection: &RowSelection) -> usize { + selection + .iter() + .fold(0, |acc, x| if x.skip { acc } else { acc + x.row_count }) +} + fn update_selection( current_selection: Option, row_selection: RowSelection, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs index d3bc8030cf7f..a8e3e72c11f3 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs @@ -105,7 +105,9 @@ pub(crate) struct DatafusionArrowPredicate { /// Columns required to evaluate the expression in the arrow schema projection: Vec, /// how many rows were filtered out by this predicate - rows_filtered: metrics::Count, + rows_pruned: metrics::Count, + /// how many rows passed this predicate + rows_matched: metrics::Count, /// how long was spent evaluating this predicate time: metrics::Time, /// used to perform type coercion while filtering rows @@ -118,7 +120,8 @@ impl DatafusionArrowPredicate { candidate: FilterCandidate, schema: &Schema, metadata: &ParquetMetaData, - rows_filtered: metrics::Count, + rows_pruned: metrics::Count, + rows_matched: metrics::Count, time: metrics::Time, schema_mapping: Arc, ) -> Result { @@ -140,7 +143,8 @@ impl DatafusionArrowPredicate { metadata.file_metadata().schema_descr(), candidate.projection, ), - rows_filtered, + rows_pruned, + rows_matched, time, schema_mapping, }) @@ -167,8 +171,10 @@ impl ArrowPredicate for DatafusionArrowPredicate { .and_then(|v| v.into_array(batch.num_rows())) .and_then(|array| { let bool_arr = as_boolean_array(&array)?.clone(); - let num_filtered = bool_arr.len() - bool_arr.true_count(); - self.rows_filtered.add(num_filtered); + let num_matched = bool_arr.true_count(); + let num_pruned = bool_arr.len() - num_matched; + self.rows_pruned.add(num_pruned); + self.rows_matched.add(num_matched); timer.stop(); Ok(bool_arr) }) @@ -523,7 +529,8 @@ pub fn build_row_filter( file_metrics: &ParquetFileMetrics, schema_mapping: Arc, ) -> Result> { - let rows_filtered = &file_metrics.pushdown_rows_filtered; + let rows_pruned = &file_metrics.pushdown_rows_pruned; + let rows_matched = &file_metrics.pushdown_rows_matched; let time = &file_metrics.pushdown_eval_time; // Split into conjuncts: @@ -563,7 +570,8 @@ pub fn build_row_filter( candidate, file_schema, metadata, - rows_filtered.clone(), + rows_pruned.clone(), + rows_matched.clone(), time.clone(), Arc::clone(&schema_mapping), ) @@ -705,6 +713,7 @@ mod test { &file_schema, &metadata, Count::new(), + Count::new(), Time::new(), Arc::clone(&schema_mapping), ) @@ -728,6 +737,7 @@ mod test { &file_schema, &metadata, Count::new(), + Count::new(), Time::new(), schema_mapping, ) diff --git a/datafusion/core/tests/parquet/filter_pushdown.rs b/datafusion/core/tests/parquet/filter_pushdown.rs index 8c7624f07813..8def192f9331 100644 --- a/datafusion/core/tests/parquet/filter_pushdown.rs +++ b/datafusion/core/tests/parquet/filter_pushdown.rs @@ -538,24 +538,28 @@ impl<'a> TestCase<'a> { PushdownExpected::None }; - let pushdown_rows_filtered = get_value(&metrics, "pushdown_rows_filtered"); - println!(" pushdown_rows_filtered: {pushdown_rows_filtered}"); + let pushdown_rows_pruned = get_value(&metrics, "pushdown_rows_pruned"); + println!(" pushdown_rows_pruned: {pushdown_rows_pruned}"); + let pushdown_rows_matched = get_value(&metrics, "pushdown_rows_matched"); + println!(" pushdown_rows_matched: {pushdown_rows_matched}"); match pushdown_expected { PushdownExpected::None => { - assert_eq!(pushdown_rows_filtered, 0, "{}", self.name); + assert_eq!(pushdown_rows_pruned, 0, "{}", self.name); } PushdownExpected::Some => { assert!( - pushdown_rows_filtered > 0, + pushdown_rows_pruned > 0, "{}: Expected to filter rows via pushdown, but none were", self.name ); } }; - let page_index_rows_filtered = get_value(&metrics, "page_index_rows_filtered"); - println!(" page_index_rows_filtered: {page_index_rows_filtered}"); + let page_index_rows_pruned = get_value(&metrics, "page_index_rows_pruned"); + println!(" page_index_rows_pruned: {page_index_rows_pruned}"); + let page_index_rows_matched = get_value(&metrics, "page_index_rows_matched"); + println!(" page_index_rows_matched: {page_index_rows_matched}"); let page_index_filtering_expected = if scan_options.enable_page_index { self.page_index_filtering_expected @@ -567,11 +571,11 @@ impl<'a> TestCase<'a> { match page_index_filtering_expected { PageIndexFilteringExpected::None => { - assert_eq!(page_index_rows_filtered, 0); + assert_eq!(page_index_rows_pruned, 0); } PageIndexFilteringExpected::Some => { assert!( - page_index_rows_filtered > 0, + page_index_rows_pruned > 0, "Expected to filter rows via page index but none were", ); } diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index 60a8dd400786..cfa2a3df3ba2 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -171,7 +171,7 @@ impl TestOutput { /// The number of row pages pruned fn row_pages_pruned(&self) -> Option { - self.metric_value("page_index_rows_filtered") + self.metric_value("page_index_rows_pruned") } fn description(&self) -> String { diff --git a/docs/source/user-guide/explain-usage.md b/docs/source/user-guide/explain-usage.md index a65fad92d104..b376c2eedb53 100644 --- a/docs/source/user-guide/explain-usage.md +++ b/docs/source/user-guide/explain-usage.md @@ -223,6 +223,21 @@ Again, reading from bottom up: - `SortPreservingMergeExec` - `output_rows=5`, `elapsed_compute=2.375µs`: Produced the final 5 rows in 2.375µs (microseconds) +When predicate pushdown is enabled, `ParquetExec` gains the following metrics: + +- `page_index_rows_matched`: number of rows in pages that were tested by a page index filter, and passed +- `page_index_rows_pruned`: number of rows in pages that were tested by a page index filter, and did not pass +- `row_groups_matched_bloom_filter`: number of rows in row groups that were tested by a Bloom Filter, and passed +- `row_groups_pruned_bloom_filter`: number of rows in row groups that were tested by a Bloom Filter, and did not pass +- `row_groups_matched_statistics`: number of rows in row groups that were tested by row group statistics (min and max value), and passed +- `row_groups_pruned_statistics`: number of rows in row groups that were tested by row group statistics (min and max value), and did not pass +- `pushdown_rows_matched`: rows that were tested by any of the above filtered, and passed all of them (this should be minimum of `page_index_rows_matched`, `row_groups_pruned_bloom_filter`, and `row_groups_pruned_statistics`) +- `pushdown_rows_pruned`: rows that were tested by any of the above filtered, and did not pass one of them (this should be sum of `page_index_rows_matched`, `row_groups_pruned_bloom_filter`, and `row_groups_pruned_statistics`) +- `predicate_evaluation_errors`: number of times evaluating the filter expression failed (expected to be zero in normal operation) +- `num_predicate_creation_errors`: number of errors creating predicates (expected to be zero in normal operation) +- `pushdown_eval_time`: time spent evaluating these filters +- `page_index_eval_time`: time required to evaluate the page index filters + ## Partitions and Execution DataFusion determines the optimal number of cores to use as part of query From 72a1053006b8a714c6ebbe7572de532882d8b0c8 Mon Sep 17 00:00:00 2001 From: OussamaSaoudi <45303303+OussamaSaoudi@users.noreply.github.com> Date: Sun, 22 Sep 2024 04:32:03 -0700 Subject: [PATCH 10/25] Create try_new and change calls to new (#12566) --- datafusion-cli/src/main.rs | 2 +- datafusion/execution/src/runtime_env.rs | 10 +++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/datafusion-cli/src/main.rs b/datafusion-cli/src/main.rs index cdefada5e24a..4c6c352ff339 100644 --- a/datafusion-cli/src/main.rs +++ b/datafusion-cli/src/main.rs @@ -232,7 +232,7 @@ async fn main_inner() -> Result<()> { } fn create_runtime_env(rn_config: RuntimeConfig) -> Result { - RuntimeEnv::new(rn_config) + RuntimeEnv::try_new(rn_config) } fn parse_valid_file(dir: &str) -> Result { diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index 574d387ae697..4022eb07de0c 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -66,8 +66,12 @@ impl Debug for RuntimeEnv { } impl RuntimeEnv { - /// Create env based on configuration + #[deprecated(note = "please use `try_new` instead")] pub fn new(config: RuntimeConfig) -> Result { + Self::try_new(config) + } + /// Create env based on configuration + pub fn try_new(config: RuntimeConfig) -> Result { let RuntimeConfig { memory_pool, disk_manager, @@ -100,7 +104,7 @@ impl RuntimeEnv { /// # use std::sync::Arc; /// # use url::Url; /// # use datafusion_execution::runtime_env::RuntimeEnv; - /// # let runtime_env = RuntimeEnv::new(Default::default()).unwrap(); + /// # let runtime_env = RuntimeEnv::try_new(Default::default()).unwrap(); /// let url = Url::try_from("file://").unwrap(); /// let object_store = object_store::local::LocalFileSystem::new(); /// // register the object store with the runtime environment @@ -115,7 +119,7 @@ impl RuntimeEnv { /// # use std::sync::Arc; /// # use url::Url; /// # use datafusion_execution::runtime_env::RuntimeEnv; - /// # let runtime_env = RuntimeEnv::new(Default::default()).unwrap(); + /// # let runtime_env = RuntimeEnv::try_new(Default::default()).unwrap(); /// # // use local store for example as http feature is not enabled /// # let http_store = object_store::local::LocalFileSystem::new(); /// // create a new object store via object_store::http::HttpBuilder; From f6a0ed0dbf8983fbbe815d9c7c0e381de5789397 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 22 Sep 2024 07:49:34 -0400 Subject: [PATCH 11/25] Make SessionContext::enable_url_table consume self (#12573) --- .../external_dependency/query-aws-s3.rs | 2 +- datafusion/core/src/execution/context/mod.rs | 74 ++++++++++++++++--- 2 files changed, 64 insertions(+), 12 deletions(-) diff --git a/datafusion-examples/examples/external_dependency/query-aws-s3.rs b/datafusion-examples/examples/external_dependency/query-aws-s3.rs index 9c4d76703c9c..da2d7e4879f9 100644 --- a/datafusion-examples/examples/external_dependency/query-aws-s3.rs +++ b/datafusion-examples/examples/external_dependency/query-aws-s3.rs @@ -64,7 +64,7 @@ async fn main() -> Result<()> { df.show().await?; // dynamic query by the file path - ctx.enable_url_table(); + let ctx = ctx.enable_url_table(); let df = ctx .sql(format!(r#"SELECT * FROM '{}' LIMIT 10"#, &path).as_str()) .await?; diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 27c0e9c83222..53eb7c431b47 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -343,11 +343,17 @@ impl SessionContext { } } - /// Enable dynamic file querying for the current session. + /// Enable querying local files as tables. /// - /// This allows queries to directly access arbitrary file names via SQL like - /// `SELECT * from 'my_file.parquet'` - /// so it should only be enabled for systems that such access is not a security risk + /// This feature is security sensitive and should only be enabled for + /// systems that wish to permit direct access to the file system from SQL. + /// + /// When enabled, this feature permits direct access to arbitrary files via + /// SQL like + /// + /// ```sql + /// SELECT * from 'my_file.parquet' + /// ``` /// /// See [DynamicFileCatalog] for more details /// @@ -356,7 +362,8 @@ impl SessionContext { /// # use datafusion::{error::Result, assert_batches_eq}; /// # #[tokio::main] /// # async fn main() -> Result<()> { - /// let ctx = SessionContext::new().enable_url_table(); + /// let ctx = SessionContext::new() + /// .enable_url_table(); // permit local file access /// let results = ctx /// .sql("SELECT a, MIN(b) FROM 'tests/data/example.csv' as example GROUP BY a LIMIT 100") /// .await? @@ -375,21 +382,60 @@ impl SessionContext { /// # Ok(()) /// # } /// ``` - pub fn enable_url_table(&self) -> Self { - let state_ref = self.state(); + pub fn enable_url_table(self) -> Self { + let current_catalog_list = Arc::clone(self.state.read().catalog_list()); let factory = Arc::new(DynamicListTableFactory::new(SessionStore::new())); let catalog_list = Arc::new(DynamicFileCatalog::new( - Arc::clone(state_ref.catalog_list()), + current_catalog_list, Arc::clone(&factory) as Arc, )); - let new_state = SessionStateBuilder::new_from_existing(self.state()) + let ctx: SessionContext = self + .into_state_builder() .with_catalog_list(catalog_list) - .build(); - let ctx = SessionContext::new_with_state(new_state); + .build() + .into(); + // register new state with the factory factory.session_store().with_state(ctx.state_weak_ref()); ctx } + /// Convert the current `SessionContext` into a [`SessionStateBuilder`] + /// + /// This is useful to switch back to `SessionState` with custom settings such as + /// [`Self::enable_url_table`]. + /// + /// Avoids cloning the SessionState if possible. + /// + /// # Example + /// ``` + /// # use std::sync::Arc; + /// # use datafusion::prelude::*; + /// # use datafusion::execution::SessionStateBuilder; + /// # use datafusion_optimizer::push_down_filter::PushDownFilter; + /// let my_rule = PushDownFilter{}; // pretend it is a new rule + /// // Create a new builder with a custom optimizer rule + /// let context: SessionContext = SessionStateBuilder::new() + /// .with_optimizer_rule(Arc::new(my_rule)) + /// .build() + /// .into(); + /// // Enable local file access and convert context back to a builder + /// let builder = context + /// .enable_url_table() + /// .into_state_builder(); + /// ``` + pub fn into_state_builder(self) -> SessionStateBuilder { + let SessionContext { + session_id: _, + session_start_time: _, + state, + } = self; + let state = match Arc::try_unwrap(state) { + Ok(rwlock) => rwlock.into_inner(), + Err(state) => state.read().clone(), + }; + SessionStateBuilder::from(state) + } + /// Returns the time this `SessionContext` was created pub fn session_start_time(&self) -> DateTime { self.session_start_time @@ -1496,6 +1542,12 @@ impl From for SessionContext { } } +impl From for SessionStateBuilder { + fn from(session: SessionContext) -> Self { + session.into_state_builder() + } +} + /// A planner used to add extensions to DataFusion logical and physical plans. #[async_trait] pub trait QueryPlanner { From 7dc01515cef4300e8da8467065fdfe4a4528cddc Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Sun, 22 Sep 2024 19:50:09 +0800 Subject: [PATCH 12/25] Reorgnize the StringView tests in sqllogictests (#12572) * move the string function tests * move the operator test * add back the test for dictionary string --- .../test_files/string/string_literal.slt | 22 + .../test_files/string/string_query.slt.part | 368 ++++++ .../test_files/string/string_view.slt | 646 ++++++++++ .../sqllogictest/test_files/string_view.slt | 1093 ----------------- 4 files changed, 1036 insertions(+), 1093 deletions(-) delete mode 100644 datafusion/sqllogictest/test_files/string_view.slt diff --git a/datafusion/sqllogictest/test_files/string/string_literal.slt b/datafusion/sqllogictest/test_files/string/string_literal.slt index 37dc1a8ce84d..7cb921a6603e 100644 --- a/datafusion/sqllogictest/test_files/string/string_literal.slt +++ b/datafusion/sqllogictest/test_files/string/string_literal.slt @@ -145,3 +145,25 @@ select arrow_cast('Xiangpeng', 'Utf8View') <> arrow_cast('Andrew', 'Utf8View'); ---- false true true true + + +query II +SELECT + ASCII('hello'), + ASCII(arrow_cast('world', 'Utf8View')) +---- +104 119 + +query III +SELECT + ASCII(arrow_cast('äöüß', 'Utf8View')) as c1, + ASCII(arrow_cast('', 'Utf8View')) as c2, + ASCII(arrow_cast(NULL, 'Utf8View')) as c3 +---- +228 0 NULL + +# coercion from stringview to integer, as input to make_date +query D +select make_date(arrow_cast('2024', 'Utf8View'), arrow_cast('01', 'Utf8View'), arrow_cast('23', 'Utf8View')) +---- +2024-01-23 diff --git a/datafusion/sqllogictest/test_files/string/string_query.slt.part b/datafusion/sqllogictest/test_files/string/string_query.slt.part index 4f737954000b..f22edae01de9 100644 --- a/datafusion/sqllogictest/test_files/string/string_query.slt.part +++ b/datafusion/sqllogictest/test_files/string/string_query.slt.part @@ -164,6 +164,19 @@ ThisIsAVeryLongASCIIString isIsAVeryLongASCIIString (empty) ThisIsAVeryLongASCII (empty) (empty) (empty) (empty) (empty) (empty) (empty) (empty) (empty) (empty) (empty) (empty) (empty) (empty) NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL +query TTTT +SELECT + SUBSTR(ascii_1, 1, 3) as c1, + SUBSTR(ascii_2, 1, 3) as c2, + SUBSTR(unicode_1, 1, 3) as c3, + SUBSTR(unicode_2, 1, 3) as c4 +FROM test_basic_operator; +---- +And X dat 🔥 +Xia Xia dat dat +Rap R dat аФу +NULL R NULL 🔥 + # -------------------------------------- # test distinct aggregate # -------------------------------------- @@ -324,3 +337,358 @@ NULL NULL statement ok drop table test_lowercase; + +# -------------------------------------- +# Test ASCII +# -------------------------------------- + +query IIII +SELECT + ASCII(ascii_1) as c1, + ASCII(ascii_2) as c2, + ASCII(unicode_1) as c3, + ASCII(unicode_2) as c4 +FROM test_basic_operator; +---- +65 88 100 128293 +88 88 100 100 +82 82 100 1072 +NULL 82 NULL 128293 + +# -------------------------------------- +# Test BTRIM +# -------------------------------------- + +# Test BTRIM outputs +query TTTTTT +SELECT + BTRIM(ascii_1, 'foo'), + BTRIM(ascii_1, 'A'), + BTRIM(ascii_1, NULL), + BTRIM(unicode_1), + BTRIM(unicode_1, '🔥'), + BTRIM(unicode_1, NULL) +FROM test_basic_operator; +---- +Andrew ndrew NULL datafusion📊🔥 datafusion📊 NULL +Xiangpeng Xiangpeng NULL datafusion数据融合 datafusion数据融合 NULL +Raphael Raphael NULL datafusionДатаФусион datafusionДатаФусион NULL +NULL NULL NULL NULL NULL NULL + +# -------------------------------------- +# Test LTRIM +# -------------------------------------- + +# Test LTRIM outputs +query TTTTTT +SELECT + LTRIM(ascii_1, 'foo'), + LTRIM(ascii_1, ascii_2), + LTRIM(ascii_1, NULL), + LTRIM(unicode_1), + LTRIM(unicode_1, NULL), + LTRIM(unicode_1, '🔥') +FROM test_basic_operator; +---- +Andrew Andrew NULL datafusion📊🔥 NULL datafusion📊🔥 +Xiangpeng (empty) NULL datafusion数据融合 NULL datafusion数据融合 +Raphael aphael NULL datafusionДатаФусион NULL datafusionДатаФусион +NULL NULL NULL NULL NULL NULL + +# -------------------------------------- +# Test RTRIM +# -------------------------------------- + +# Test RTRIM outputs +query TTTTT +SELECT + RTRIM(ascii_1, 'rew'), + RTRIM(ascii_1, ascii_2), + RTRIM(ascii_1), + RTRIM(unicode_1, NULL), + RTRIM(unicode_1, '🔥') +FROM test_basic_operator; +---- +And Andrew Andrew NULL datafusion📊 +Xiangpeng (empty) Xiangpeng NULL datafusion数据融合 +Raphael Raphael Raphael NULL datafusionДатаФусион +NULL NULL NULL NULL NULL + +# -------------------------------------- +# Test CONTAINS +# -------------------------------------- + +query BBBBBB +SELECT + CONTAINS(ascii_1, 'foo') as c1, + CONTAINS(ascii_1, ascii_2) as c2, + CONTAINS(ascii_1, NULL) as c3, + CONTAINS(unicode_1, unicode_2) as c4, + CONTAINS(unicode_1, NULL) as c5, + CONTAINS(unicode_1, '🔥') as c6 +FROM test_basic_operator; +---- +false false NULL true NULL true +false true NULL true NULL false +false true NULL true NULL false +NULL NULL NULL NULL NULL NULL + +# -------------------------------------- +# Test LOWER +# -------------------------------------- + +query TT +SELECT LOWER(ascii_1) as c1, LOWER(unicode_1) as c2 FROM test_basic_operator; +---- +andrew datafusion📊🔥 +xiangpeng datafusion数据融合 +raphael datafusionдатафусион +NULL NULL + +# -------------------------------------- +# Test UPPER +# -------------------------------------- + +query TT +SELECT UPPER(ascii_1) as c1, UPPER(unicode_1) as c2 FROM test_basic_operator; +---- +ANDREW DATAFUSION📊🔥 +XIANGPENG DATAFUSION数据融合 +RAPHAEL DATAFUSIONДАТАФУСИОН +NULL NULL + +# -------------------------------------- +# Test Concat +# -------------------------------------- + +query TTTTTTTTTTTT +SELECT + concat(ascii_1, ':Data'), + concat(ascii_1, ascii_2), + concat(ascii_1, NULL), + concat(ascii_1, unicode_1), + concat(ascii_1, unicode_2), + concat(unicode_1, ascii_1), + concat(unicode_1, unicode_2), + concat(unicode_1, NULL), + concat(unicode_1, '🔥'), + concat(NULL, '🔥'), + concat(NULL, NULL), + concat(ascii_1, ',', unicode_1) +FROM test_basic_operator; +---- +Andrew:Data AndrewX Andrew Andrewdatafusion📊🔥 Andrew🔥 datafusion📊🔥Andrew datafusion📊🔥🔥 datafusion📊🔥 datafusion📊🔥🔥 🔥 (empty) Andrew,datafusion📊🔥 +Xiangpeng:Data XiangpengXiangpeng Xiangpeng Xiangpengdatafusion数据融合 Xiangpengdatafusion数据融合 datafusion数据融合Xiangpeng datafusion数据融合datafusion数据融合 datafusion数据融合 datafusion数据融合🔥 🔥 (empty) Xiangpeng,datafusion数据融合 +Raphael:Data RaphaelR Raphael RaphaeldatafusionДатаФусион RaphaelаФус datafusionДатаФусионRaphael datafusionДатаФусионаФус datafusionДатаФусион datafusionДатаФусион🔥 🔥 (empty) Raphael,datafusionДатаФусион +:Data R (empty) (empty) 🔥 (empty) 🔥 (empty) 🔥 🔥 (empty) , + +# -------------------------------------- +# Test OVERLAY +# -------------------------------------- + +query TTTTTT +SELECT + OVERLAY(ascii_1 PLACING 'foo' FROM 2 ), + OVERLAY(unicode_1 PLACING 'foo' FROM 2), + OVERLAY(ascii_1 PLACING '🔥' FROM 2), + OVERLAY(unicode_1 PLACING '🔥' FROM 2), + OVERLAY(ascii_1 PLACING NULL FROM 2), + OVERLAY(unicode_1 PLACING NULL FROM 2) +FROM test_basic_operator; +---- +Afooew dfoofusion📊🔥 A🔥drew d🔥tafusion📊🔥 NULL NULL +Xfoogpeng dfoofusion数据融合 X🔥angpeng d🔥tafusion数据融合 NULL NULL +Rfooael dfoofusionДатаФусион R🔥phael d🔥tafusionДатаФусион NULL NULL +NULL NULL NULL NULL NULL NULL + +# -------------------------------------- +# Test REPLACE +# -------------------------------------- + +query TTTTTT +SELECT + REPLACE(ascii_1, 'foo', 'bar'), + REPLACE(ascii_1, ascii_2, 'bar'), + REPLACE(ascii_1, NULL, 'bar'), + REPLACE(unicode_1, unicode_2, 'bar'), + REPLACE(unicode_1, NULL, 'bar'), + REPLACE(unicode_1, '🔥', 'bar') +FROM test_basic_operator; +---- +Andrew Andrew NULL datafusion📊bar NULL datafusion📊bar +Xiangpeng bar NULL bar NULL datafusion数据融合 +Raphael baraphael NULL datafusionДатbarион NULL datafusionДатаФусион +NULL NULL NULL NULL NULL NULL + +# -------------------------------------- +# Test RIGHT +# -------------------------------------- +# Test outputs of RIGHT +query TTTTTT +SELECT + RIGHT(ascii_1, 3), + RIGHT(ascii_1, 0), + RIGHT(ascii_1, -3), + RIGHT(unicode_1, 3), + RIGHT(unicode_1, 0), + RIGHT(unicode_1, -3) +FROM test_basic_operator; +---- +rew (empty) rew n📊🔥 (empty) afusion📊🔥 +eng (empty) ngpeng 据融合 (empty) afusion数据融合 +ael (empty) hael ион (empty) afusionДатаФусион +NULL NULL NULL NULL NULL NULL + +# -------------------------------------- +# Test LEFT +# -------------------------------------- + +# Test outputs of LEFT +query TTTTTT +SELECT + LEFT(ascii_1, 3), + LEFT(ascii_1, 0), + LEFT(ascii_1, -3), + LEFT(unicode_1, 3), + LEFT(unicode_1, 0), + LEFT(unicode_1, -3) +FROM test_basic_operator; +---- +And (empty) And dat (empty) datafusio +Xia (empty) Xiangp dat (empty) datafusion数 +Rap (empty) Raph dat (empty) datafusionДатаФус +NULL NULL NULL NULL NULL NULL + +# -------------------------------------- +# Test SUBSTR_INDEX +# -------------------------------------- + +query TTTT +SELECT + SUBSTR_INDEX(ascii_1, 'a', 1), + SUBSTR_INDEX(ascii_1, 'a', 2), + SUBSTR_INDEX(unicode_1, 'а', 1), + SUBSTR_INDEX(unicode_1, 'а', 2) +FROM test_basic_operator; +---- +Andrew Andrew datafusion📊🔥 datafusion📊🔥 +Xi Xiangpeng datafusion数据融合 datafusion数据融合 +R Raph datafusionД datafusionДат +NULL NULL NULL NULL + +# -------------------------------------- +# Test FIND_IN_SET +# -------------------------------------- + +query IIII +SELECT + FIND_IN_SET(ascii_1, 'a,b,c,d'), + FIND_IN_SET(ascii_1, 'Andrew,Xiangpeng,Raphael'), + FIND_IN_SET(unicode_1, 'a,b,c,d'), + FIND_IN_SET(unicode_1, 'datafusion📊🔥,datafusion数据融合,datafusionДатаФусион') +FROM test_basic_operator; +---- +0 1 0 1 +0 2 0 2 +0 3 0 3 +NULL NULL NULL NULL + +# -------------------------------------- +# Test || operator +# -------------------------------------- + +# || constants +# expect all results to be the same for each row as they all have the same values +query TTTT +SELECT + ascii_1 || 'foo', + ascii_1 || '🔥', + unicode_1 || 'foo', + unicode_1 || '🔥' +FROM test_basic_operator; +---- +Andrewfoo Andrew🔥 datafusion📊🔥foo datafusion📊🔥🔥 +Xiangpengfoo Xiangpeng🔥 datafusion数据融合foo datafusion数据融合🔥 +Raphaelfoo Raphael🔥 datafusionДатаФусионfoo datafusionДатаФусион🔥 +NULL NULL NULL NULL + +# || same type (column1 has null, so also tests NULL || NULL) +# expect all results to be the same for each row as they all have the same values +query TTTT +SELECT + ascii_1 || ascii_2, + ascii_1 || unicode_2, + unicode_1 || ascii_2, + unicode_1 || unicode_2 +FROM test_basic_operator; +---- +AndrewX Andrew🔥 datafusion📊🔥X datafusion📊🔥🔥 +XiangpengXiangpeng Xiangpengdatafusion数据融合 datafusion数据融合Xiangpeng datafusion数据融合datafusion数据融合 +RaphaelR RaphaelаФус datafusionДатаФусионR datafusionДатаФусионаФус +NULL NULL NULL NULL + +# -------------------------------------- +# Test ~ operator +# -------------------------------------- + +query BB +SELECT + ascii_1 ~ 'an', + unicode_1 ~ 'таФ' +FROM test_basic_operator; +---- +false false +true false +false true +NULL NULL + +query BB +SELECT + ascii_1 ~* '^a.{3}e', + unicode_1 ~* '^d.*Фу' +FROM test_basic_operator; +---- +true false +false false +false true +NULL NULL + +query BB +SELECT + ascii_1 !~~ 'xia_g%g', + unicode_1 !~~ 'datafusion数据融合' +FROM test_basic_operator; +---- +true true +true false +true true +NULL NULL + +query BB +SELECT + ascii_1 !~~* 'xia_g%g', + unicode_1 !~~* 'datafusion数据融合' +FROM test_basic_operator; +---- +true true +false false +true true +NULL NULL + +# -------------------------------------- +# Test || operator +# -------------------------------------- + +query TTTTT +select + ascii_1 || ' nice', + ascii_1 || ' and ' || ascii_2, + unicode_1 || ' cool', + unicode_1 || ' and ' || unicode_2, + ascii_1 || ' 🔥 ' || unicode_1 +from test_basic_operator; +---- +Andrew nice Andrew and X datafusion📊🔥 cool datafusion📊🔥 and 🔥 Andrew 🔥 datafusion📊🔥 +Xiangpeng nice Xiangpeng and Xiangpeng datafusion数据融合 cool datafusion数据融合 and datafusion数据融合 Xiangpeng 🔥 datafusion数据融合 +Raphael nice Raphael and R datafusionДатаФусион cool datafusionДатаФусион and аФус Raphael 🔥 datafusionДатаФусион +NULL NULL NULL NULL NULL diff --git a/datafusion/sqllogictest/test_files/string/string_view.slt b/datafusion/sqllogictest/test_files/string/string_view.slt index ca08f647d246..da7d99cc311b 100644 --- a/datafusion/sqllogictest/test_files/string/string_view.slt +++ b/datafusion/sqllogictest/test_files/string/string_view.slt @@ -352,5 +352,651 @@ logical_plan statement ok drop table test_lowercase +# Ensure string functions use native StringView implementation +# and do not fall back to Utf8 or LargeUtf8 +# Should see no casts to Utf8 in the plans below + +## Ensure no casts for LIKE/ILIKE +query TT +EXPLAIN SELECT + column1_utf8view like 'foo' as "like", + column1_utf8view ilike 'foo' as "ilike" +FROM test; +---- +logical_plan +01)Projection: test.column1_utf8view LIKE Utf8View("foo") AS like, test.column1_utf8view ILIKE Utf8View("foo") AS ilike +02)--TableScan: test projection=[column1_utf8view] + + +query TT +EXPLAIN SELECT + SUBSTR(column1_utf8view, 1, 3) as c1, + SUBSTR(column2_utf8, 1, 3) as c2, + SUBSTR(column2_large_utf8, 1, 3) as c3 +FROM test; +---- +logical_plan +01)Projection: substr(test.column1_utf8view, Int64(1), Int64(3)) AS c1, substr(test.column2_utf8, Int64(1), Int64(3)) AS c2, substr(test.column2_large_utf8, Int64(1), Int64(3)) AS c3 +02)--TableScan: test projection=[column2_utf8, column2_large_utf8, column1_utf8view] + +## Ensure no casts for SUBSTR + +query TT +EXPLAIN SELECT + SUBSTR(column1_utf8view, 1, 3) as c1, + SUBSTR(column2_utf8, 1, 3) as c2, + SUBSTR(column2_large_utf8, 1, 3) as c3 +FROM test; +---- +logical_plan +01)Projection: substr(test.column1_utf8view, Int64(1), Int64(3)) AS c1, substr(test.column2_utf8, Int64(1), Int64(3)) AS c2, substr(test.column2_large_utf8, Int64(1), Int64(3)) AS c3 +02)--TableScan: test projection=[column2_utf8, column2_large_utf8, column1_utf8view] + +# Test ASCII with utf8view against utf8view, utf8, and largeutf8 +# (should be no casts) +query TT +EXPLAIN SELECT + ASCII(column1_utf8view) as c1, + ASCII(column2_utf8) as c2, + ASCII(column2_large_utf8) as c3 +FROM test; +---- +logical_plan +01)Projection: ascii(test.column1_utf8view) AS c1, ascii(test.column2_utf8) AS c2, ascii(test.column2_large_utf8) AS c3 +02)--TableScan: test projection=[column2_utf8, column2_large_utf8, column1_utf8view] + +query TT +EXPLAIN SELECT + ASCII(column1_utf8) as c1, + ASCII(column1_large_utf8) as c2, + ASCII(column2_utf8view) as c3, + ASCII('hello') as c4, + ASCII(arrow_cast('world', 'Utf8View')) as c5 +FROM test; +---- +logical_plan +01)Projection: ascii(test.column1_utf8) AS c1, ascii(test.column1_large_utf8) AS c2, ascii(test.column2_utf8view) AS c3, Int32(104) AS c4, Int32(119) AS c5 +02)--TableScan: test projection=[column1_utf8, column1_large_utf8, column2_utf8view] + +# Test ASCII with literals cast to Utf8View +query TT +EXPLAIN SELECT + ASCII(arrow_cast('äöüß', 'Utf8View')) as c1, + ASCII(arrow_cast('', 'Utf8View')) as c2, + ASCII(arrow_cast(NULL, 'Utf8View')) as c3 +FROM test; +---- +logical_plan +01)Projection: Int32(228) AS c1, Int32(0) AS c2, Int32(NULL) AS c3 +02)--TableScan: test projection=[] + +## Ensure no casts for BTRIM +# Test BTRIM with Utf8View input +query TT +EXPLAIN SELECT + BTRIM(column1_utf8view) AS l +FROM test; +---- +logical_plan +01)Projection: btrim(test.column1_utf8view) AS l +02)--TableScan: test projection=[column1_utf8view] + +# Test BTRIM with Utf8View input and Utf8View pattern +query TT +EXPLAIN SELECT + BTRIM(column1_utf8view, 'foo') AS l +FROM test; +---- +logical_plan +01)Projection: btrim(test.column1_utf8view, Utf8View("foo")) AS l +02)--TableScan: test projection=[column1_utf8view] + +# Test BTRIM with Utf8View bytes longer than 12 +query TT +EXPLAIN SELECT + BTRIM(column1_utf8view, 'this is longer than 12') AS l +FROM test; +---- +logical_plan +01)Projection: btrim(test.column1_utf8view, Utf8View("this is longer than 12")) AS l +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for LTRIM +# Test LTRIM with Utf8View input +query TT +EXPLAIN SELECT + LTRIM(column1_utf8view) AS l +FROM test; +---- +logical_plan +01)Projection: ltrim(test.column1_utf8view) AS l +02)--TableScan: test projection=[column1_utf8view] + +# Test LTRIM with Utf8View input and Utf8View pattern +query TT +EXPLAIN SELECT + LTRIM(column1_utf8view, 'foo') AS l +FROM test; +---- +logical_plan +01)Projection: ltrim(test.column1_utf8view, Utf8View("foo")) AS l +02)--TableScan: test projection=[column1_utf8view] + +# Test LTRIM with Utf8View bytes longer than 12 +query TT +EXPLAIN SELECT + LTRIM(column1_utf8view, 'this is longer than 12') AS l +FROM test; +---- +logical_plan +01)Projection: ltrim(test.column1_utf8view, Utf8View("this is longer than 12")) AS l +02)--TableScan: test projection=[column1_utf8view] + +## ensure no casts for RTRIM +# Test RTRIM with Utf8View input +query TT +EXPLAIN SELECT + RTRIM(column1_utf8view) AS l +FROM test; +---- +logical_plan +01)Projection: rtrim(test.column1_utf8view) AS l +02)--TableScan: test projection=[column1_utf8view] + +# Test RTRIM with Utf8View input and Utf8View pattern +query TT +EXPLAIN SELECT + RTRIM(column1_utf8view, 'foo') AS l +FROM test; +---- +logical_plan +01)Projection: rtrim(test.column1_utf8view, Utf8View("foo")) AS l +02)--TableScan: test projection=[column1_utf8view] + +# Test RTRIM with Utf8View bytes longer than 12 +query TT +EXPLAIN SELECT + RTRIM(column1_utf8view, 'this is longer than 12') AS l +FROM test; +---- +logical_plan +01)Projection: rtrim(test.column1_utf8view, Utf8View("this is longer than 12")) AS l +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for CHARACTER_LENGTH +query TT +EXPLAIN SELECT + CHARACTER_LENGTH(column1_utf8view) AS l +FROM test; +---- +logical_plan +01)Projection: character_length(test.column1_utf8view) AS l +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for CONCAT Utf8View +query TT +EXPLAIN SELECT + concat(column1_utf8view, column2_utf8view) as c +FROM test; +---- +logical_plan +01)Projection: concat(test.column1_utf8view, test.column2_utf8view) AS c +02)--TableScan: test projection=[column1_utf8view, column2_utf8view] + +## Ensure no casts for CONCAT LargeUtf8 +query TT +EXPLAIN SELECT + concat(column1_large_utf8, column2_large_utf8) as c +FROM test; +---- +logical_plan +01)Projection: concat(test.column1_large_utf8, test.column2_large_utf8) AS c +02)--TableScan: test projection=[column1_large_utf8, column2_large_utf8] + +## Ensure no casts for CONCAT_WS +query TT +EXPLAIN SELECT + concat_ws(', ', column1_utf8view, column2_utf8view) as c +FROM test; +---- +logical_plan +01)Projection: concat_ws(Utf8(", "), test.column1_utf8view, test.column2_utf8view) AS c +02)--TableScan: test projection=[column1_utf8view, column2_utf8view] + +## Ensure no casts for CONTAINS +query TT +EXPLAIN SELECT + CONTAINS(column1_utf8view, 'foo') as c1, + CONTAINS(column1_utf8view, column2_utf8view) as c2, + CONTAINS(column1_utf8view, column2_large_utf8) as c3, + CONTAINS(column1_utf8, column2_utf8view) as c4, + CONTAINS(column1_utf8, column2_utf8) as c5, + CONTAINS(column1_utf8, column2_large_utf8) as c6, + CONTAINS(column1_large_utf8, column1_utf8view) as c7, + CONTAINS(column1_large_utf8, column2_utf8) as c8, + CONTAINS(column1_large_utf8, column2_large_utf8) as c9 +FROM test; +---- +logical_plan +01)Projection: contains(test.column1_utf8view, Utf8("foo")) AS c1, contains(test.column1_utf8view, test.column2_utf8view) AS c2, contains(test.column1_utf8view, test.column2_large_utf8) AS c3, contains(test.column1_utf8, test.column2_utf8view) AS c4, contains(test.column1_utf8, test.column2_utf8) AS c5, contains(test.column1_utf8, test.column2_large_utf8) AS c6, contains(test.column1_large_utf8, test.column1_utf8view) AS c7, contains(test.column1_large_utf8, test.column2_utf8) AS c8, contains(test.column1_large_utf8, test.column2_large_utf8) AS c9 +02)--TableScan: test projection=[column1_utf8, column2_utf8, column1_large_utf8, column2_large_utf8, column1_utf8view, column2_utf8view] + +## Ensure no casts for ENDS_WITH +query TT +EXPLAIN SELECT + ENDS_WITH(column1_utf8view, 'foo') as c1, + ENDS_WITH(column2_utf8view, column2_utf8view) as c2 +FROM test; +---- +logical_plan +01)Projection: ends_with(test.column1_utf8view, Utf8View("foo")) AS c1, ends_with(test.column2_utf8view, test.column2_utf8view) AS c2 +02)--TableScan: test projection=[column1_utf8view, column2_utf8view] + +## Ensure no casts for LEVENSHTEIN +query TT +EXPLAIN SELECT + levenshtein(column1_utf8view, 'foo') as c1, + levenshtein(column1_utf8view, column2_utf8view) as c2 +FROM test; +---- +logical_plan +01)Projection: levenshtein(test.column1_utf8view, Utf8View("foo")) AS c1, levenshtein(test.column1_utf8view, test.column2_utf8view) AS c2 +02)--TableScan: test projection=[column1_utf8view, column2_utf8view] + +## Ensure no casts for LOWER +query TT +EXPLAIN SELECT + LOWER(column1_utf8view) as c1 +FROM test; +---- +logical_plan +01)Projection: lower(test.column1_utf8view) AS c1 +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for UPPER +query TT +EXPLAIN SELECT + UPPER(column1_utf8view) as c1 +FROM test; +---- +logical_plan +01)Projection: upper(test.column1_utf8view) AS c1 +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for LPAD +query TT +EXPLAIN SELECT + LPAD(column1_utf8view, 12, ' ') as c1 +FROM test; +---- +logical_plan +01)Projection: lpad(test.column1_utf8view, Int64(12), Utf8(" ")) AS c1 +02)--TableScan: test projection=[column1_utf8view] + +query TT +EXPLAIN SELECT + LPAD(column1_utf8view, 12, column2_large_utf8) as c1 +FROM test; +---- +logical_plan +01)Projection: lpad(test.column1_utf8view, Int64(12), test.column2_large_utf8) AS c1 +02)--TableScan: test projection=[column2_large_utf8, column1_utf8view] + +query TT +EXPLAIN SELECT + LPAD(column1_utf8view, 12, column2_utf8view) as c1 +FROM test; +---- +logical_plan +01)Projection: lpad(test.column1_utf8view, Int64(12), test.column2_utf8view) AS c1 +02)--TableScan: test projection=[column1_utf8view, column2_utf8view] + +## Ensure no casts for OCTET_LENGTH +query TT +EXPLAIN SELECT + OCTET_LENGTH(column1_utf8view) as c1 +FROM test; +---- +logical_plan +01)Projection: octet_length(test.column1_utf8view) AS c1 +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for OVERLAY +query TT +EXPLAIN SELECT + OVERLAY(column1_utf8view PLACING 'foo' FROM 2 ) as c1 +FROM test; +---- +logical_plan +01)Projection: overlay(test.column1_utf8view, Utf8View("foo"), Int64(2)) AS c1 +02)--TableScan: test projection=[column1_utf8view] + +## Should run CONCAT successfully with utf8 and utf8view +query T +SELECT + concat(column1_utf8view, column2_utf8) as c +FROM test; +---- +AndrewX +XiangpengXiangpeng +RaphaelR +R + +## Should run CONCAT successfully with utf8 utf8view and largeutf8 +query T +SELECT + concat(column1_utf8view, column2_utf8, column2_large_utf8) as c +FROM test; +---- +AndrewXX +XiangpengXiangpengXiangpeng +RaphaelRR +RR + +## Ensure no casts for REGEXP_LIKE +query TT +EXPLAIN SELECT + REGEXP_LIKE(column1_utf8view, '^https?://(?:www\.)?([^/]+)/.*$') AS k +FROM test; +---- +logical_plan +01)Projection: regexp_like(CAST(test.column1_utf8view AS Utf8), Utf8("^https?://(?:www\.)?([^/]+)/.*$")) AS k +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for REGEXP_MATCH +query TT +EXPLAIN SELECT + REGEXP_MATCH(column1_utf8view, '^https?://(?:www\.)?([^/]+)/.*$') AS k +FROM test; +---- +logical_plan +01)Projection: regexp_match(CAST(test.column1_utf8view AS Utf8), Utf8("^https?://(?:www\.)?([^/]+)/.*$")) AS k +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for REGEXP_REPLACE +query TT +EXPLAIN SELECT + REGEXP_REPLACE(column1_utf8view, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k +FROM test; +---- +logical_plan +01)Projection: regexp_replace(test.column1_utf8view, Utf8("^https?://(?:www\.)?([^/]+)/.*$"), Utf8("\1")) AS k +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for REPEAT +query TT +EXPLAIN SELECT + REPEAT(column1_utf8view, 2) as c1 +FROM test; +---- +logical_plan +01)Projection: repeat(test.column1_utf8view, Int64(2)) AS c1 +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for REPLACE +query TT +EXPLAIN SELECT + REPLACE(column1_utf8view, 'foo', 'bar') as c1, + REPLACE(column1_utf8view, column2_utf8view, 'bar') as c2 +FROM test; +---- +logical_plan +01)Projection: replace(test.column1_utf8view, Utf8View("foo"), Utf8View("bar")) AS c1, replace(test.column1_utf8view, test.column2_utf8view, Utf8View("bar")) AS c2 +02)--TableScan: test projection=[column1_utf8view, column2_utf8view] + +## Ensure no casts for REVERSE +query TT +EXPLAIN SELECT + REVERSE(column1_utf8view) as c1 +FROM test; +---- +logical_plan +01)Projection: reverse(test.column1_utf8view) AS c1 +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for RIGHT +query TT +EXPLAIN SELECT + RIGHT(column1_utf8view, 3) as c2 +FROM test; +---- +logical_plan +01)Projection: right(test.column1_utf8view, Int64(3)) AS c2 +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for LEFT +query TT +EXPLAIN SELECT + LEFT(column1_utf8view, 3) as c2 +FROM test; +---- +logical_plan +01)Projection: left(test.column1_utf8view, Int64(3)) AS c2 +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for RPAD +query TT +EXPLAIN SELECT + RPAD(column1_utf8view, 1) as c1, + RPAD(column1_utf8view, 2, column2_utf8view) as c2 +FROM test; +---- +logical_plan +01)Projection: rpad(test.column1_utf8view, Int64(1)) AS c1, rpad(test.column1_utf8view, Int64(2), test.column2_utf8view) AS c2 +02)--TableScan: test projection=[column1_utf8view, column2_utf8view] + +query TT +EXPLAIN SELECT + RPAD(column1_utf8view, 12, column2_large_utf8) as c1 +FROM test; +---- +logical_plan +01)Projection: rpad(test.column1_utf8view, Int64(12), test.column2_large_utf8) AS c1 +02)--TableScan: test projection=[column2_large_utf8, column1_utf8view] + +query TT +EXPLAIN SELECT + RPAD(column1_utf8view, 12, column2_utf8view) as c1 +FROM test; +---- +logical_plan +01)Projection: rpad(test.column1_utf8view, Int64(12), test.column2_utf8view) AS c1 +02)--TableScan: test projection=[column1_utf8view, column2_utf8view] + +## Ensure no casts for SPLIT_PART +query TT +EXPLAIN SELECT + SPLIT_PART(column1_utf8view, 'f', 1) as c1, + SPLIT_PART('testtesttest',column1_utf8view, 1) as c2 +FROM test; +---- +logical_plan +01)Projection: split_part(test.column1_utf8view, Utf8("f"), Int64(1)) AS c1, split_part(Utf8("testtesttest"), test.column1_utf8view, Int64(1)) AS c2 +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for STRPOS +query TT +EXPLAIN SELECT + STRPOS(column1_utf8view, 'f') as c, + STRPOS(column1_utf8view, column2_utf8view) as c2 +FROM test; +---- +logical_plan +01)Projection: strpos(test.column1_utf8view, Utf8("f")) AS c, strpos(test.column1_utf8view, test.column2_utf8view) AS c2 +02)--TableScan: test projection=[column1_utf8view, column2_utf8view] + +## Ensure no casts for SUBSTR +query TT +EXPLAIN SELECT + SUBSTR(column1_utf8view, 1) as c, + SUBSTR(column1_utf8view, 1 ,2) as c2 +FROM test; +---- +logical_plan +01)Projection: substr(test.column1_utf8view, Int64(1)) AS c, substr(test.column1_utf8view, Int64(1), Int64(2)) AS c2 +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for SUBSTRINDEX +query TT +EXPLAIN SELECT + SUBSTR_INDEX(column1_utf8view, 'a', 1) as c, + SUBSTR_INDEX(column1_utf8view, 'a', 2) as c2 +FROM test; +---- +logical_plan +01)Projection: substr_index(test.column1_utf8view, Utf8View("a"), Int64(1)) AS c, substr_index(test.column1_utf8view, Utf8View("a"), Int64(2)) AS c2 +02)--TableScan: test projection=[column1_utf8view] + + +## Ensure no casts on columns for STARTS_WITH +query TT +EXPLAIN SELECT + STARTS_WITH(column1_utf8view, 'foo') as c, + STARTS_WITH(column1_utf8view, column2_utf8view) as c2 +FROM test; +---- +logical_plan +01)Projection: starts_with(test.column1_utf8view, Utf8View("foo")) AS c, starts_with(test.column1_utf8view, test.column2_utf8view) AS c2 +02)--TableScan: test projection=[column1_utf8view, column2_utf8view] + +## Ensure no casts for TRANSLATE +query TT +EXPLAIN SELECT + TRANSLATE(column1_utf8view, 'foo', 'bar') as c +FROM test; +---- +logical_plan +01)Projection: translate(test.column1_utf8view, Utf8("foo"), Utf8("bar")) AS c +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for FIND_IN_SET +query TT +EXPLAIN SELECT + FIND_IN_SET(column1_utf8view, 'a,b,c,d') as c +FROM test; +---- +logical_plan +01)Projection: find_in_set(test.column1_utf8view, Utf8View("a,b,c,d")) AS c +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no casts for binary operators +# `~` operator (regex match) +query TT +EXPLAIN SELECT + column1_utf8view ~ 'an' AS c1 +FROM test; +---- +logical_plan +01)Projection: CAST(test.column1_utf8view AS Utf8) LIKE Utf8("%an%") AS c1 +02)--TableScan: test projection=[column1_utf8view] + +# `~*` operator (regex match case-insensitive) +query TT +EXPLAIN SELECT + column1_utf8view ~* '^a.{3}e' AS c1 +FROM test; +---- +logical_plan +01)Projection: CAST(test.column1_utf8view AS Utf8) ~* Utf8("^a.{3}e") AS c1 +02)--TableScan: test projection=[column1_utf8view] + +# `!~~` operator (not like match) +query TT +EXPLAIN SELECT + column1_utf8view !~~ 'xia_g%g' AS c1 +FROM test; +---- +logical_plan +01)Projection: CAST(test.column1_utf8view AS Utf8) !~~ Utf8("xia_g%g") AS c1 +02)--TableScan: test projection=[column1_utf8view] + +# `!~~*` operator (not like match case-insensitive) +query TT +EXPLAIN SELECT + column1_utf8view !~~* 'xia_g%g' AS c1 +FROM test; +---- +logical_plan +01)Projection: CAST(test.column1_utf8view AS Utf8) !~~* Utf8("xia_g%g") AS c1 +02)--TableScan: test projection=[column1_utf8view] + +# coercions between stringview and date types +statement ok +create table dates (dt date) as values + (date '2024-01-23'), + (date '2023-11-30'); + +query D +select t.dt from dates t where arrow_cast('2024-01-01', 'Utf8View') < t.dt; +---- +2024-01-23 + +statement ok +drop table dates; + +### Tests for `||` with Utf8View specifically + +statement ok +create table temp as values +('value1', arrow_cast('rust', 'Utf8View'), arrow_cast('fast', 'Utf8View')), +('value2', arrow_cast('datafusion', 'Utf8View'), arrow_cast('cool', 'Utf8View')); + +query TTT +select arrow_typeof(column1), arrow_typeof(column2), arrow_typeof(column3) from temp; +---- +Utf8 Utf8View Utf8View +Utf8 Utf8View Utf8View + +query TT +explain select column2 || 'is' || column3 from temp; +---- +logical_plan +01)Projection: temp.column2 || Utf8View("is") || temp.column3 AS temp.column2 || Utf8("is") || temp.column3 +02)--TableScan: temp projection=[column2, column3] + +# should not cast the column2 to utf8 +query TT +explain select column2||' is fast' from temp; +---- +logical_plan +01)Projection: temp.column2 || Utf8View(" is fast") AS temp.column2 || Utf8(" is fast") +02)--TableScan: temp projection=[column2] + +query TT +explain select column2||column3 from temp; +---- +logical_plan +01)Projection: temp.column2 || temp.column3 +02)--TableScan: temp projection=[column2, column3] + +################################################ +# Test for Dictionary String concatenation +################################################ + +# || same type (column1 has null, so also tests NULL || NULL) +# expect all results to be the same for each row as they all have the same values +query T +SELECT + column1_dict || column1_dict +FROM test; +---- +AndrewAndrew +XiangpengXiangpeng +RaphaelRaphael +NULL + +# || constants +# expect all results to be the same for each row as they all have the same values +query TT +SELECT + column1_dict || 'foo', + 'foo' || column1_dict +FROM test; +---- +Andrewfoo fooAndrew +Xiangpengfoo fooXiangpeng +Raphaelfoo fooRaphael +NULL NULL + statement ok drop table test diff --git a/datafusion/sqllogictest/test_files/string_view.slt b/datafusion/sqllogictest/test_files/string_view.slt deleted file mode 100644 index 4e99d8cfa675..000000000000 --- a/datafusion/sqllogictest/test_files/string_view.slt +++ /dev/null @@ -1,1093 +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. - -######## -## Test setup -######## - -statement ok -create table test_source as values - ('Andrew', 'X'), - ('Xiangpeng', 'Xiangpeng'), - ('Raphael', 'R'), - (NULL, 'R'); - -# Table with the different combination of column types -statement ok -create table test as -SELECT - arrow_cast(column1, 'Utf8') as column1_utf8, - arrow_cast(column2, 'Utf8') as column2_utf8, - arrow_cast(column1, 'LargeUtf8') as column1_large_utf8, - arrow_cast(column2, 'LargeUtf8') as column2_large_utf8, - arrow_cast(column1, 'Utf8View') as column1_utf8view, - arrow_cast(column2, 'Utf8View') as column2_utf8view, - arrow_cast(column1, 'Dictionary(Int32, Utf8)') as column1_dict, - arrow_cast(column2, 'Dictionary(Int32, Utf8)') as column2_dict -FROM test_source; - -statement ok -drop table test_source - -statement ok -set datafusion.explain.logical_plan_only = true; - -# Ensure string functions use native StringView implementation -# and do not fall back to Utf8 or LargeUtf8 -# Should see no casts to Utf8 in the plans below - -## Ensure no casts for LIKE/ILIKE -query TT -EXPLAIN SELECT - column1_utf8view like 'foo' as "like", - column1_utf8view ilike 'foo' as "ilike" -FROM test; ----- -logical_plan -01)Projection: test.column1_utf8view LIKE Utf8View("foo") AS like, test.column1_utf8view ILIKE Utf8View("foo") AS ilike -02)--TableScan: test projection=[column1_utf8view] - -## Ensure no casts for SUBSTR - -query TT -EXPLAIN SELECT - SUBSTR(column1_utf8view, 1, 3) as c1, - SUBSTR(column2_utf8, 1, 3) as c2, - SUBSTR(column2_large_utf8, 1, 3) as c3 -FROM test; ----- -logical_plan -01)Projection: substr(test.column1_utf8view, Int64(1), Int64(3)) AS c1, substr(test.column2_utf8, Int64(1), Int64(3)) AS c2, substr(test.column2_large_utf8, Int64(1), Int64(3)) AS c3 -02)--TableScan: test projection=[column2_utf8, column2_large_utf8, column1_utf8view] - -query TTT -SELECT - SUBSTR(column1_utf8view, 1, 3) as c1, - SUBSTR(column2_utf8, 1, 3) as c2, - SUBSTR(column2_large_utf8, 1, 3) as c3 -FROM test; ----- -And X X -Xia Xia Xia -Rap R R -NULL R R - -## Ensure no casts for ASCII - -### ASCII - -# Test ASCII with utf8view against utf8view, utf8, and largeutf8 -# (should be no casts) -query TT -EXPLAIN SELECT - ASCII(column1_utf8view) as c1, - ASCII(column2_utf8) as c2, - ASCII(column2_large_utf8) as c3 -FROM test; ----- -logical_plan -01)Projection: ascii(test.column1_utf8view) AS c1, ascii(test.column2_utf8) AS c2, ascii(test.column2_large_utf8) AS c3 -02)--TableScan: test projection=[column2_utf8, column2_large_utf8, column1_utf8view] - -query III -SELECT - ASCII(column1_utf8view) as c1, - ASCII(column2_utf8) as c2, - ASCII(column2_large_utf8) as c3 -FROM test; ----- -65 88 88 -88 88 88 -82 82 82 -NULL 82 82 - -query TT -EXPLAIN SELECT - ASCII(column1_utf8) as c1, - ASCII(column1_large_utf8) as c2, - ASCII(column2_utf8view) as c3, - ASCII('hello') as c4, - ASCII(arrow_cast('world', 'Utf8View')) as c5 -FROM test; ----- -logical_plan -01)Projection: ascii(test.column1_utf8) AS c1, ascii(test.column1_large_utf8) AS c2, ascii(test.column2_utf8view) AS c3, Int32(104) AS c4, Int32(119) AS c5 -02)--TableScan: test projection=[column1_utf8, column1_large_utf8, column2_utf8view] - -query IIIII -SELECT - ASCII(column1_utf8) as c1, - ASCII(column1_large_utf8) as c2, - ASCII(column2_utf8view) as c3, - ASCII('hello') as c4, - ASCII(arrow_cast('world', 'Utf8View')) as c5 -FROM test; ----- -65 65 88 104 119 -88 88 88 104 119 -82 82 82 104 119 -NULL NULL 82 104 119 - -# Test ASCII with literals cast to Utf8View -query TT -EXPLAIN SELECT - ASCII(arrow_cast('äöüß', 'Utf8View')) as c1, - ASCII(arrow_cast('', 'Utf8View')) as c2, - ASCII(arrow_cast(NULL, 'Utf8View')) as c3 -FROM test; ----- -logical_plan -01)Projection: Int32(228) AS c1, Int32(0) AS c2, Int32(NULL) AS c3 -02)--TableScan: test projection=[] - -query III -SELECT - ASCII(arrow_cast('äöüß', 'Utf8View')) as c1, - ASCII(arrow_cast('', 'Utf8View')) as c2, - ASCII(arrow_cast(NULL, 'Utf8View')) as c3 ----- -228 0 NULL - -## Ensure no casts for BTRIM -# Test BTRIM with Utf8View input -query TT -EXPLAIN SELECT - BTRIM(column1_utf8view) AS l -FROM test; ----- -logical_plan -01)Projection: btrim(test.column1_utf8view) AS l -02)--TableScan: test projection=[column1_utf8view] - -# Test BTRIM with Utf8View input and Utf8View pattern -query TT -EXPLAIN SELECT - BTRIM(column1_utf8view, 'foo') AS l -FROM test; ----- -logical_plan -01)Projection: btrim(test.column1_utf8view, Utf8View("foo")) AS l -02)--TableScan: test projection=[column1_utf8view] - -# Test BTRIM with Utf8View bytes longer than 12 -query TT -EXPLAIN SELECT - BTRIM(column1_utf8view, 'this is longer than 12') AS l -FROM test; ----- -logical_plan -01)Projection: btrim(test.column1_utf8view, Utf8View("this is longer than 12")) AS l -02)--TableScan: test projection=[column1_utf8view] - -# Test BTRIM outputs -query TTTT -SELECT - BTRIM(column1_utf8view, 'foo') AS l1, - BTRIM(column1_utf8view, 'A') AS l2, - BTRIM(column1_utf8view) AS l3, - BTRIM(column1_utf8view, NULL) AS l4 -FROM test; ----- -Andrew ndrew Andrew NULL -Xiangpeng Xiangpeng Xiangpeng NULL -Raphael Raphael Raphael NULL -NULL NULL NULL NULL - -## Ensure no casts for LTRIM -# Test LTRIM with Utf8View input -query TT -EXPLAIN SELECT - LTRIM(column1_utf8view) AS l -FROM test; ----- -logical_plan -01)Projection: ltrim(test.column1_utf8view) AS l -02)--TableScan: test projection=[column1_utf8view] - -# Test LTRIM with Utf8View input and Utf8View pattern -query TT -EXPLAIN SELECT - LTRIM(column1_utf8view, 'foo') AS l -FROM test; ----- -logical_plan -01)Projection: ltrim(test.column1_utf8view, Utf8View("foo")) AS l -02)--TableScan: test projection=[column1_utf8view] - -# Test LTRIM with Utf8View bytes longer than 12 -query TT -EXPLAIN SELECT - LTRIM(column1_utf8view, 'this is longer than 12') AS l -FROM test; ----- -logical_plan -01)Projection: ltrim(test.column1_utf8view, Utf8View("this is longer than 12")) AS l -02)--TableScan: test projection=[column1_utf8view] - -# Test LTRIM outputs -query TTTTT -SELECT - LTRIM(column1_utf8view, 'foo') AS l1, - LTRIM(column1_utf8view, column2_utf8view) AS l2, - LTRIM(column1_utf8view) AS l3, - LTRIM(column1_utf8view, NULL) AS l4, - LTRIM(column1_utf8view, 'Xiang') AS l5 -FROM test; ----- -Andrew Andrew Andrew NULL Andrew -Xiangpeng (empty) Xiangpeng NULL peng -Raphael aphael Raphael NULL Raphael -NULL NULL NULL NULL NULL - -## ensure no casts for RTRIM -# Test RTRIM with Utf8View input -query TT -EXPLAIN SELECT - RTRIM(column1_utf8view) AS l -FROM test; ----- -logical_plan -01)Projection: rtrim(test.column1_utf8view) AS l -02)--TableScan: test projection=[column1_utf8view] - -# Test RTRIM with Utf8View input and Utf8View pattern -query TT -EXPLAIN SELECT - RTRIM(column1_utf8view, 'foo') AS l -FROM test; ----- -logical_plan -01)Projection: rtrim(test.column1_utf8view, Utf8View("foo")) AS l -02)--TableScan: test projection=[column1_utf8view] - -# Test RTRIM with Utf8View bytes longer than 12 -query TT -EXPLAIN SELECT - RTRIM(column1_utf8view, 'this is longer than 12') AS l -FROM test; ----- -logical_plan -01)Projection: rtrim(test.column1_utf8view, Utf8View("this is longer than 12")) AS l -02)--TableScan: test projection=[column1_utf8view] - -# Test RTRIM outputs -query TTTTT -SELECT - RTRIM(column1_utf8view, 'foo') AS l1, - RTRIM(column1_utf8view, column2_utf8view) AS l2, - RTRIM(column1_utf8view) AS l3, - RTRIM(column1_utf8view, NULL) AS l4, - RTRIM(column1_utf8view, 'peng') As l5 -FROM test; ----- -Andrew Andrew Andrew NULL Andrew -Xiangpeng (empty) Xiangpeng NULL Xia -Raphael Raphael Raphael NULL Raphael -NULL NULL NULL NULL NULL - - -## Ensure no casts for CHARACTER_LENGTH -query TT -EXPLAIN SELECT - CHARACTER_LENGTH(column1_utf8view) AS l -FROM test; ----- -logical_plan -01)Projection: character_length(test.column1_utf8view) AS l -02)--TableScan: test projection=[column1_utf8view] - -## Ensure no casts for CONCAT Utf8View -query TT -EXPLAIN SELECT - concat(column1_utf8view, column2_utf8view) as c -FROM test; ----- -logical_plan -01)Projection: concat(test.column1_utf8view, test.column2_utf8view) AS c -02)--TableScan: test projection=[column1_utf8view, column2_utf8view] - -## Ensure no casts for CONCAT LargeUtf8 -query TT -EXPLAIN SELECT - concat(column1_large_utf8, column2_large_utf8) as c -FROM test; ----- -logical_plan -01)Projection: concat(test.column1_large_utf8, test.column2_large_utf8) AS c -02)--TableScan: test projection=[column1_large_utf8, column2_large_utf8] - -## Ensure no casts for CONCAT_WS -## TODO https://github.com/apache/datafusion/issues/11837 -query TT -EXPLAIN SELECT - concat_ws(', ', column1_utf8view, column2_utf8view) as c -FROM test; ----- -logical_plan -01)Projection: concat_ws(Utf8(", "), test.column1_utf8view, test.column2_utf8view) AS c -02)--TableScan: test projection=[column1_utf8view, column2_utf8view] - -## Ensure no casts for CONTAINS -query TT -EXPLAIN SELECT - CONTAINS(column1_utf8view, 'foo') as c1, - CONTAINS(column1_utf8view, column2_utf8view) as c2, - CONTAINS(column1_utf8view, column2_large_utf8) as c3, - CONTAINS(column1_utf8, column2_utf8view) as c4, - CONTAINS(column1_utf8, column2_utf8) as c5, - CONTAINS(column1_utf8, column2_large_utf8) as c6, - CONTAINS(column1_large_utf8, column1_utf8view) as c7, - CONTAINS(column1_large_utf8, column2_utf8) as c8, - CONTAINS(column1_large_utf8, column2_large_utf8) as c9 -FROM test; ----- -logical_plan -01)Projection: contains(test.column1_utf8view, Utf8("foo")) AS c1, contains(test.column1_utf8view, test.column2_utf8view) AS c2, contains(test.column1_utf8view, test.column2_large_utf8) AS c3, contains(test.column1_utf8, test.column2_utf8view) AS c4, contains(test.column1_utf8, test.column2_utf8) AS c5, contains(test.column1_utf8, test.column2_large_utf8) AS c6, contains(test.column1_large_utf8, test.column1_utf8view) AS c7, contains(test.column1_large_utf8, test.column2_utf8) AS c8, contains(test.column1_large_utf8, test.column2_large_utf8) AS c9 -02)--TableScan: test projection=[column1_utf8, column2_utf8, column1_large_utf8, column2_large_utf8, column1_utf8view, column2_utf8view] - -query BBBBBBBBB -SELECT - CONTAINS(column1_utf8view, 'foo') as c1, - CONTAINS(column1_utf8view, column2_utf8view) as c2, - CONTAINS(column1_utf8view, column2_large_utf8) as c3, - CONTAINS(column1_utf8, column2_utf8view) as c4, - CONTAINS(column1_utf8, column2_utf8) as c5, - CONTAINS(column1_utf8, column2_large_utf8) as c6, - CONTAINS(column1_large_utf8, column1_utf8view) as c7, - CONTAINS(column1_large_utf8, column2_utf8) as c8, - CONTAINS(column1_large_utf8, column2_large_utf8) as c9 -FROM test; ----- -false false false false false false true false false -false true true true true true true true true -false true true true true true true true true -NULL NULL NULL NULL NULL NULL NULL NULL NULL - -## Ensure no casts for ENDS_WITH -query TT -EXPLAIN SELECT - ENDS_WITH(column1_utf8view, 'foo') as c1, - ENDS_WITH(column2_utf8view, column2_utf8view) as c2 -FROM test; ----- -logical_plan -01)Projection: ends_with(test.column1_utf8view, Utf8View("foo")) AS c1, ends_with(test.column2_utf8view, test.column2_utf8view) AS c2 -02)--TableScan: test projection=[column1_utf8view, column2_utf8view] - -## Ensure no casts for LEVENSHTEIN -query TT -EXPLAIN SELECT - levenshtein(column1_utf8view, 'foo') as c1, - levenshtein(column1_utf8view, column2_utf8view) as c2 -FROM test; ----- -logical_plan -01)Projection: levenshtein(test.column1_utf8view, Utf8View("foo")) AS c1, levenshtein(test.column1_utf8view, test.column2_utf8view) AS c2 -02)--TableScan: test projection=[column1_utf8view, column2_utf8view] - -## Ensure no casts for LOWER -query TT -EXPLAIN SELECT - LOWER(column1_utf8view) as c1 -FROM test; ----- -logical_plan -01)Projection: lower(test.column1_utf8view) AS c1 -02)--TableScan: test projection=[column1_utf8view] - -query T -SELECT LOWER(column1_utf8view) as c1 -FROM test; ----- -andrew -xiangpeng -raphael -NULL - -## Ensure no casts for UPPER -query TT -EXPLAIN SELECT - UPPER(column1_utf8view) as c1 -FROM test; ----- -logical_plan -01)Projection: upper(test.column1_utf8view) AS c1 -02)--TableScan: test projection=[column1_utf8view] - -query T -SELECT UPPER(column1_utf8view) as c1 -FROM test; ----- -ANDREW -XIANGPENG -RAPHAEL -NULL - -## Should run CONCAT successfully with utf8view -query T -SELECT - concat(column1_utf8view, column2_utf8view) as c -FROM test; ----- -AndrewX -XiangpengXiangpeng -RaphaelR -R - -## Should run CONCAT successfully with utf8 -query T -SELECT - concat(column1_utf8, column2_utf8) as c -FROM test; ----- -AndrewX -XiangpengXiangpeng -RaphaelR -R - -## Should run CONCAT successfully with utf8 and utf8view -query T -SELECT - concat(column1_utf8view, column2_utf8) as c -FROM test; ----- -AndrewX -XiangpengXiangpeng -RaphaelR -R - -## Should run CONCAT successfully with utf8 utf8view and largeutf8 -query T -SELECT - concat(column1_utf8view, column2_utf8, column2_large_utf8) as c -FROM test; ----- -AndrewXX -XiangpengXiangpengXiangpeng -RaphaelRR -RR - -## Should run CONCAT successfully with utf8large -query T -SELECT - concat(column1_large_utf8, column2_large_utf8) as c -FROM test; ----- -AndrewX -XiangpengXiangpeng -RaphaelR -R - -## Should run CONCAT successfully with utf8view -query T -SELECT - concat(column1_utf8view, column2_utf8view) as c -FROM test; ----- -AndrewX -XiangpengXiangpeng -RaphaelR -R - -## Should run CONCAT_WS successfully with utf8 -query T -SELECT - concat_ws(',', column1_utf8, column2_utf8) as c -FROM test; ----- -Andrew,X -Xiangpeng,Xiangpeng -Raphael,R -R - -## Should run CONCAT_WS successfully with utf8view -query T -SELECT - concat_ws(',', column1_utf8view, column2_utf8view) as c -FROM test; ----- -Andrew,X -Xiangpeng,Xiangpeng -Raphael,R -R - -## Should run CONCAT_WS successfully with largeutf8 -query T -SELECT - concat_ws(',', column1_large_utf8, column2_large_utf8) as c -FROM test; ----- -Andrew,X -Xiangpeng,Xiangpeng -Raphael,R -R - -## Should run CONCAT_WS successfully with utf8 and largeutf8 -query T -SELECT - concat_ws(',', column1_utf8, column2_large_utf8) as c -FROM test; ----- -Andrew,X -Xiangpeng,Xiangpeng -Raphael,R -R - -## Should run CONCAT_WS successfully with utf8 and utf8view -query T -SELECT - concat_ws(',', column1_utf8view, column2_utf8) as c -FROM test; ----- -Andrew,X -Xiangpeng,Xiangpeng -Raphael,R -R - -## Should run CONCAT_WS successfully with largeutf8 and utf8view -query T -SELECT - concat_ws(',', column1_utf8view, column2_large_utf8) as c -FROM test; ----- -Andrew,X -Xiangpeng,Xiangpeng -Raphael,R -R - -## Ensure no casts for LPAD -query TT -EXPLAIN SELECT - LPAD(column1_utf8view, 12, ' ') as c1 -FROM test; ----- -logical_plan -01)Projection: lpad(test.column1_utf8view, Int64(12), Utf8(" ")) AS c1 -02)--TableScan: test projection=[column1_utf8view] - -query TT -EXPLAIN SELECT - LPAD(column1_utf8view, 12, column2_large_utf8) as c1 -FROM test; ----- -logical_plan -01)Projection: lpad(test.column1_utf8view, Int64(12), test.column2_large_utf8) AS c1 -02)--TableScan: test projection=[column2_large_utf8, column1_utf8view] - -query TT -EXPLAIN SELECT - LPAD(column1_utf8view, 12, column2_utf8view) as c1 -FROM test; ----- -logical_plan -01)Projection: lpad(test.column1_utf8view, Int64(12), test.column2_utf8view) AS c1 -02)--TableScan: test projection=[column1_utf8view, column2_utf8view] - -## Ensure no casts for OCTET_LENGTH -query TT -EXPLAIN SELECT - OCTET_LENGTH(column1_utf8view) as c1 -FROM test; ----- -logical_plan -01)Projection: octet_length(test.column1_utf8view) AS c1 -02)--TableScan: test projection=[column1_utf8view] - -## Ensure no casts for OVERLAY -query TT -EXPLAIN SELECT - OVERLAY(column1_utf8view PLACING 'foo' FROM 2 ) as c1 -FROM test; ----- -logical_plan -01)Projection: overlay(test.column1_utf8view, Utf8View("foo"), Int64(2)) AS c1 -02)--TableScan: test projection=[column1_utf8view] - -query T -SELECT OVERLAY(column1_utf8view PLACING 'foo' FROM 2 ) as c1 FROM test; ----- -Afooew -Xfoogpeng -Rfooael -NULL - -## Ensure no casts for REGEXP_LIKE -query TT -EXPLAIN SELECT - REGEXP_LIKE(column1_utf8view, '^https?://(?:www\.)?([^/]+)/.*$') AS k -FROM test; ----- -logical_plan -01)Projection: regexp_like(CAST(test.column1_utf8view AS Utf8), Utf8("^https?://(?:www\.)?([^/]+)/.*$")) AS k -02)--TableScan: test projection=[column1_utf8view] - -## Ensure no casts for REGEXP_MATCH -query TT -EXPLAIN SELECT - REGEXP_MATCH(column1_utf8view, '^https?://(?:www\.)?([^/]+)/.*$') AS k -FROM test; ----- -logical_plan -01)Projection: regexp_match(CAST(test.column1_utf8view AS Utf8), Utf8("^https?://(?:www\.)?([^/]+)/.*$")) AS k -02)--TableScan: test projection=[column1_utf8view] - -## Ensure no casts for REGEXP_REPLACE -query TT -EXPLAIN SELECT - REGEXP_REPLACE(column1_utf8view, '^https?://(?:www\.)?([^/]+)/.*$', '\1') AS k -FROM test; ----- -logical_plan -01)Projection: regexp_replace(test.column1_utf8view, Utf8("^https?://(?:www\.)?([^/]+)/.*$"), Utf8("\1")) AS k -02)--TableScan: test projection=[column1_utf8view] - -## Ensure no casts for REPEAT -query TT -EXPLAIN SELECT - REPEAT(column1_utf8view, 2) as c1 -FROM test; ----- -logical_plan -01)Projection: repeat(test.column1_utf8view, Int64(2)) AS c1 -02)--TableScan: test projection=[column1_utf8view] - -## Ensure no casts for REPLACE -query TT -EXPLAIN SELECT - REPLACE(column1_utf8view, 'foo', 'bar') as c1, - REPLACE(column1_utf8view, column2_utf8view, 'bar') as c2 -FROM test; ----- -logical_plan -01)Projection: replace(test.column1_utf8view, Utf8View("foo"), Utf8View("bar")) AS c1, replace(test.column1_utf8view, test.column2_utf8view, Utf8View("bar")) AS c2 -02)--TableScan: test projection=[column1_utf8view, column2_utf8view] - -query TT -SELECT - REPLACE(column1_utf8view, 'foo', 'bar') as c1, - REPLACE(column1_utf8view, column2_utf8view, 'bar') as c2 -FROM test; ----- -Andrew Andrew -Xiangpeng bar -Raphael baraphael -NULL NULL - - -## Ensure no casts for REVERSE -query TT -EXPLAIN SELECT - REVERSE(column1_utf8view) as c1 -FROM test; ----- -logical_plan -01)Projection: reverse(test.column1_utf8view) AS c1 -02)--TableScan: test projection=[column1_utf8view] - - -## Ensure no casts for RIGHT -query TT -EXPLAIN SELECT - RIGHT(column1_utf8view, 3) as c2 -FROM test; ----- -logical_plan -01)Projection: right(test.column1_utf8view, Int64(3)) AS c2 -02)--TableScan: test projection=[column1_utf8view] - -# Test outputs of RIGHT -query TTT -SELECT - RIGHT(column1_utf8view, 3) as c1, - RIGHT(column1_utf8view, 0) as c2, - RIGHT(column1_utf8view, -3) as c3 -FROM test; ----- -rew (empty) rew -eng (empty) ngpeng -ael (empty) hael -NULL NULL NULL - -## Ensure no casts for LEFT -query TT -EXPLAIN SELECT - LEFT(column1_utf8view, 3) as c2 -FROM test; ----- -logical_plan -01)Projection: left(test.column1_utf8view, Int64(3)) AS c2 -02)--TableScan: test projection=[column1_utf8view] - -# Test outputs of LEFT -query TTT -SELECT - LEFT(column1_utf8view, 3) as c1, - LEFT(column1_utf8view, 0) as c2, - LEFT(column1_utf8view, -3) as c3 -FROM test; ----- -And (empty) And -Xia (empty) Xiangp -Rap (empty) Raph -NULL NULL NULL - -## Ensure no casts for RPAD -query TT -EXPLAIN SELECT - RPAD(column1_utf8view, 1) as c1, - RPAD(column1_utf8view, 2, column2_utf8view) as c2 -FROM test; ----- -logical_plan -01)Projection: rpad(test.column1_utf8view, Int64(1)) AS c1, rpad(test.column1_utf8view, Int64(2), test.column2_utf8view) AS c2 -02)--TableScan: test projection=[column1_utf8view, column2_utf8view] - -query TT -EXPLAIN SELECT - RPAD(column1_utf8view, 12, column2_large_utf8) as c1 -FROM test; ----- -logical_plan -01)Projection: rpad(test.column1_utf8view, Int64(12), test.column2_large_utf8) AS c1 -02)--TableScan: test projection=[column2_large_utf8, column1_utf8view] - -query TT -EXPLAIN SELECT - RPAD(column1_utf8view, 12, column2_utf8view) as c1 -FROM test; ----- -logical_plan -01)Projection: rpad(test.column1_utf8view, Int64(12), test.column2_utf8view) AS c1 -02)--TableScan: test projection=[column1_utf8view, column2_utf8view] - -## Ensure no casts for SPLIT_PART -query TT -EXPLAIN SELECT - SPLIT_PART(column1_utf8view, 'f', 1) as c1, - SPLIT_PART('testtesttest',column1_utf8view, 1) as c2 -FROM test; ----- -logical_plan -01)Projection: split_part(test.column1_utf8view, Utf8("f"), Int64(1)) AS c1, split_part(Utf8("testtesttest"), test.column1_utf8view, Int64(1)) AS c2 -02)--TableScan: test projection=[column1_utf8view] - -## Ensure no casts for STRPOS -query TT -EXPLAIN SELECT - STRPOS(column1_utf8view, 'f') as c, - STRPOS(column1_utf8view, column2_utf8view) as c2 -FROM test; ----- -logical_plan -01)Projection: strpos(test.column1_utf8view, Utf8("f")) AS c, strpos(test.column1_utf8view, test.column2_utf8view) AS c2 -02)--TableScan: test projection=[column1_utf8view, column2_utf8view] - -## Ensure no casts for SUBSTR -query TT -EXPLAIN SELECT - SUBSTR(column1_utf8view, 1) as c, - SUBSTR(column1_utf8view, 1 ,2) as c2 -FROM test; ----- -logical_plan -01)Projection: substr(test.column1_utf8view, Int64(1)) AS c, substr(test.column1_utf8view, Int64(1), Int64(2)) AS c2 -02)--TableScan: test projection=[column1_utf8view] - -## Ensure no casts for SUBSTRINDEX -query TT -EXPLAIN SELECT - SUBSTR_INDEX(column1_utf8view, 'a', 1) as c, - SUBSTR_INDEX(column1_utf8view, 'a', 2) as c2 -FROM test; ----- -logical_plan -01)Projection: substr_index(test.column1_utf8view, Utf8View("a"), Int64(1)) AS c, substr_index(test.column1_utf8view, Utf8View("a"), Int64(2)) AS c2 -02)--TableScan: test projection=[column1_utf8view] - -query TT -SELECT - SUBSTR_INDEX(column1_utf8view, 'a', 1) as c, - SUBSTR_INDEX(column1_utf8view, 'a', 2) as c2 -FROM test; ----- -Andrew Andrew -Xi Xiangpeng -R Raph -NULL NULL - -## Ensure no casts on columns for STARTS_WITH -query TT -EXPLAIN SELECT - STARTS_WITH(column1_utf8view, 'foo') as c, - STARTS_WITH(column1_utf8view, column2_utf8view) as c2 -FROM test; ----- -logical_plan -01)Projection: starts_with(test.column1_utf8view, Utf8View("foo")) AS c, starts_with(test.column1_utf8view, test.column2_utf8view) AS c2 -02)--TableScan: test projection=[column1_utf8view, column2_utf8view] - -## Ensure no casts for TRANSLATE -query TT -EXPLAIN SELECT - TRANSLATE(column1_utf8view, 'foo', 'bar') as c -FROM test; ----- -logical_plan -01)Projection: translate(test.column1_utf8view, Utf8("foo"), Utf8("bar")) AS c -02)--TableScan: test projection=[column1_utf8view] - -## Ensure no casts for FIND_IN_SET -query TT -EXPLAIN SELECT - FIND_IN_SET(column1_utf8view, 'a,b,c,d') as c -FROM test; ----- -logical_plan -01)Projection: find_in_set(test.column1_utf8view, Utf8View("a,b,c,d")) AS c -02)--TableScan: test projection=[column1_utf8view] - -query I -SELECT - FIND_IN_SET(column1_utf8view, 'a,b,c,d') as c -FROM test; ----- -0 -0 -0 -NULL - -# || mixed types -# expect all results to be the same for each row as they all have the same values -query TTTTTTTT -SELECT - column1_utf8view || column2_utf8view, - column1_utf8 || column2_utf8view, - column1_large_utf8 || column2_utf8view, - column1_dict || column2_utf8view, - -- reverse argument order - column2_utf8view || column1_utf8view, - column2_utf8view || column1_utf8, - column2_utf8view || column1_large_utf8, - column2_utf8view || column1_dict -FROM test; ----- -AndrewX AndrewX AndrewX AndrewX XAndrew XAndrew XAndrew XAndrew -XiangpengXiangpeng XiangpengXiangpeng XiangpengXiangpeng XiangpengXiangpeng XiangpengXiangpeng XiangpengXiangpeng XiangpengXiangpeng XiangpengXiangpeng -RaphaelR RaphaelR RaphaelR RaphaelR RRaphael RRaphael RRaphael RRaphael -NULL NULL NULL NULL NULL NULL NULL NULL - -# || constants -# expect all results to be the same for each row as they all have the same values -query TTTTTTTT -SELECT - column1_utf8view || 'foo', - column1_utf8 || 'foo', - column1_large_utf8 || 'foo', - column1_dict || 'foo', - -- reverse argument order - 'foo' || column1_utf8view, - 'foo' || column1_utf8, - 'foo' || column1_large_utf8, - 'foo' || column1_dict -FROM test; ----- -Andrewfoo Andrewfoo Andrewfoo Andrewfoo fooAndrew fooAndrew fooAndrew fooAndrew -Xiangpengfoo Xiangpengfoo Xiangpengfoo Xiangpengfoo fooXiangpeng fooXiangpeng fooXiangpeng fooXiangpeng -Raphaelfoo Raphaelfoo Raphaelfoo Raphaelfoo fooRaphael fooRaphael fooRaphael fooRaphael -NULL NULL NULL NULL NULL NULL NULL NULL - -# || same type (column1 has null, so also tests NULL || NULL) -# expect all results to be the same for each row as they all have the same values -query TTT -SELECT - column1_utf8view || column1_utf8view, - column1_utf8 || column1_utf8, - column1_large_utf8 || column1_large_utf8 - -- Dictionary/Dictionary coercion doesn't work - -- https://github.com/apache/datafusion/issues/12101 - --column1_dict || column1_dict -FROM test; ----- -AndrewAndrew AndrewAndrew AndrewAndrew -XiangpengXiangpeng XiangpengXiangpeng XiangpengXiangpeng -RaphaelRaphael RaphaelRaphael RaphaelRaphael -NULL NULL NULL - -## Ensure no casts for binary operators -## TODO: https://github.com/apache/datafusion/issues/12180 -# `~` operator (regex match) -query TT -EXPLAIN SELECT - column1_utf8view ~ 'an' AS c1 -FROM test; ----- -logical_plan -01)Projection: CAST(test.column1_utf8view AS Utf8) LIKE Utf8("%an%") AS c1 -02)--TableScan: test projection=[column1_utf8view] - -query B -SELECT - column1_utf8view ~ 'an' AS c1 -FROM test; ----- -false -true -false -NULL - -# `~*` operator (regex match case-insensitive) -query TT -EXPLAIN SELECT - column1_utf8view ~* '^a.{3}e' AS c1 -FROM test; ----- -logical_plan -01)Projection: CAST(test.column1_utf8view AS Utf8) ~* Utf8("^a.{3}e") AS c1 -02)--TableScan: test projection=[column1_utf8view] - -query B -SELECT - column1_utf8view ~* '^a.{3}e' AS c1 -FROM test; ----- -true -false -false -NULL - -# `!~~` operator (not like match) -query TT -EXPLAIN SELECT - column1_utf8view !~~ 'xia_g%g' AS c1 -FROM test; ----- -logical_plan -01)Projection: CAST(test.column1_utf8view AS Utf8) !~~ Utf8("xia_g%g") AS c1 -02)--TableScan: test projection=[column1_utf8view] - -query B -SELECT - column1_utf8view !~~ 'xia_g%g' AS c1 -FROM test; ----- -true -true -true -NULL - -# `!~~*` operator (not like match case-insensitive) -query TT -EXPLAIN SELECT - column1_utf8view !~~* 'xia_g%g' AS c1 -FROM test; ----- -logical_plan -01)Projection: CAST(test.column1_utf8view AS Utf8) !~~* Utf8("xia_g%g") AS c1 -02)--TableScan: test projection=[column1_utf8view] - -query B -SELECT - column1_utf8view !~~* 'xia_g%g' AS c1 -FROM test; ----- -true -false -true -NULL - -statement ok -drop table test; - -# coercion from stringview to integer, as input to make_date -query D -select make_date(arrow_cast('2024', 'Utf8View'), arrow_cast('01', 'Utf8View'), arrow_cast('23', 'Utf8View')) ----- -2024-01-23 - -# coercions between stringview and date types -statement ok -create table dates (dt date) as values - (date '2024-01-23'), - (date '2023-11-30'); - -query D -select t.dt from dates t where arrow_cast('2024-01-01', 'Utf8View') < t.dt; ----- -2024-01-23 - -statement ok -drop table dates; - -### Tests for `||` with Utf8View specifically - -statement ok -create table temp as values -('value1', arrow_cast('rust', 'Utf8View'), arrow_cast('fast', 'Utf8View')), -('value2', arrow_cast('datafusion', 'Utf8View'), arrow_cast('cool', 'Utf8View')); - -query TTT -select arrow_typeof(column1), arrow_typeof(column2), arrow_typeof(column3) from temp; ----- -Utf8 Utf8View Utf8View -Utf8 Utf8View Utf8View - -query T -select column2||' is fast' from temp; ----- -rust is fast -datafusion is fast - -query T -select column2 || ' is ' || column3 from temp; ----- -rust is fast -datafusion is cool - -query TT -explain select column2 || 'is' || column3 from temp; ----- -logical_plan -01)Projection: temp.column2 || Utf8View("is") || temp.column3 AS temp.column2 || Utf8("is") || temp.column3 -02)--TableScan: temp projection=[column2, column3] - -# should not cast the column2 to utf8 -query TT -explain select column2||' is fast' from temp; ----- -logical_plan -01)Projection: temp.column2 || Utf8View(" is fast") AS temp.column2 || Utf8(" is fast") -02)--TableScan: temp projection=[column2] - - -query T -select column2||column3 from temp; ----- -rustfast -datafusioncool - -query TT -explain select column2||column3 from temp; ----- -logical_plan -01)Projection: temp.column2 || temp.column3 -02)--TableScan: temp projection=[column2, column3] - -query T -select column2|| ' ' ||column3 from temp; ----- -rust fast -datafusion cool From 64a38963cb10b629ddfbd97e08208cc1c717ef2e Mon Sep 17 00:00:00 2001 From: Xiangpeng Hao Date: Sun, 22 Sep 2024 07:21:16 -0500 Subject: [PATCH 13/25] fix infer statistics for BinaryView types (#12575) --- datafusion/core/src/datasource/file_format/mod.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index a503e36adbeb..60f2b2dcefa9 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -268,7 +268,9 @@ pub(crate) fn coerce_file_schema_to_view_type( .iter() .map(|f| { let dt = f.data_type(); - if dt.equals_datatype(&DataType::Utf8View) { + if dt.equals_datatype(&DataType::Utf8View) + || dt.equals_datatype(&DataType::BinaryView) + { transform = true; } (f.name(), dt) From 4549168e7f34a42067d969f28c11c9e31baf7bcd Mon Sep 17 00:00:00 2001 From: Eason <30045503+Eason0729@users.noreply.github.com> Date: Mon, 23 Sep 2024 02:50:19 +0800 Subject: [PATCH 14/25] docs: :memo: Add expected answers to `DataFrame` method examples (#12564) * docs: :memo: Add expected answers to `DataFrame` method examples * test: :memo: use assert_batches_sorted_eq and simplify example_long.csv * Update datafusion/core/src/dataframe/mod.rs Co-authored-by: Oleks V --------- Co-authored-by: Andrew Lamb Co-authored-by: Oleks V --- datafusion/core/src/dataframe/mod.rs | 243 ++++++++++++++++++-- datafusion/core/tests/data/example_long.csv | 4 + 2 files changed, 224 insertions(+), 23 deletions(-) create mode 100644 datafusion/core/tests/data/example_long.csv diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 0faf53633020..663d8bf022f0 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -184,7 +184,7 @@ impl DataFrame { } /// Creates logical expression from a SQL query text. - /// The expression is created and processed againt the current schema. + /// The expression is created and processed against the current schema. /// /// # Example: Parsing SQL queries /// ``` @@ -221,11 +221,20 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; /// let df = df.select_columns(&["a", "b"])?; + /// let expected = vec![ + /// "+---+---+", + /// "| a | b |", + /// "+---+---+", + /// "| 1 | 2 |", + /// "+---+---+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &df.collect().await?); /// # Ok(()) /// # } /// ``` @@ -279,11 +288,20 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; - /// let df = df.select(vec![col("a") * col("b"), col("c")])?; + /// let df = df.select(vec![col("a"), col("b") * col("c")])?; + /// let expected = vec![ + /// "+---+-----------------------+", + /// "| a | ?table?.b * ?table?.c |", + /// "+---+-----------------------+", + /// "| 1 | 6 |", + /// "+---+-----------------------+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &df.collect().await?); /// # Ok(()) /// # } /// ``` @@ -307,11 +325,25 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; + /// // +----+----+----+ + /// // | a | b | c | + /// // +----+----+----+ + /// // | 1 | 2 | 3 | + /// // +----+----+----+ /// let df = df.drop_columns(&["a"])?; + /// let expected = vec![ + /// "+---+---+", + /// "| b | c |", + /// "+---+---+", + /// "| 2 | 3 |", + /// "+---+---+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &df.collect().await?); /// # Ok(()) /// # } /// ``` @@ -369,11 +401,26 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); /// let df = ctx.read_json("tests/data/unnest.json", NdJsonReadOptions::default()).await?; + /// // expand into multiple columns if it's json array, flatten field name if it's nested structure /// let df = df.unnest_columns(&["b","c","d"])?; + /// let expected = vec![ + /// "+---+------+-------+-----+-----+", + /// "| a | b | c | d.e | d.f |", + /// "+---+------+-------+-----+-----+", + /// "| 1 | 2.0 | false | 1 | 2 |", + /// "| 1 | 1.3 | true | 1 | 2 |", + /// "| 1 | -6.1 | | 1 | 2 |", + /// "| 2 | 3.0 | false | | |", + /// "| 2 | 2.3 | true | | |", + /// "| 2 | -7.1 | | | |", + /// "+---+------+-------+-----+-----+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &df.collect().await?); /// # Ok(()) /// # } /// ``` @@ -411,11 +458,23 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); - /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; + /// let df = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?; /// let df = df.filter(col("a").lt_eq(col("b")))?; + /// // all rows where a <= b are returned + /// let expected = vec![ + /// "+---+---+---+", + /// "| a | b | c |", + /// "+---+---+---+", + /// "| 1 | 2 | 3 |", + /// "| 4 | 5 | 6 |", + /// "| 7 | 8 | 9 |", + /// "+---+---+---+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &df.collect().await?); /// # Ok(()) /// # } /// ``` @@ -437,16 +496,34 @@ impl DataFrame { /// # use datafusion::prelude::*; /// # use datafusion::error::Result; /// # use datafusion::functions_aggregate::expr_fn::min; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); - /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; + /// let df = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?; /// /// // The following use is the equivalent of "SELECT MIN(b) GROUP BY a" - /// let _ = df.clone().aggregate(vec![col("a")], vec![min(col("b"))])?; - /// + /// let df1 = df.clone().aggregate(vec![col("a")], vec![min(col("b"))])?; + /// let expected1 = vec![ + /// "+---+----------------+", + /// "| a | min(?table?.b) |", + /// "+---+----------------+", + /// "| 1 | 2 |", + /// "| 4 | 5 |", + /// "| 7 | 8 |", + /// "+---+----------------+" + /// ]; + /// assert_batches_sorted_eq!(expected1, &df1.collect().await?); /// // The following use is the equivalent of "SELECT MIN(b)" - /// let _ = df.aggregate(vec![], vec![min(col("b"))])?; + /// let df2 = df.aggregate(vec![], vec![min(col("b"))])?; + /// let expected2 = vec![ + /// "+----------------+", + /// "| min(?table?.b) |", + /// "+----------------+", + /// "| 2 |", + /// "+----------------+" + /// ]; + /// # assert_batches_sorted_eq!(expected2, &df2.collect().await?); /// # Ok(()) /// # } /// ``` @@ -486,11 +563,21 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); - /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; - /// let df = df.limit(0, Some(100))?; + /// let df = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?; + /// let df = df.limit(1, Some(2))?; + /// let expected = vec![ + /// "+---+---+---+", + /// "| a | b | c |", + /// "+---+---+---+", + /// "| 4 | 5 | 6 |", + /// "| 7 | 8 | 9 |", + /// "+---+---+---+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &df.collect().await?); /// # Ok(()) /// # } /// ``` @@ -512,12 +599,22 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); - /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; + /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await? ; /// let d2 = df.clone(); /// let df = df.union(d2)?; + /// let expected = vec![ + /// "+---+---+---+", + /// "| a | b | c |", + /// "+---+---+---+", + /// "| 1 | 2 | 3 |", + /// "| 1 | 2 | 3 |", + /// "+---+---+---+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &df.collect().await?); /// # Ok(()) /// # } /// ``` @@ -540,12 +637,22 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; /// let d2 = df.clone(); /// let df = df.union_distinct(d2)?; + /// // df2 are duplicate of df + /// let expected = vec![ + /// "+---+---+---+", + /// "| a | b | c |", + /// "+---+---+---+", + /// "| 1 | 2 | 3 |", + /// "+---+---+---+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &df.collect().await?); /// # Ok(()) /// # } /// ``` @@ -565,11 +672,20 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; /// let df = df.distinct()?; + /// let expected = vec![ + /// "+---+---+---+", + /// "| a | b | c |", + /// "+---+---+---+", + /// "| 1 | 2 | 3 |", + /// "+---+---+---+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &df.collect().await?); /// # Ok(()) /// # } /// ``` @@ -589,12 +705,21 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await? /// // Return a single row (a, b) for each distinct value of a /// .distinct_on(vec![col("a")], vec![col("a"), col("b")], None)?; + /// let expected = vec![ + /// "+---+---+", + /// "| a | b |", + /// "+---+---+", + /// "| 1 | 2 |", + /// "+---+---+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &df.collect().await?); /// # Ok(()) /// # } /// ``` @@ -623,12 +748,26 @@ impl DataFrame { /// # use datafusion::prelude::*; /// # use datafusion::error::Result; /// # use arrow::util::pretty; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); /// let df = ctx.read_csv("tests/tpch-csv/customer.csv", CsvReadOptions::new()).await?; - /// df.describe().await.unwrap(); - /// + /// let stat = df.describe().await?; + /// # // some output column are ignored + /// let expected = vec![ + /// "+------------+--------------------+--------------------+------------------------------------+--------------------+-----------------+--------------------+--------------+----------------------------------------------------------------------------------------------------------+", + /// "| describe | c_custkey | c_name | c_address | c_nationkey | c_phone | c_acctbal | c_mktsegment | c_comment |", + /// "+------------+--------------------+--------------------+------------------------------------+--------------------+-----------------+--------------------+--------------+----------------------------------------------------------------------------------------------------------+", + /// "| count | 9.0 | 9 | 9 | 9.0 | 9 | 9.0 | 9 | 9 |", + /// "| max | 10.0 | Customer#000000010 | xKiAFTjUsCuxfeleNqefumTrjS | 20.0 | 30-114-968-4951 | 9561.95 | MACHINERY | tions. even deposits boost according to the slyly bold packages. final accounts cajole requests. furious |", + /// "| mean | 6.0 | null | null | 9.88888888888889 | null | 5153.2155555555555 | null | null |", + /// "| median | 6.0 | null | null | 8.0 | null | 6819.74 | null | null |", + /// "| min | 2.0 | Customer#000000002 | 6LrEaV6KR6PLVcgl2ArL Q3rqzLzcT1 v2 | 1.0 | 11-719-748-3364 | 121.65 | AUTOMOBILE | deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov |", + /// "| null_count | 0.0 | 0 | 0 | 0.0 | 0 | 0.0 | 0 | 0 |", + /// "| std | 2.7386127875258306 | null | null | 7.2188026092359046 | null | 3522.169804254585 | null | null |", + /// "+------------+--------------------+--------------------+------------------------------------+--------------------+-----------------+--------------------+--------------+----------------------------------------------------------------------------------------------------------+"]; + /// assert_batches_sorted_eq!(expected, &stat.collect().await?); /// # Ok(()) /// # } /// ``` @@ -820,14 +959,25 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); - /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; + /// let df = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?; /// let df = df.sort(vec![ - /// col("a").sort(true, true), // a ASC, nulls first - /// col("b").sort(false, false), // b DESC, nulls last + /// col("a").sort(false, true), // a DESC, nulls first + /// col("b").sort(true, false), // b ASC, nulls last /// ])?; + /// let expected = vec![ + /// "+---+---+---+", + /// "| a | b | c |", + /// "+---+---+---+", + /// "| 1 | 2 | 3 |", + /// "| 4 | 5 | 6 |", + /// "| 7 | 8 | 9 |", + /// "+---+---+---+", + /// ]; + /// # assert_batches_sorted_eq!(expected, &df.collect().await?); /// # Ok(()) /// # } /// ``` @@ -857,6 +1007,7 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); @@ -869,7 +1020,14 @@ impl DataFrame { /// // Perform the equivalent of `left INNER JOIN right ON (a = a2 AND b = b2)` /// // finding all pairs of rows from `left` and `right` where `a = a2` and `b = b2`. /// let join = left.join(right, JoinType::Inner, &["a", "b"], &["a2", "b2"], None)?; - /// let batches = join.collect().await?; + /// let expected = vec![ + /// "+---+---+---+----+----+----+", + /// "| a | b | c | a2 | b2 | c2 |", + /// "+---+---+---+----+----+----+", + /// "| 1 | 2 | 3 | 1 | 2 | 3 |", + /// "+---+---+---+----+----+----+" + /// ]; + /// assert_batches_sorted_eq!(expected, &join.collect().await?); /// # Ok(()) /// # } /// ``` @@ -906,6 +1064,7 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); @@ -929,7 +1088,13 @@ impl DataFrame { /// JoinType::Inner, /// [col("a").not_eq(col("a2")), col("b").not_eq(col("b2"))], /// )?; - /// let batches = join_on.collect().await?; + /// let expected = vec![ + /// "+---+---+---+----+----+----+", + /// "| a | b | c | a2 | b2 | c2 |", + /// "+---+---+---+----+----+----+", + /// "+---+---+---+----+----+----+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &join_on.collect().await?); /// # Ok(()) /// # } /// ``` @@ -954,11 +1119,22 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); - /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; + /// let df = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?; /// let df1 = df.repartition(Partitioning::RoundRobinBatch(4))?; + /// let expected = vec![ + /// "+---+---+---+", + /// "| a | b | c |", + /// "+---+---+---+", + /// "| 1 | 2 | 3 |", + /// "| 4 | 5 | 6 |", + /// "| 7 | 8 | 9 |", + /// "+---+---+---+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &df1.collect().await?); /// # Ok(()) /// # } /// ``` @@ -985,7 +1161,8 @@ impl DataFrame { /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; - /// let count = df.count().await?; + /// let count = df.count().await?; // 1 + /// # assert_eq!(count, 1); /// # Ok(()) /// # } /// ``` @@ -1266,12 +1443,21 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; - /// let d2 = df.clone(); + /// let d2 = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?; /// let df = df.intersect(d2)?; + /// let expected = vec![ + /// "+---+---+---+", + /// "| a | b | c |", + /// "+---+---+---+", + /// "| 1 | 2 | 3 |", + /// "+---+---+---+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &df.collect().await?); /// # Ok(()) /// # } /// ``` @@ -1290,12 +1476,23 @@ impl DataFrame { /// ``` /// # use datafusion::prelude::*; /// # use datafusion::error::Result; + /// # use datafusion_common::assert_batches_sorted_eq; /// # #[tokio::main] /// # async fn main() -> Result<()> { /// let ctx = SessionContext::new(); - /// let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; - /// let d2 = df.clone(); - /// let df = df.except(d2)?; + /// let df = ctx.read_csv("tests/data/example_long.csv", CsvReadOptions::new()).await?; + /// let d2 = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; + /// let result = df.except(d2)?; + /// // those columns are not in example.csv, but in example_long.csv + /// let expected = vec![ + /// "+---+---+---+", + /// "| a | b | c |", + /// "+---+---+---+", + /// "| 4 | 5 | 6 |", + /// "| 7 | 8 | 9 |", + /// "+---+---+---+" + /// ]; + /// # assert_batches_sorted_eq!(expected, &result.collect().await?); /// # Ok(()) /// # } /// ``` diff --git a/datafusion/core/tests/data/example_long.csv b/datafusion/core/tests/data/example_long.csv new file mode 100644 index 000000000000..83d4cdde1ce1 --- /dev/null +++ b/datafusion/core/tests/data/example_long.csv @@ -0,0 +1,4 @@ +a,b,c +1,2,3 +4,5,6 +7,8,9 \ No newline at end of file From 3978e5d11dfd0532b4d19e27563bfbe7b8375389 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 22 Sep 2024 14:51:00 -0400 Subject: [PATCH 15/25] Minor: add example to of assert_batches_eq (#12580) --- datafusion/common/src/test_util.rs | 24 ++++++++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/test_util.rs b/datafusion/common/src/test_util.rs index 9c3dfe62e119..36254192550c 100644 --- a/datafusion/common/src/test_util.rs +++ b/datafusion/common/src/test_util.rs @@ -29,6 +29,27 @@ use std::{error::Error, path::PathBuf}; /// Expects to be called about like this: /// /// `assert_batch_eq!(expected_lines: &[&str], batches: &[RecordBatch])` +/// +/// # Example +/// ``` +/// # use std::sync::Arc; +/// # use arrow::record_batch::RecordBatch; +/// # use arrow_array::{ArrayRef, Int32Array}; +/// # use datafusion_common::assert_batches_eq; +/// let col: ArrayRef = Arc::new(Int32Array::from(vec![1, 2])); +/// let batch = RecordBatch::try_from_iter([("column", col)]).unwrap(); +/// // Expected output is a vec of strings +/// let expected = vec![ +/// "+--------+", +/// "| column |", +/// "+--------+", +/// "| 1 |", +/// "| 2 |", +/// "+--------+", +/// ]; +/// // compare the formatted output of the record batch with the expected output +/// assert_batches_eq!(expected, &[batch]); +/// ``` #[macro_export] macro_rules! assert_batches_eq { ($EXPECTED_LINES: expr, $CHUNKS: expr) => { @@ -56,8 +77,7 @@ macro_rules! assert_batches_eq { /// vector of strings in a way that order does not matter. /// This is a macro so errors appear on the correct line /// -/// Designed so that failure output can be directly copy/pasted -/// into the test code as expected results. +/// See [`assert_batches_eq`] for more details and example. /// /// Expects to be called about like this: /// From d73c9d8bf28e6797cd05a5b4296b3a7a121a8e23 Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Mon, 23 Sep 2024 10:16:06 +0800 Subject: [PATCH 16/25] Use qualified aliases to simplify searching DFSchema (#12546) --- datafusion/common/src/dfschema.rs | 13 +---------- .../src/single_distinct_to_groupby.rs | 23 +++++++------------ datafusion/sqllogictest/test_files/select.slt | 4 ++++ 3 files changed, 13 insertions(+), 27 deletions(-) diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index 7fb2d4dc12b2..0dec14e9178a 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -355,18 +355,7 @@ impl DFSchema { // qualifier and name. (Some(q), Some(field_q)) => q.resolved_eq(field_q) && f.name() == name, // field to lookup is qualified but current field is unqualified. - (Some(qq), None) => { - // the original field may now be aliased with a name that matches the - // original qualified name - let column = Column::from_qualified_name(f.name()); - match column { - Column { - relation: Some(r), - name: column_name, - } => &r == qq && column_name == name, - _ => false, - } - } + (Some(_), None) => false, // field to lookup is unqualified, no need to compare qualifier (None, Some(_)) | (None, None) => f.name() == name, }) diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index dd82b056d0a6..2509e67dbec7 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -22,9 +22,7 @@ use std::sync::Arc; use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::{ - internal_err, qualified_name, tree_node::Transformed, DataFusionError, Result, -}; +use datafusion_common::{internal_err, tree_node::Transformed, DataFusionError, Result}; use datafusion_expr::builder::project; use datafusion_expr::{ col, @@ -135,7 +133,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { // alias all original group_by exprs let (mut inner_group_exprs, out_group_expr_with_alias): ( Vec, - Vec<(Expr, Option)>, + Vec<(Expr, _)>, ) = group_expr .into_iter() .enumerate() @@ -166,10 +164,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { let (qualifier, field) = schema.qualified_field(i); ( group_expr.alias(alias_str.clone()), - ( - col(alias_str), - Some(qualified_name(qualifier, field.name())), - ), + (col(alias_str), Some((qualifier, field.name()))), ) } }) @@ -253,19 +248,17 @@ impl OptimizerRule for SingleDistinctToGroupBy { // - aggr expr let alias_expr: Vec<_> = out_group_expr_with_alias .into_iter() - .map(|(group_expr, original_field)| { - if let Some(name) = original_field { - group_expr.alias(name) - } else { - group_expr + .map(|(group_expr, original_name)| match original_name { + Some((qualifier, name)) => { + group_expr.alias_qualified(qualifier.cloned(), name) } + None => group_expr, }) .chain(outer_aggr_exprs.iter().cloned().enumerate().map( |(idx, expr)| { let idx = idx + group_size; let (qualifier, field) = schema.qualified_field(idx); - let name = qualified_name(qualifier, field.name()); - expr.alias(name) + expr.alias_qualified(qualifier.cloned(), field.name()) }, )) .collect(); diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index 05de3e0b8091..6950c41f5405 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1756,3 +1756,7 @@ SELECT "test.a" FROM (SELECT a AS "test.a" FROM test) statement ok DROP TABLE test; + +# Can't reference an unqualified column by a qualified name +query error DataFusion error: Schema error: No field named t1\.v1\. Valid fields are "t1\.v1"\. +SELECT t1.v1 FROM (SELECT 1 AS "t1.v1"); From 30d4368127db80c125a1adea22a955a496a516df Mon Sep 17 00:00:00 2001 From: waruto Date: Mon, 23 Sep 2024 18:21:50 +0800 Subject: [PATCH 17/25] return absent stats when filters are pushed down (#12471) * do not pushdown filters that can be resolved only using partition cols return absent stats when filters are pushed down * fix and add test * Update datafusion/core/src/datasource/physical_plan/parquet/mod.rs Co-authored-by: Andrew Lamb * add test for partition pruning filters --------- Co-authored-by: Andrew Lamb --- .../core/src/datasource/listing/table.rs | 14 ++++ .../datasource/physical_plan/parquet/mod.rs | 13 +++- .../core/tests/parquet/file_statistics.rs | 21 +++++ datafusion/core/tests/sql/path_partition.rs | 78 +++++++++++++++++-- .../physical-plan/src/execution_plan.rs | 3 + 5 files changed, 122 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index d2c2bd7a1281..9142e922eb1e 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -819,6 +819,20 @@ impl TableProvider for ListingTable { .map(|col| Ok(self.table_schema.field_with_name(&col.0)?.clone())) .collect::>>()?; + // If the filters can be resolved using only partition cols, there is no need to + // pushdown it to TableScan, otherwise, `unhandled` pruning predicates will be generated + let table_partition_col_names = table_partition_cols + .iter() + .map(|field| field.name().as_str()) + .collect::>(); + let filters = filters + .iter() + .filter(|filter| { + !expr_applicable_for_cols(&table_partition_col_names, filter) + }) + .cloned() + .collect::>(); + let filters = conjunction(filters.to_vec()) .map(|expr| -> Result<_> { // NOTE: Use the table schema (NOT file schema) here because `expr` may contain references to partition columns. diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 7aacf5cf5c5c..ce679bfa76c5 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -743,7 +743,18 @@ impl ExecutionPlan for ParquetExec { } fn statistics(&self) -> Result { - Ok(self.projected_statistics.clone()) + // When filters are pushed down, we have no way of knowing the exact statistics. + // Note that pruning predicate is also a kind of filter pushdown. + // (bloom filters use `pruning_predicate` too) + let stats = if self.pruning_predicate.is_some() + || self.page_pruning_predicate.is_some() + || (self.predicate.is_some() && self.pushdown_filters()) + { + self.projected_statistics.clone().to_inexact() + } else { + self.projected_statistics.clone() + }; + Ok(stats) } fn fetch(&self) -> Option { diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index cd62c3bf426f..18d8300fb254 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -36,8 +36,29 @@ use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion::execution::session_state::SessionStateBuilder; +use datafusion_expr::{col, lit, Expr}; use tempfile::tempdir; +#[tokio::test] +async fn check_stats_precision_with_filter_pushdown() { + let testdata = datafusion::test_util::parquet_test_data(); + let filename = format!("{}/{}", testdata, "alltypes_plain.parquet"); + let table_path = ListingTableUrl::parse(filename).unwrap(); + + let opt = ListingOptions::new(Arc::new(ParquetFormat::default())); + let table = get_listing_table(&table_path, None, &opt).await; + let (_, _, state) = get_cache_runtime_state(); + // Scan without filter, stats are exact + let exec = table.scan(&state, None, &[], None).await.unwrap(); + assert_eq!(exec.statistics().unwrap().num_rows, Precision::Exact(8)); + + // Scan with filter pushdown, stats are inexact + let filter = Expr::gt(col("id"), lit(1)); + + let exec = table.scan(&state, None, &[filter], None).await.unwrap(); + assert_eq!(exec.statistics().unwrap().num_rows, Precision::Inexact(8)); +} + #[tokio::test] async fn load_table_stats_with_session_level_cache() { let testdata = datafusion::test_util::parquet_test_data(); diff --git a/datafusion/core/tests/sql/path_partition.rs b/datafusion/core/tests/sql/path_partition.rs index 7e7544bdb7c0..919054e8330f 100644 --- a/datafusion/core/tests/sql/path_partition.rs +++ b/datafusion/core/tests/sql/path_partition.rs @@ -25,6 +25,7 @@ use std::sync::Arc; use arrow::datatypes::DataType; use datafusion::datasource::listing::ListingTableUrl; +use datafusion::datasource::physical_plan::ParquetExec; use datafusion::{ assert_batches_sorted_eq, datasource::{ @@ -36,6 +37,7 @@ use datafusion::{ prelude::SessionContext, test_util::{self, arrow_test_data, parquet_test_data}, }; +use datafusion_catalog::TableProvider; use datafusion_common::stats::Precision; use datafusion_common::ScalarValue; use datafusion_execution::config::SessionConfig; @@ -43,6 +45,9 @@ use datafusion_execution::config::SessionConfig; use async_trait::async_trait; use bytes::Bytes; use chrono::{TimeZone, Utc}; +use datafusion_expr::{col, lit, Expr, Operator}; +use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; +use datafusion_physical_expr::PhysicalExpr; use futures::stream::{self, BoxStream}; use object_store::{ path::Path, GetOptions, GetResult, GetResultPayload, ListResult, ObjectMeta, @@ -51,6 +56,52 @@ use object_store::{ use object_store::{Attributes, MultipartUpload, PutMultipartOpts, PutPayload}; use url::Url; +#[tokio::test] +async fn parquet_partition_pruning_filter() -> Result<()> { + let ctx = SessionContext::new(); + + let table = create_partitioned_alltypes_parquet_table( + &ctx, + &[ + "year=2021/month=09/day=09/file.parquet", + "year=2021/month=10/day=09/file.parquet", + "year=2021/month=10/day=28/file.parquet", + ], + &[ + ("year", DataType::Int32), + ("month", DataType::Int32), + ("day", DataType::Int32), + ], + "mirror:///", + "alltypes_plain.parquet", + ) + .await; + + // The first three filters can be resolved using only the partition columns. + let filters = [ + Expr::eq(col("year"), lit(2021)), + Expr::eq(col("month"), lit(10)), + Expr::eq(col("day"), lit(28)), + Expr::gt(col("id"), lit(1)), + ]; + let exec = table.scan(&ctx.state(), None, &filters, None).await?; + let parquet_exec = exec.as_any().downcast_ref::().unwrap(); + let pred = parquet_exec.predicate().unwrap(); + // Only the last filter should be pushdown to TableScan + let expected = Arc::new(BinaryExpr::new( + Arc::new(Column::new_with_schema("id", &exec.schema()).unwrap()), + Operator::Gt, + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), + )); + + assert!(pred.as_any().is::()); + let pred = pred.as_any().downcast_ref::().unwrap(); + + assert_eq!(pred, expected.as_any()); + + Ok(()) +} + #[tokio::test] async fn parquet_distinct_partition_col() -> Result<()> { let ctx = SessionContext::new(); @@ -491,7 +542,7 @@ async fn parquet_statistics() -> Result<()> { // stats for the first col are read from the parquet file assert_eq!(stat_cols[0].null_count, Precision::Exact(1)); // TODO assert partition column stats once implemented (#1186) - assert_eq!(stat_cols[1], ColumnStatistics::new_unknown(),); + assert_eq!(stat_cols[1], ColumnStatistics::new_unknown()); Ok(()) } @@ -563,6 +614,25 @@ async fn register_partitioned_alltypes_parquet( table_path: &str, source_file: &str, ) { + let table = create_partitioned_alltypes_parquet_table( + ctx, + store_paths, + partition_cols, + table_path, + source_file, + ) + .await; + ctx.register_table("t", table) + .expect("registering listing table failed"); +} + +async fn create_partitioned_alltypes_parquet_table( + ctx: &SessionContext, + store_paths: &[&str], + partition_cols: &[(&str, DataType)], + table_path: &str, + source_file: &str, +) -> Arc { let testdata = parquet_test_data(); let parquet_file_path = format!("{testdata}/{source_file}"); let url = Url::parse("mirror://").unwrap(); @@ -591,11 +661,7 @@ async fn register_partitioned_alltypes_parquet( let config = ListingTableConfig::new(table_path) .with_listing_options(options) .with_schema(file_schema); - - let table = ListingTable::try_new(config).unwrap(); - - ctx.register_table("t", Arc::new(table)) - .expect("registering listing table failed"); + Arc::new(ListingTable::try_new(config).unwrap()) } #[derive(Debug)] diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 40d18ebece0c..542861688dfe 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -379,6 +379,9 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Returns statistics for this `ExecutionPlan` node. If statistics are not /// available, should return [`Statistics::new_unknown`] (the default), not /// an error. + /// + /// For TableScan executors, which supports filter pushdown, special attention + /// needs to be paid to whether the stats returned by this method are exact or not fn statistics(&self) -> Result { Ok(Statistics::new_unknown(&self.schema())) } From fe987e9d2683daa6dbc3d3602f4bae95becce2c2 Mon Sep 17 00:00:00 2001 From: Smith Cruise Date: Mon, 23 Sep 2024 18:22:24 +0800 Subject: [PATCH 18/25] add new() function for ParquetReadOptions (#12579) Signed-off-by: Smith Cruise --- datafusion/core/src/datasource/file_format/options.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/datafusion/core/src/datasource/file_format/options.rs b/datafusion/core/src/datasource/file_format/options.rs index 7ef5a2182d1c..1e0e28ef88cb 100644 --- a/datafusion/core/src/datasource/file_format/options.rs +++ b/datafusion/core/src/datasource/file_format/options.rs @@ -257,6 +257,11 @@ impl<'a> Default for ParquetReadOptions<'a> { } impl<'a> ParquetReadOptions<'a> { + /// Create a new ParquetReadOptions with default values + pub fn new() -> Self { + Default::default() + } + /// Specify parquet_pruning pub fn parquet_pruning(mut self, parquet_pruning: bool) -> Self { self.parquet_pruning = Some(parquet_pruning); From 5768bba81bb3b13dea95ce10b877ac6ba1a41e7f Mon Sep 17 00:00:00 2001 From: Samuel Colvin Date: Mon, 23 Sep 2024 11:23:30 +0100 Subject: [PATCH 19/25] make Debug for MemoryExec prettier (#12582) --- datafusion/physical-plan/src/memory.rs | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 6b2c78902eae..3aa445d295cb 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -56,13 +56,12 @@ pub struct MemoryExec { impl fmt::Debug for MemoryExec { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "partitions: [...]")?; - write!(f, "schema: {:?}", self.projected_schema)?; - write!(f, "projection: {:?}", self.projection)?; - if let Some(sort_info) = &self.sort_information.first() { - write!(f, ", output_ordering: {:?}", sort_info)?; - } - Ok(()) + f.debug_struct("MemoryExec") + .field("partitions", &"[...]") + .field("schema", &self.schema) + .field("projection", &self.projection) + .field("sort_information", &self.sort_information) + .finish() } } From a35d0075744a058f81bd9ebed747e2e597434019 Mon Sep 17 00:00:00 2001 From: OussamaSaoudi <45303303+OussamaSaoudi@users.noreply.github.com> Date: Mon, 23 Sep 2024 03:24:09 -0700 Subject: [PATCH 20/25] Add method SessionStateBuilder::with_object_store (#12578) This PR adds the the method `SessionStateBuilder::with_object_store`, which registers an object store with a specified URL to the RuntimeEnv. --- .../core/src/execution/session_state.rs | 37 +++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index a46e5c36156c..e447220a1fba 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -68,6 +68,7 @@ use datafusion_sql::parser::{DFParser, Statement}; use datafusion_sql::planner::{ContextProvider, ParserOptions, PlannerContext, SqlToRel}; use itertools::Itertools; use log::{debug, info}; +use object_store::ObjectStore; use sqlparser::ast::Expr as SQLExpr; use sqlparser::dialect::dialect_from_str; use std::any::Any; @@ -75,6 +76,7 @@ use std::collections::hash_map::Entry; use std::collections::{HashMap, HashSet}; use std::fmt::Debug; use std::sync::Arc; +use url::Url; use uuid::Uuid; /// `SessionState` contains all the necessary state to plan and execute queries, @@ -1229,6 +1231,41 @@ impl SessionStateBuilder { self } + /// Register an `ObjectStore` to the [`RuntimeEnv`]. See [`RuntimeEnv::register_object_store`] + /// for more details. + /// + /// Note that this creates a default [`RuntimeEnv`] if there isn't one passed in already. + /// + /// ``` + /// # use datafusion::prelude::*; + /// # use datafusion::execution::session_state::SessionStateBuilder; + /// # use datafusion_execution::runtime_env::RuntimeEnv; + /// # use url::Url; + /// # use std::sync::Arc; + /// # let http_store = object_store::local::LocalFileSystem::new(); + /// let url = Url::try_from("file://").unwrap(); + /// let object_store = object_store::local::LocalFileSystem::new(); + /// let state = SessionStateBuilder::new() + /// .with_config(SessionConfig::new()) + /// .with_object_store(&url, Arc::new(object_store)) + /// .with_default_features() + /// .build(); + /// ``` + pub fn with_object_store( + mut self, + url: &Url, + object_store: Arc, + ) -> Self { + if self.runtime_env.is_none() { + self.runtime_env = Some(Arc::new(RuntimeEnv::default())); + } + self.runtime_env + .as_ref() + .unwrap() + .register_object_store(url, object_store); + self + } + /// Builds a [`SessionState`] with the current configuration. /// /// Note that there is an explicit option for enabling catalog and schema defaults From b00723a2a0c0c1a5a30171729f7091407c819110 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Mon, 23 Sep 2024 16:31:01 +0300 Subject: [PATCH 21/25] Fix and Improve Sort Pushdown for Nested Loop and Hash Join (#12559) * Update sort_pushdown.rs * Update joins.slt --- .../src/physical_optimizer/sort_pushdown.rs | 128 ++++++++++++++++-- datafusion/sqllogictest/test_files/joins.slt | 37 +++++ 2 files changed, 153 insertions(+), 12 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 41059ef7b1ef..34410bd6b1a4 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -22,22 +22,27 @@ use super::utils::{add_sort_above, is_sort}; use crate::physical_optimizer::utils::{is_sort_preserving_merge, is_union, is_window}; use crate::physical_plan::filter::FilterExec; use crate::physical_plan::joins::utils::calculate_join_output_ordering; -use crate::physical_plan::joins::{HashJoinExec, SortMergeJoinExec}; +use crate::physical_plan::joins::SortMergeJoinExec; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::tree_node::PlanContext; use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; -use datafusion_common::tree_node::{ConcreteTreeNode, Transformed, TreeNodeRecursion}; +use datafusion_common::tree_node::{ + ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, +}; use datafusion_common::{plan_err, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{ LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; use datafusion_physical_expr_common::sort_expr::LexRequirement; +use hashbrown::HashSet; + /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total /// computational cost by pushing down `SortExec`s through some executors. The @@ -253,7 +258,6 @@ fn pushdown_requirement_to_children( || plan.as_any().is::() // TODO: Add support for Projection push down || plan.as_any().is::() - || plan.as_any().is::() || pushdown_would_violate_requirements(parent_required, plan.as_ref()) { // If the current plan is a leaf node or can not maintain any of the input ordering, can not pushed down requirements. @@ -277,15 +281,7 @@ fn pushdown_requirement_to_children( Ok(Some(vec![req])) } } else { - Ok(Some( - maintains_input_order - .into_iter() - .map(|flag| { - (flag && !parent_required.is_empty()) - .then(|| parent_required.to_vec()) - }) - .collect(), - )) + handle_custom_pushdown(plan, parent_required, maintains_input_order) } // TODO: Add support for Projection push down } @@ -483,6 +479,114 @@ fn shift_right_required( } } +/// Handles the custom pushdown of parent-required sorting requirements down to +/// the child execution plans, considering whether the input order is maintained. +/// +/// # Arguments +/// +/// * `plan` - A reference to an `ExecutionPlan` for which the pushdown will be applied. +/// * `parent_required` - The sorting requirements expected by the parent node. +/// * `maintains_input_order` - A vector of booleans indicating whether each child +/// maintains the input order. +/// +/// # Returns +/// +/// Returns `Ok(Some(Vec>))` if the sorting requirements can be +/// pushed down, `Ok(None)` if not. On error, returns a `Result::Err`. +fn handle_custom_pushdown( + plan: &Arc, + parent_required: LexRequirementRef, + maintains_input_order: Vec, +) -> Result>>> { + // If there's no requirement from the parent or the plan has no children, return early + if parent_required.is_empty() || plan.children().is_empty() { + return Ok(None); + } + + // Collect all unique column indices used in the parent-required sorting expression + let all_indices: HashSet = parent_required + .iter() + .flat_map(|order| { + collect_columns(&order.expr) + .iter() + .map(|col| col.index()) + .collect::>() + }) + .collect(); + + // Get the number of fields in each child's schema + let len_of_child_schemas: Vec = plan + .children() + .iter() + .map(|c| c.schema().fields().len()) + .collect(); + + // Find the index of the child that maintains input order + let Some(maintained_child_idx) = maintains_input_order + .iter() + .enumerate() + .find(|(_, m)| **m) + .map(|pair| pair.0) + else { + return Ok(None); + }; + + // Check if all required columns come from the child that maintains input order + let start_idx = len_of_child_schemas[..maintained_child_idx] + .iter() + .sum::(); + let end_idx = start_idx + len_of_child_schemas[maintained_child_idx]; + let all_from_maintained_child = + all_indices.iter().all(|i| i >= &start_idx && i < &end_idx); + + // If all columns are from the maintained child, update the parent requirements + if all_from_maintained_child { + let sub_offset = len_of_child_schemas + .iter() + .take(maintained_child_idx) + .sum::(); + // Transform the parent-required expression for the child schema by adjusting columns + let updated_parent_req = parent_required + .iter() + .map(|req| { + let child_schema = plan.children()[maintained_child_idx].schema(); + let updated_columns = req + .expr + .clone() + .transform_up(|expr| { + if let Some(col) = expr.as_any().downcast_ref::() { + let new_index = col.index() - sub_offset; + Ok(Transformed::yes(Arc::new(Column::new( + child_schema.field(new_index).name(), + new_index, + )))) + } else { + Ok(Transformed::no(expr)) + } + })? + .data; + Ok(PhysicalSortRequirement::new(updated_columns, req.options)) + }) + .collect::>>()?; + + // Prepare the result, populating with the updated requirements for children that maintain order + let result = maintains_input_order + .iter() + .map(|&maintains_order| { + if maintains_order { + Some(updated_parent_req.clone()) + } else { + None + } + }) + .collect(); + + Ok(Some(result)) + } else { + Ok(None) + } +} + /// Define the Requirements Compatibility #[derive(Debug)] enum RequirementsCompatibility { diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 7d0262952b31..64d5f6c7b88f 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -4151,3 +4151,40 @@ DROP TABLE sales_global; statement ok DROP TABLE exchange_rates; + +# HashJoinExec and NestedLoopJoinExec can propagate SortExec down through its right child. + +statement ok +CREATE TABLE left_table(a INT, b INT, c INT) + +statement ok +CREATE TABLE right_table(x INT, y INT, z INT) + +query TT +EXPLAIN SELECT * FROM left_table JOIN right_table ON left_table.a Date: Mon, 23 Sep 2024 13:06:10 -0400 Subject: [PATCH 22/25] Add Docs and Examples to PhysicalSortExpr, as well as `asc` and `desc` methods (#12589) --- .../core/src/datasource/listing/table.rs | 21 ++--- .../physical-expr-common/src/sort_expr.rs | 76 ++++++++++++++++++- datafusion/physical-plan/src/sorts/sort.rs | 10 +-- .../src/sorts/sort_preserving_merge.rs | 11 +-- 4 files changed, 90 insertions(+), 28 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 9142e922eb1e..5cfc1f40090a 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1252,20 +1252,13 @@ mod tests { col("int_col").sort(false, true), ]], Ok(vec![vec![ - PhysicalSortExpr { - expr: physical_col("string_col", &schema).unwrap(), - options: SortOptions { - descending: false, - nulls_first: false, - }, - }, - PhysicalSortExpr { - expr: physical_col("int_col", &schema).unwrap(), - options: SortOptions { - descending: true, - nulls_first: true, - }, - }, + PhysicalSortExpr::new_default(physical_col("string_col", &schema).unwrap()) + .asc() + .nulls_last(), + + PhysicalSortExpr::new_default(physical_col("int_col", &schema).unwrap()) + .desc() + .nulls_first() ]]) ), ]; diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 745ec543c31a..e35aec26bb23 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -30,6 +30,51 @@ use datafusion_common::Result; use datafusion_expr_common::columnar_value::ColumnarValue; /// Represents Sort operation for a column in a RecordBatch +/// +/// Example: +/// ``` +/// # use std::any::Any; +/// # use std::fmt::Display; +/// # use std::hash::Hasher; +/// # use std::sync::Arc; +/// # use arrow::array::RecordBatch; +/// # use datafusion_common::Result; +/// # use arrow::compute::SortOptions; +/// # use arrow::datatypes::{DataType, Schema}; +/// # use datafusion_expr_common::columnar_value::ColumnarValue; +/// # use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +/// # use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +/// # // this crate doesn't have a physical expression implementation +/// # // so make a really simple one +/// # #[derive(Clone, Debug, PartialEq, Eq, Hash)] +/// # struct MyPhysicalExpr; +/// # impl PhysicalExpr for MyPhysicalExpr { +/// # fn as_any(&self) -> &dyn Any {todo!() } +/// # fn data_type(&self, input_schema: &Schema) -> Result {todo!()} +/// # fn nullable(&self, input_schema: &Schema) -> Result {todo!() } +/// # fn evaluate(&self, batch: &RecordBatch) -> Result {todo!() } +/// # fn children(&self) -> Vec<&Arc> {todo!()} +/// # fn with_new_children(self: Arc, children: Vec>) -> Result> {todo!()} +/// # fn dyn_hash(&self, _state: &mut dyn Hasher) {todo!()} +/// # } +/// # impl Display for MyPhysicalExpr { +/// # fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!(f, "a") } +/// # } +/// # impl PartialEq for MyPhysicalExpr { +/// # fn eq(&self, _other: &dyn Any) -> bool { true } +/// # } +/// # fn col(name: &str) -> Arc { Arc::new(MyPhysicalExpr) } +/// // Sort by a ASC +/// let options = SortOptions::default(); +/// let sort_expr = PhysicalSortExpr::new(col("a"), options); +/// assert_eq!(sort_expr.to_string(), "a ASC"); +/// +/// // Sort by a DESC NULLS LAST +/// let sort_expr = PhysicalSortExpr::new_default(col("a")) +/// .desc() +/// .nulls_last(); +/// assert_eq!(sort_expr.to_string(), "a DESC NULLS LAST"); +/// ``` #[derive(Clone, Debug)] pub struct PhysicalSortExpr { /// Physical expression representing the column to sort @@ -43,6 +88,35 @@ impl PhysicalSortExpr { pub fn new(expr: Arc, options: SortOptions) -> Self { Self { expr, options } } + + /// Create a new PhysicalSortExpr with default [`SortOptions`] + pub fn new_default(expr: Arc) -> Self { + Self::new(expr, SortOptions::default()) + } + + /// Set the sort sort options to ASC + pub fn asc(mut self) -> Self { + self.options.descending = false; + self + } + + /// Set the sort sort options to DESC + pub fn desc(mut self) -> Self { + self.options.descending = true; + self + } + + /// Set the sort sort options to NULLS FIRST + pub fn nulls_first(mut self) -> Self { + self.options.nulls_first = true; + self + } + + /// Set the sort sort options to NULLS LAST + pub fn nulls_last(mut self) -> Self { + self.options.nulls_first = false; + self + } } impl PartialEq for PhysicalSortExpr { @@ -60,7 +134,7 @@ impl Hash for PhysicalSortExpr { } } -impl std::fmt::Display for PhysicalSortExpr { +impl Display for PhysicalSortExpr { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!(f, "{} {}", self.expr, to_str(&self.options)) } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index fa9628abdfbb..18bdedae119e 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1022,9 +1022,8 @@ mod tests { impl SortedUnboundedExec { fn compute_properties(schema: SchemaRef) -> PlanProperties { let mut eq_properties = EquivalenceProperties::new(schema); - eq_properties.add_new_orderings(vec![vec![PhysicalSortExpr::new( + eq_properties.add_new_orderings(vec![vec![PhysicalSortExpr::new_default( Arc::new(Column::new("c1", 0)), - SortOptions::default(), )]]); let mode = ExecutionMode::Unbounded; PlanProperties::new(eq_properties, Partitioning::UnknownPartitioning(1), mode) @@ -1560,10 +1559,9 @@ mod tests { cache: SortedUnboundedExec::compute_properties(Arc::new(schema.clone())), }; let mut plan = SortExec::new( - vec![PhysicalSortExpr::new( - Arc::new(Column::new("c1", 0)), - SortOptions::default(), - )], + vec![PhysicalSortExpr::new_default(Arc::new(Column::new( + "c1", 0, + )))], Arc::new(source), ); plan = plan.with_fetch(Some(9)); diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 4d333175bf75..f83bb58d08dd 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -1174,9 +1174,7 @@ mod tests { let mut eq_properties = EquivalenceProperties::new(schema); eq_properties.add_new_orderings(vec![columns .iter() - .map(|expr| { - PhysicalSortExpr::new(Arc::clone(expr), SortOptions::default()) - }) + .map(|expr| PhysicalSortExpr::new_default(Arc::clone(expr))) .collect::>()]); let mode = ExecutionMode::Unbounded; PlanProperties::new(eq_properties, Partitioning::Hash(columns, 3), mode) @@ -1286,10 +1284,9 @@ mod tests { congestion_cleared: Arc::new(Mutex::new(false)), }; let spm = SortPreservingMergeExec::new( - vec![PhysicalSortExpr::new( - Arc::new(Column::new("c1", 0)), - SortOptions::default(), - )], + vec![PhysicalSortExpr::new_default(Arc::new(Column::new( + "c1", 0, + )))], Arc::new(source), ); let spm_task = SpawnedTask::spawn(collect(Arc::new(spm), task_ctx)); From bd8960eabc06801c9984b6472661b6eaa5c2e2d5 Mon Sep 17 00:00:00 2001 From: ngli-me <107162634+ngli-me@users.noreply.github.com> Date: Mon, 23 Sep 2024 12:10:19 -0500 Subject: [PATCH 23/25] LexRequirement as a struct, instead of a type (#12583) * Converted LexRequirement into a struct. * Adjusted the wrapping to return the correct type, since the LexRequirement was not being converted after the merge. --------- Co-authored-by: nglime --- .../core/src/datasource/listing/table.rs | 5 +- .../enforce_distribution.rs | 2 +- .../physical_optimizer/projection_pushdown.rs | 13 ++-- .../src/physical_optimizer/sort_pushdown.rs | 23 ++++--- .../physical-expr-common/src/sort_expr.rs | 63 ++++++++++++++++--- .../physical-expr/src/equivalence/class.rs | 6 +- .../physical-expr/src/equivalence/mod.rs | 2 +- .../src/equivalence/properties.rs | 5 +- .../physical-plan/src/aggregates/mod.rs | 18 +++--- datafusion/physical-plan/src/sorts/sort.rs | 8 ++- datafusion/physical-plan/src/windows/mod.rs | 30 ++++++--- .../tests/cases/roundtrip_physical_plan.rs | 16 ++--- 12 files changed, 134 insertions(+), 57 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 5cfc1f40090a..6115af34e69e 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -53,6 +53,7 @@ use datafusion_physical_expr::{ use async_trait::async_trait; use datafusion_catalog::Session; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::{future, stream, StreamExt, TryStreamExt}; use itertools::Itertools; use object_store::ObjectStore; @@ -987,12 +988,12 @@ impl TableProvider for ListingTable { ))? .clone(); // Converts Vec> into type required by execution plan to specify its required input ordering - Some( + Some(LexRequirement::new( ordering .into_iter() .map(PhysicalSortRequirement::from) .collect::>(), - ) + )) } else { None }; diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 095590fe03f6..ceb701ad00c9 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1272,7 +1272,7 @@ fn ensure_distribution( // Make sure to satisfy ordering requirement: child = add_sort_above_with_check( child, - required_input_ordering.to_vec(), + required_input_ordering.clone(), None, ); } diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 61463b6ea6d5..4ca59da55bad 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -54,6 +54,7 @@ use datafusion_physical_expr::{ use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_optimizer::PhysicalOptimizerRule; use itertools::Itertools; @@ -334,10 +335,10 @@ fn try_swapping_with_output_req( return Ok(None); } - let mut updated_sort_reqs = vec![]; + let mut updated_sort_reqs = LexRequirement::new(vec![]); // None or empty_vec can be treated in the same way. if let Some(reqs) = &output_req.required_input_ordering()[0] { - for req in reqs { + for req in &reqs.inner { let Some(new_expr) = update_expr(&req.expr, projection.expr(), false)? else { return Ok(None); }; @@ -1995,7 +1996,7 @@ mod tests { let csv = create_simple_csv_exec(); let sort_req: Arc = Arc::new(OutputRequirementExec::new( csv.clone(), - Some(vec![ + Some(LexRequirement::new(vec![ PhysicalSortRequirement { expr: Arc::new(Column::new("b", 1)), options: Some(SortOptions::default()), @@ -2008,7 +2009,7 @@ mod tests { )), options: Some(SortOptions::default()), }, - ]), + ])), Distribution::HashPartitioned(vec![ Arc::new(Column::new("a", 0)), Arc::new(Column::new("b", 1)), @@ -2041,7 +2042,7 @@ mod tests { ]; assert_eq!(get_plan_string(&after_optimize), expected); - let expected_reqs = vec![ + let expected_reqs = LexRequirement::new(vec![ PhysicalSortRequirement { expr: Arc::new(Column::new("b", 2)), options: Some(SortOptions::default()), @@ -2054,7 +2055,7 @@ mod tests { )), options: Some(SortOptions::default()), }, - ]; + ]); assert_eq!( after_optimize .as_any() diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 34410bd6b1a4..c7677d725b03 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -173,7 +173,8 @@ fn pushdown_requirement_to_children( let child_plan = plan.children().swap_remove(0); match determine_children_requirement(parent_required, request_child, child_plan) { RequirementsCompatibility::Satisfy => { - let req = (!request_child.is_empty()).then(|| request_child.to_vec()); + let req = (!request_child.is_empty()) + .then(|| LexRequirement::new(request_child.to_vec())); Ok(Some(vec![req])) } RequirementsCompatibility::Compatible(adjusted) => Ok(Some(vec![adjusted])), @@ -189,7 +190,9 @@ fn pushdown_requirement_to_children( .requirements_compatible(parent_required, &sort_req) { debug_assert!(!parent_required.is_empty()); - Ok(Some(vec![Some(parent_required.to_vec())])) + Ok(Some(vec![Some(LexRequirement::new( + parent_required.to_vec(), + ))])) } else { Ok(None) } @@ -211,7 +214,8 @@ fn pushdown_requirement_to_children( .eq_properties .requirements_compatible(parent_required, &output_req) { - let req = (!parent_required.is_empty()).then(|| parent_required.to_vec()); + let req = (!parent_required.is_empty()) + .then(|| LexRequirement::new(parent_required.to_vec())); Ok(Some(vec![req])) } else { Ok(None) @@ -219,7 +223,8 @@ fn pushdown_requirement_to_children( } else if is_union(plan) { // UnionExec does not have real sort requirements for its input. Here we change the adjusted_request_ordering to UnionExec's output ordering and // propagate the sort requirements down to correct the unnecessary descendant SortExec under the UnionExec - let req = (!parent_required.is_empty()).then(|| parent_required.to_vec()); + let req = (!parent_required.is_empty()) + .then(|| LexRequirement::new(parent_required.to_vec())); Ok(Some(vec![req; plan.children().len()])) } else if let Some(smj) = plan.as_any().downcast_ref::() { // If the current plan is SortMergeJoinExec @@ -277,7 +282,8 @@ fn pushdown_requirement_to_children( } else { // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. - let req = (!parent_required.is_empty()).then(|| parent_required.to_vec()); + let req = (!parent_required.is_empty()) + .then(|| LexRequirement::new(parent_required.to_vec())); Ok(Some(vec![req])) } } else { @@ -331,7 +337,8 @@ fn determine_children_requirement( { // Parent requirements are more specific, adjust child's requirements // and push down the new requirements: - let adjusted = (!parent_required.is_empty()).then(|| parent_required.to_vec()); + let adjusted = (!parent_required.is_empty()) + .then(|| LexRequirement::new(parent_required.to_vec())); RequirementsCompatibility::Compatible(adjusted) } else { RequirementsCompatibility::NonCompatible @@ -471,7 +478,7 @@ fn shift_right_required( }) .collect::>(); if new_right_required.len() == parent_required.len() { - Ok(new_right_required) + Ok(LexRequirement::new(new_right_required)) } else { plan_err!( "Expect to shift all the parent required column indexes for SortMergeJoin" @@ -574,7 +581,7 @@ fn handle_custom_pushdown( .iter() .map(|&maintains_order| { if maintains_order { - Some(updated_parent_req.clone()) + Some(LexRequirement::new(updated_parent_req.clone())) } else { None } diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index e35aec26bb23..be183b72b38b 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -19,6 +19,7 @@ use std::fmt::Display; use std::hash::{Hash, Hasher}; +use std::ops::Deref; use std::sync::Arc; use crate::physical_expr::PhysicalExpr; @@ -296,11 +297,13 @@ impl PhysicalSortRequirement { pub fn from_sort_exprs<'a>( ordering: impl IntoIterator, ) -> LexRequirement { - ordering - .into_iter() - .cloned() - .map(PhysicalSortRequirement::from) - .collect() + LexRequirement::new( + ordering + .into_iter() + .cloned() + .map(PhysicalSortRequirement::from) + .collect(), + ) } /// Converts an iterator of [`PhysicalSortRequirement`] into a Vec @@ -338,9 +341,55 @@ pub type LexOrdering = Vec; /// a reference to a lexicographical ordering. pub type LexOrderingRef<'a> = &'a [PhysicalSortExpr]; -///`LexRequirement` is an alias for the type `Vec`, which +///`LexRequirement` is an struct containing a `Vec`, which /// represents a lexicographical ordering requirement. -pub type LexRequirement = Vec; +#[derive(Debug, Default, Clone, PartialEq)] +pub struct LexRequirement { + pub inner: Vec, +} + +impl LexRequirement { + pub fn new(inner: Vec) -> Self { + Self { inner } + } + + pub fn iter(&self) -> impl Iterator { + self.inner.iter() + } + + pub fn push(&mut self, physical_sort_requirement: PhysicalSortRequirement) { + self.inner.push(physical_sort_requirement) + } +} + +impl Deref for LexRequirement { + type Target = [PhysicalSortRequirement]; + + fn deref(&self) -> &Self::Target { + self.inner.as_slice() + } +} + +impl FromIterator for LexRequirement { + fn from_iter>(iter: T) -> Self { + let mut lex_requirement = LexRequirement::new(vec![]); + + for i in iter { + lex_requirement.inner.push(i); + } + + lex_requirement + } +} + +impl IntoIterator for LexRequirement { + type Item = PhysicalSortRequirement; + type IntoIter = std::vec::IntoIter; + + fn into_iter(self) -> Self::IntoIter { + self.inner.into_iter() + } +} ///`LexRequirementRef` is an alias for the type &`[PhysicalSortRequirement]`, which /// represents a reference to a lexicographical ordering requirement. diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 0296b7a247d6..a0396ea1333f 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -418,7 +418,7 @@ impl EquivalenceGroup { // Normalize the requirements: let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); // Convert sort requirements back to sort expressions: - PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs) + PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs.inner) } /// This function applies the `normalize_sort_requirement` function for all @@ -428,12 +428,12 @@ impl EquivalenceGroup { &self, sort_reqs: LexRequirementRef, ) -> LexRequirement { - collapse_lex_req( + collapse_lex_req(LexRequirement::new( sort_reqs .iter() .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) .collect(), - ) + )) } /// Projects `expr` according to the given projection mapping. diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index d862eda5018e..50b95c4454ce 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -48,7 +48,7 @@ pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { output.push(item); } } - output + LexRequirement::new(output) } /// Adds the `offset` value to `Column` indices inside `expr`. This function is diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index a5d54ee56cff..365b51e1a460 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -515,8 +515,9 @@ impl EquivalenceProperties { ) -> Option { let mut lhs = self.normalize_sort_requirements(req1); let mut rhs = self.normalize_sort_requirements(req2); - lhs.iter_mut() - .zip(rhs.iter_mut()) + lhs.inner + .iter_mut() + .zip(rhs.inner.iter_mut()) .all(|(lhs, rhs)| { lhs.expr.eq(&rhs.expr) && match (lhs.options, rhs.options) { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index c3bc7b042e65..e0f19757385e 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -370,13 +370,15 @@ impl AggregateExec { // prefix requirements with this section. In this case, aggregation will // work more efficiently. let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); - let mut new_requirement = indices - .iter() - .map(|&idx| PhysicalSortRequirement { - expr: Arc::clone(&groupby_exprs[idx]), - options: None, - }) - .collect::>(); + let mut new_requirement = LexRequirement::new( + indices + .iter() + .map(|&idx| PhysicalSortRequirement { + expr: Arc::clone(&groupby_exprs[idx]), + options: None, + }) + .collect::>(), + ); let req = get_finer_aggregate_exprs_requirement( &mut aggr_expr, @@ -384,7 +386,7 @@ impl AggregateExec { input_eq_properties, &mode, )?; - new_requirement.extend(req); + new_requirement.inner.extend(req); new_requirement = collapse_lex_req(new_requirement); // If our aggregation has grouping sets then our base grouping exprs will diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 18bdedae119e..fb03ceb15c37 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -792,7 +792,9 @@ impl SortExec { ) -> PlanProperties { // Determine execution mode: let sort_satisfied = input.equivalence_properties().ordering_satisfy_requirement( - PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()).as_slice(), + PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()) + .inner + .as_slice(), ); let mode = match input.execution_mode() { ExecutionMode::Unbounded if sort_satisfied => ExecutionMode::Unbounded, @@ -895,7 +897,9 @@ impl ExecutionPlan for SortExec { .input .equivalence_properties() .ordering_satisfy_requirement( - PhysicalSortRequirement::from_sort_exprs(self.expr.iter()).as_slice(), + PhysicalSortRequirement::from_sort_exprs(self.expr.iter()) + .inner + .as_slice(), ); match (sort_satisfied, self.fetch.as_ref()) { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 981a8e285166..3507b359ddc3 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -399,12 +399,14 @@ pub(crate) fn calc_requirements< partition_by_exprs: impl IntoIterator, orderby_sort_exprs: impl IntoIterator, ) -> Option { - let mut sort_reqs = partition_by_exprs - .into_iter() - .map(|partition_by| { - PhysicalSortRequirement::new(Arc::clone(partition_by.borrow()), None) - }) - .collect::>(); + let mut sort_reqs = LexRequirement::new( + partition_by_exprs + .into_iter() + .map(|partition_by| { + PhysicalSortRequirement::new(Arc::clone(partition_by.borrow()), None) + }) + .collect::>(), + ); for element in orderby_sort_exprs.into_iter() { let PhysicalSortExpr { expr, options } = element.borrow(); if !sort_reqs.iter().any(|e| e.expr.eq(expr)) { @@ -568,12 +570,18 @@ pub fn get_window_mode( input: &Arc, ) -> Option<(bool, InputOrderMode)> { let input_eqs = input.equivalence_properties().clone(); - let mut partition_by_reqs: LexRequirement = vec![]; + let mut partition_by_reqs: LexRequirement = LexRequirement::new(vec![]); let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); - partition_by_reqs.extend(indices.iter().map(|&idx| PhysicalSortRequirement { + vec![].extend(indices.iter().map(|&idx| PhysicalSortRequirement { expr: Arc::clone(&partitionby_exprs[idx]), options: None, })); + partition_by_reqs + .inner + .extend(indices.iter().map(|&idx| PhysicalSortRequirement { + expr: Arc::clone(&partitionby_exprs[idx]), + options: None, + })); // Treat partition by exprs as constant. During analysis of requirements are satisfied. let const_exprs = partitionby_exprs.iter().map(ConstExpr::from); let partition_by_eqs = input_eqs.add_constants(const_exprs); @@ -583,7 +591,9 @@ pub fn get_window_mode( for (should_swap, order_by_reqs) in [(false, order_by_reqs), (true, reverse_order_by_reqs)] { - let req = [partition_by_reqs.clone(), order_by_reqs].concat(); + let req = LexRequirement::new( + [partition_by_reqs.inner.clone(), order_by_reqs.inner].concat(), + ); let req = collapse_lex_req(req); if partition_by_eqs.ordering_satisfy_requirement(&req) { // Window can be run with existing ordering @@ -736,7 +746,7 @@ mod tests { if let Some(expected) = &mut expected { expected.push(res); } else { - expected = Some(vec![res]); + expected = Some(LexRequirement::new(vec![res])); } } assert_eq!(calc_requirements(partitionbys, orderbys), expected); diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index f4b32e662ea9..f6f2683839b7 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -50,7 +50,9 @@ use datafusion::functions_aggregate::sum::sum_udaf; use datafusion::logical_expr::{create_udf, JoinType, Operator, Volatility}; use datafusion::physical_expr::expressions::Literal; use datafusion::physical_expr::window::SlidingAggregateWindowExpr; -use datafusion::physical_expr::{PhysicalSortRequirement, ScalarFunctionExpr}; +use datafusion::physical_expr::{ + LexRequirement, PhysicalSortRequirement, ScalarFunctionExpr, +}; use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; @@ -1148,13 +1150,13 @@ fn roundtrip_json_sink() -> Result<()> { file_sink_config, JsonWriterOptions::new(CompressionTypeVariant::UNCOMPRESSED), )); - let sort_order = vec![PhysicalSortRequirement::new( + let sort_order = LexRequirement::new(vec![PhysicalSortRequirement::new( Arc::new(Column::new("plan_type", 0)), Some(SortOptions { descending: true, nulls_first: false, }), - )]; + )]); roundtrip_test(Arc::new(DataSinkExec::new( input, @@ -1184,13 +1186,13 @@ fn roundtrip_csv_sink() -> Result<()> { file_sink_config, CsvWriterOptions::new(WriterBuilder::default(), CompressionTypeVariant::ZSTD), )); - let sort_order = vec![PhysicalSortRequirement::new( + let sort_order = LexRequirement::new(vec![PhysicalSortRequirement::new( Arc::new(Column::new("plan_type", 0)), Some(SortOptions { descending: true, nulls_first: false, }), - )]; + )]); let ctx = SessionContext::new(); let codec = DefaultPhysicalExtensionCodec {}; @@ -1243,13 +1245,13 @@ fn roundtrip_parquet_sink() -> Result<()> { file_sink_config, TableParquetOptions::default(), )); - let sort_order = vec![PhysicalSortRequirement::new( + let sort_order = LexRequirement::new(vec![PhysicalSortRequirement::new( Arc::new(Column::new("plan_type", 0)), Some(SortOptions { descending: true, nulls_first: false, }), - )]; + )]); roundtrip_test(Arc::new(DataSinkExec::new( input, From 99b5673fa2960a26dfaf72c7dfdf9cfb6d7fba33 Mon Sep 17 00:00:00 2001 From: Samuel Colvin Date: Mon, 23 Sep 2024 18:14:20 +0100 Subject: [PATCH 24/25] Warn instead of error for unused imports (#12588) * warn not error on unused imports * pre-commit error on clippy --- Cargo.toml | 1 - pre-commit.sh | 2 ++ 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index e8cd52315a51..629992177913 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -169,4 +169,3 @@ large_futures = "warn" [workspace.lints.rust] unexpected_cfgs = { level = "warn", check-cfg = ["cfg(tarpaulin)"] } -unused_imports = "deny" diff --git a/pre-commit.sh b/pre-commit.sh index c81dd9f8e5e8..7ab121919861 100755 --- a/pre-commit.sh +++ b/pre-commit.sh @@ -25,6 +25,8 @@ # This file be run directly: # $ ./pre-commit.sh +set -e + function RED() { echo "\033[0;31m$@\033[0m" } From 2e274bfbdfc19f11b2951456bb2a48e88733c9bf Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 23 Sep 2024 13:15:35 -0400 Subject: [PATCH 25/25] Require `Debug` for `AnalyzerRule`, `FunctionRewriter`, and `OptimizerRule` (#12556) --- datafusion-examples/examples/optimizer_rule.rs | 1 + datafusion/core/src/execution/session_state.rs | 1 + datafusion/core/tests/user_defined/user_defined_plan.rs | 3 +++ datafusion/expr/src/expr_rewriter/mod.rs | 3 ++- datafusion/optimizer/src/analyzer/count_wildcard_rule.rs | 2 +- datafusion/optimizer/src/analyzer/expand_wildcard_rule.rs | 2 +- datafusion/optimizer/src/analyzer/function_rewrite.rs | 2 +- datafusion/optimizer/src/analyzer/inline_table_scan.rs | 2 +- datafusion/optimizer/src/analyzer/mod.rs | 6 ++++-- datafusion/optimizer/src/analyzer/type_coercion.rs | 2 +- datafusion/optimizer/src/common_subexpr_eliminate.rs | 1 + .../optimizer/src/decorrelate_predicate_subquery.rs | 2 +- datafusion/optimizer/src/eliminate_cross_join.rs | 2 +- datafusion/optimizer/src/eliminate_duplicated_expr.rs | 2 +- datafusion/optimizer/src/eliminate_filter.rs | 2 +- datafusion/optimizer/src/eliminate_group_by_constant.rs | 2 +- datafusion/optimizer/src/eliminate_join.rs | 2 +- datafusion/optimizer/src/eliminate_limit.rs | 2 +- datafusion/optimizer/src/eliminate_nested_union.rs | 2 +- datafusion/optimizer/src/eliminate_one_union.rs | 2 +- datafusion/optimizer/src/eliminate_outer_join.rs | 2 +- datafusion/optimizer/src/extract_equijoin_predicate.rs | 2 +- datafusion/optimizer/src/filter_null_join_keys.rs | 2 +- datafusion/optimizer/src/optimize_projections/mod.rs | 2 +- datafusion/optimizer/src/optimizer.rs | 8 ++++++-- datafusion/optimizer/src/propagate_empty_relation.rs | 2 +- datafusion/optimizer/src/push_down_filter.rs | 2 +- datafusion/optimizer/src/push_down_limit.rs | 2 +- datafusion/optimizer/src/replace_distinct_aggregate.rs | 2 +- datafusion/optimizer/src/rewrite_disjunctive_predicate.rs | 2 +- datafusion/optimizer/src/scalar_subquery_to_join.rs | 2 +- .../optimizer/src/simplify_expressions/simplify_exprs.rs | 2 +- datafusion/optimizer/src/single_distinct_to_groupby.rs | 2 +- datafusion/optimizer/src/unwrap_cast_in_comparison.rs | 2 +- 34 files changed, 45 insertions(+), 32 deletions(-) diff --git a/datafusion-examples/examples/optimizer_rule.rs b/datafusion-examples/examples/optimizer_rule.rs index b4663b345f64..e0b552620a9a 100644 --- a/datafusion-examples/examples/optimizer_rule.rs +++ b/datafusion-examples/examples/optimizer_rule.rs @@ -100,6 +100,7 @@ pub async fn main() -> Result<()> { /// An example OptimizerRule that replaces all `col = ` predicates with a /// user defined function +#[derive(Default, Debug)] struct MyOptimizerRule {} impl OptimizerRule for MyOptimizerRule { diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index e447220a1fba..3e6577a48608 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -1954,6 +1954,7 @@ mod tests { #[test] fn test_session_state_with_optimizer_rules() { + #[derive(Default, Debug)] struct DummyRule {} impl OptimizerRule for DummyRule { diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 101e676484d2..caf639434a99 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -335,7 +335,9 @@ impl QueryPlanner for TopKQueryPlanner { } } +#[derive(Default, Debug)] struct TopKOptimizerRule {} + impl OptimizerRule for TopKOptimizerRule { fn name(&self) -> &str { "topk" @@ -686,6 +688,7 @@ impl RecordBatchStream for TopKReader { } } +#[derive(Default, Debug)] struct MyAnalyzerRule {} impl AnalyzerRule for MyAnalyzerRule { diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index 05ee3e112d62..15930914dd59 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -19,6 +19,7 @@ use std::collections::HashMap; use std::collections::HashSet; +use std::fmt::Debug; use std::sync::Arc; use crate::expr::{Alias, Sort, Unnest}; @@ -42,7 +43,7 @@ pub use order_by::rewrite_sort_cols_by_aggs; /// `Operator::ArrowAt`, but can be implemented by calling a function /// `array_concat` from the `functions-nested` crate. // This is not used in datafusion internally, but it is still helpful for downstream project so don't remove it. -pub trait FunctionRewrite { +pub trait FunctionRewrite: Debug { /// Return a human readable name for this rewrite fn name(&self) -> &str; diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 32ca790b0094..86520b3587cd 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -28,7 +28,7 @@ use datafusion_expr::{lit, Expr, LogicalPlan, WindowFunctionDefinition}; /// Rewrite `Count(Expr:Wildcard)` to `Count(Expr:Literal)`. /// /// Resolves issue: -#[derive(Default)] +#[derive(Default, Debug)] pub struct CountWildcardRule {} impl CountWildcardRule { diff --git a/datafusion/optimizer/src/analyzer/expand_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/expand_wildcard_rule.rs index b2de541a34c7..a26ec4be5c85 100644 --- a/datafusion/optimizer/src/analyzer/expand_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/expand_wildcard_rule.rs @@ -28,7 +28,7 @@ use datafusion_expr::utils::{ }; use datafusion_expr::{Expr, LogicalPlan, Projection, SubqueryAlias}; -#[derive(Default)] +#[derive(Default, Debug)] pub struct ExpandWildcardRule {} impl ExpandWildcardRule { diff --git a/datafusion/optimizer/src/analyzer/function_rewrite.rs b/datafusion/optimizer/src/analyzer/function_rewrite.rs index ec3626b2c899..c6bf14ebce2e 100644 --- a/datafusion/optimizer/src/analyzer/function_rewrite.rs +++ b/datafusion/optimizer/src/analyzer/function_rewrite.rs @@ -29,7 +29,7 @@ use datafusion_expr::LogicalPlan; use std::sync::Arc; /// Analyzer rule that invokes [`FunctionRewrite`]s on expressions -#[derive(Default)] +#[derive(Default, Debug)] pub struct ApplyFunctionRewrites { /// Expr --> Function writes to apply function_rewrites: Vec>, diff --git a/datafusion/optimizer/src/analyzer/inline_table_scan.rs b/datafusion/optimizer/src/analyzer/inline_table_scan.rs index 2fc9b91da6cb..342d85a915b4 100644 --- a/datafusion/optimizer/src/analyzer/inline_table_scan.rs +++ b/datafusion/optimizer/src/analyzer/inline_table_scan.rs @@ -28,7 +28,7 @@ use datafusion_expr::{logical_plan::LogicalPlan, Expr, LogicalPlanBuilder}; /// Analyzed rule that inlines TableScan that provide a [`LogicalPlan`] /// (DataFrame / ViewTable) -#[derive(Default)] +#[derive(Default, Debug)] pub struct InlineTableScan; impl InlineTableScan { diff --git a/datafusion/optimizer/src/analyzer/mod.rs b/datafusion/optimizer/src/analyzer/mod.rs index 6e2afeca88c9..4cd891664e7f 100644 --- a/datafusion/optimizer/src/analyzer/mod.rs +++ b/datafusion/optimizer/src/analyzer/mod.rs @@ -16,6 +16,8 @@ // under the License. //! [`Analyzer`] and [`AnalyzerRule`] + +use std::fmt::Debug; use std::sync::Arc; use log::debug; @@ -60,7 +62,7 @@ pub mod type_coercion; /// `AnalyzerRule`s. /// /// [`SessionState::add_analyzer_rule`]: https://docs.rs/datafusion/latest/datafusion/execution/session_state/struct.SessionState.html#method.add_analyzer_rule -pub trait AnalyzerRule { +pub trait AnalyzerRule: Debug { /// Rewrite `plan` fn analyze(&self, plan: LogicalPlan, config: &ConfigOptions) -> Result; @@ -72,7 +74,7 @@ pub trait AnalyzerRule { /// /// An `Analyzer` transforms a `LogicalPlan` /// prior to the rest of the DataFusion optimization process. -#[derive(Clone)] +#[derive(Clone, Debug)] pub struct Analyzer { /// Expr --> Function writes to apply prior to analysis passes pub function_rewrites: Vec>, diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 3e31ac1c6375..4dc34284c719 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -57,7 +57,7 @@ use datafusion_expr::{ /// Performs type coercion by determining the schema /// and performing the expression rewrites. -#[derive(Default)] +#[derive(Default, Debug)] pub struct TypeCoercion {} impl TypeCoercion { diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 442cc93f5b93..c13cb3a8e973 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -139,6 +139,7 @@ type CommonExprs<'n> = IndexMap, (Expr, String)>; /// ProjectionExec(exprs=[extract (day from new_col), extract (year from new_col)]) <-- reuse here /// ProjectionExec(exprs=[to_date(c1) as new_col]) <-- compute to_date once /// ``` +#[derive(Debug)] pub struct CommonSubexprEliminate { random_state: RandomState, } diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index 5bb57bf05f62..d1ac80003ba7 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -40,7 +40,7 @@ use datafusion_expr::{ use log::debug; /// Optimizer rule for rewriting predicate(IN/EXISTS) subquery to left semi/anti joins -#[derive(Default)] +#[derive(Default, Debug)] pub struct DecorrelatePredicateSubquery {} impl DecorrelatePredicateSubquery { diff --git a/datafusion/optimizer/src/eliminate_cross_join.rs b/datafusion/optimizer/src/eliminate_cross_join.rs index 93df0dcfd500..550728ddd3f9 100644 --- a/datafusion/optimizer/src/eliminate_cross_join.rs +++ b/datafusion/optimizer/src/eliminate_cross_join.rs @@ -30,7 +30,7 @@ use datafusion_expr::logical_plan::{ use datafusion_expr::utils::{can_hash, find_valid_equijoin_key_pair}; use datafusion_expr::{build_join_schema, ExprSchemable, Operator}; -#[derive(Default)] +#[derive(Default, Debug)] pub struct EliminateCrossJoin; impl EliminateCrossJoin { diff --git a/datafusion/optimizer/src/eliminate_duplicated_expr.rs b/datafusion/optimizer/src/eliminate_duplicated_expr.rs index c460d7a93d26..554985667fdf 100644 --- a/datafusion/optimizer/src/eliminate_duplicated_expr.rs +++ b/datafusion/optimizer/src/eliminate_duplicated_expr.rs @@ -26,7 +26,7 @@ use datafusion_expr::{Aggregate, Expr, Sort, SortExpr}; use indexmap::IndexSet; use std::hash::{Hash, Hasher}; /// Optimization rule that eliminate duplicated expr. -#[derive(Default)] +#[derive(Default, Debug)] pub struct EliminateDuplicatedExpr; impl EliminateDuplicatedExpr { diff --git a/datafusion/optimizer/src/eliminate_filter.rs b/datafusion/optimizer/src/eliminate_filter.rs index bb2b4547e9c2..4ed2ac8ba1a4 100644 --- a/datafusion/optimizer/src/eliminate_filter.rs +++ b/datafusion/optimizer/src/eliminate_filter.rs @@ -30,7 +30,7 @@ use crate::{OptimizerConfig, OptimizerRule}; /// /// This saves time in planning and executing the query. /// Note that this rule should be applied after simplify expressions optimizer rule. -#[derive(Default)] +#[derive(Default, Debug)] pub struct EliminateFilter; impl EliminateFilter { diff --git a/datafusion/optimizer/src/eliminate_group_by_constant.rs b/datafusion/optimizer/src/eliminate_group_by_constant.rs index c7869d9e4dd7..13d03d647fe2 100644 --- a/datafusion/optimizer/src/eliminate_group_by_constant.rs +++ b/datafusion/optimizer/src/eliminate_group_by_constant.rs @@ -26,7 +26,7 @@ use datafusion_expr::{Aggregate, Expr, LogicalPlan, LogicalPlanBuilder, Volatili /// Optimizer rule that removes constant expressions from `GROUP BY` clause /// and places additional projection on top of aggregation, to preserve /// original schema -#[derive(Default)] +#[derive(Default, Debug)] pub struct EliminateGroupByConstant {} impl EliminateGroupByConstant { diff --git a/datafusion/optimizer/src/eliminate_join.rs b/datafusion/optimizer/src/eliminate_join.rs index b15d981d1180..f9b79e036f9b 100644 --- a/datafusion/optimizer/src/eliminate_join.rs +++ b/datafusion/optimizer/src/eliminate_join.rs @@ -28,7 +28,7 @@ use datafusion_expr::{ /// Eliminates joins when join condition is false. /// Replaces joins when inner join condition is true with a cross join. -#[derive(Default)] +#[derive(Default, Debug)] pub struct EliminateJoin; impl EliminateJoin { diff --git a/datafusion/optimizer/src/eliminate_limit.rs b/datafusion/optimizer/src/eliminate_limit.rs index 2503475bd8df..25304d4ccafa 100644 --- a/datafusion/optimizer/src/eliminate_limit.rs +++ b/datafusion/optimizer/src/eliminate_limit.rs @@ -30,7 +30,7 @@ use std::sync::Arc; /// plan with an empty relation. /// /// This rule also removes OFFSET 0 from the [LogicalPlan] -#[derive(Default)] +#[derive(Default, Debug)] pub struct EliminateLimit; impl EliminateLimit { diff --git a/datafusion/optimizer/src/eliminate_nested_union.rs b/datafusion/optimizer/src/eliminate_nested_union.rs index 965771326854..94da08243d78 100644 --- a/datafusion/optimizer/src/eliminate_nested_union.rs +++ b/datafusion/optimizer/src/eliminate_nested_union.rs @@ -25,7 +25,7 @@ use datafusion_expr::{Distinct, LogicalPlan, Union}; use itertools::Itertools; use std::sync::Arc; -#[derive(Default)] +#[derive(Default, Debug)] /// An optimization rule that replaces nested unions with a single union. pub struct EliminateNestedUnion; diff --git a/datafusion/optimizer/src/eliminate_one_union.rs b/datafusion/optimizer/src/eliminate_one_union.rs index 7a1c4e118e05..3e027811420c 100644 --- a/datafusion/optimizer/src/eliminate_one_union.rs +++ b/datafusion/optimizer/src/eliminate_one_union.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use crate::optimizer::ApplyOrder; -#[derive(Default)] +#[derive(Default, Debug)] /// An optimization rule that eliminates union with one element. pub struct EliminateOneUnion; diff --git a/datafusion/optimizer/src/eliminate_outer_join.rs b/datafusion/optimizer/src/eliminate_outer_join.rs index e7c88df55122..1ecb32ca2a43 100644 --- a/datafusion/optimizer/src/eliminate_outer_join.rs +++ b/datafusion/optimizer/src/eliminate_outer_join.rs @@ -26,7 +26,6 @@ use datafusion_common::tree_node::Transformed; use datafusion_expr::expr::{BinaryExpr, Cast, TryCast}; use std::sync::Arc; -#[derive(Default)] /// /// Attempt to replace outer joins with inner joins. /// @@ -49,6 +48,7 @@ use std::sync::Arc; /// filters from the WHERE clause return false while any inputs are /// null and columns of those quals are come from nullable side of /// outer join. +#[derive(Default, Debug)] pub struct EliminateOuterJoin; impl EliminateOuterJoin { diff --git a/datafusion/optimizer/src/extract_equijoin_predicate.rs b/datafusion/optimizer/src/extract_equijoin_predicate.rs index 0dae777ab5bd..48191ec20631 100644 --- a/datafusion/optimizer/src/extract_equijoin_predicate.rs +++ b/datafusion/optimizer/src/extract_equijoin_predicate.rs @@ -38,7 +38,7 @@ type EquijoinPredicate = (Expr, Expr); /// has one equijoin predicate (`A.x = B.y`) and one filter predicate (`B.z > 50`). /// See [find_valid_equijoin_key_pair] for more information on what predicates /// are considered equijoins. -#[derive(Default)] +#[derive(Default, Debug)] pub struct ExtractEquijoinPredicate; impl ExtractEquijoinPredicate { diff --git a/datafusion/optimizer/src/filter_null_join_keys.rs b/datafusion/optimizer/src/filter_null_join_keys.rs index caf94a2c88bd..66c7463c3d5d 100644 --- a/datafusion/optimizer/src/filter_null_join_keys.rs +++ b/datafusion/optimizer/src/filter_null_join_keys.rs @@ -29,7 +29,7 @@ use std::sync::Arc; /// The FilterNullJoinKeys rule will identify joins with equi-join conditions /// where the join key is nullable and then insert an `IsNotNull` filter on the nullable side since null values /// can never match. -#[derive(Default)] +#[derive(Default, Debug)] pub struct FilterNullJoinKeys {} impl OptimizerRule for FilterNullJoinKeys { diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index 8c61d45cc883..5ab427a31699 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -57,7 +57,7 @@ use datafusion_common::tree_node::{ /// The rule analyzes the input logical plan, determines the necessary column /// indices, and then removes any unnecessary columns. It also removes any /// unnecessary projections from the plan tree. -#[derive(Default)] +#[derive(Default, Debug)] pub struct OptimizeProjections {} impl OptimizeProjections { diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 2fc560cceeb4..08dcefa22f08 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -18,6 +18,7 @@ //! [`Optimizer`] and [`OptimizerRule`] use std::collections::HashSet; +use std::fmt::Debug; use std::sync::Arc; use chrono::{DateTime, Utc}; @@ -70,7 +71,7 @@ use crate::utils::log_plan; /// [`AnalyzerRule`]: crate::analyzer::AnalyzerRule /// [`SessionState::add_optimizer_rule`]: https://docs.rs/datafusion/latest/datafusion/execution/session_state/struct.SessionState.html#method.add_optimizer_rule -pub trait OptimizerRule { +pub trait OptimizerRule: Debug { /// Try and rewrite `plan` to an optimized form, returning None if the plan /// cannot be optimized by this rule. /// @@ -214,7 +215,7 @@ impl OptimizerConfig for OptimizerContext { } /// A rule-based optimizer. -#[derive(Clone)] +#[derive(Clone, Debug)] pub struct Optimizer { /// All optimizer rules to apply pub rules: Vec>, @@ -666,6 +667,7 @@ mod tests { fn observe(_plan: &LogicalPlan, _rule: &dyn OptimizerRule) {} + #[derive(Default, Debug)] struct BadRule {} impl OptimizerRule for BadRule { @@ -687,6 +689,7 @@ mod tests { } /// Replaces whatever plan with a single table scan + #[derive(Default, Debug)] struct GetTableScanRule {} impl OptimizerRule for GetTableScanRule { @@ -713,6 +716,7 @@ mod tests { /// A goofy rule doing rotation of columns in all projections. /// /// Useful to test cycle detection. + #[derive(Default, Debug)] struct RotateProjectionRule { // reverse exprs instead of rotating on the first pass reverse_on_first_pass: Mutex, diff --git a/datafusion/optimizer/src/propagate_empty_relation.rs b/datafusion/optimizer/src/propagate_empty_relation.rs index 61d9934d897a..b5e1077ee5be 100644 --- a/datafusion/optimizer/src/propagate_empty_relation.rs +++ b/datafusion/optimizer/src/propagate_empty_relation.rs @@ -29,7 +29,7 @@ use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; /// Optimization rule that bottom-up to eliminate plan by propagating empty_relation. -#[derive(Default)] +#[derive(Default, Debug)] pub struct PropagateEmptyRelation; impl PropagateEmptyRelation { diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index a306ff7d2dbc..0787d4006ff4 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -130,7 +130,7 @@ use crate::{OptimizerConfig, OptimizerRule}; /// reaches a plan node that does not commute with that filter, it adds the /// filter to that place. When it passes through a projection, it re-writes the /// filter's expression taking into account that projection. -#[derive(Default)] +#[derive(Default, Debug)] pub struct PushDownFilter {} /// For a given JOIN type, determine whether each input of the join is preserved diff --git a/datafusion/optimizer/src/push_down_limit.rs b/datafusion/optimizer/src/push_down_limit.rs index ab7880213692..158c7592df51 100644 --- a/datafusion/optimizer/src/push_down_limit.rs +++ b/datafusion/optimizer/src/push_down_limit.rs @@ -31,7 +31,7 @@ use datafusion_expr::logical_plan::{Join, JoinType, Limit, LogicalPlan}; /// Optimization rule that tries to push down `LIMIT`. /// //. It will push down through projection, limits (taking the smaller limit) -#[derive(Default)] +#[derive(Default, Debug)] pub struct PushDownLimit {} impl PushDownLimit { diff --git a/datafusion/optimizer/src/replace_distinct_aggregate.rs b/datafusion/optimizer/src/replace_distinct_aggregate.rs index c887192f6370..c026130c426f 100644 --- a/datafusion/optimizer/src/replace_distinct_aggregate.rs +++ b/datafusion/optimizer/src/replace_distinct_aggregate.rs @@ -54,7 +54,7 @@ use datafusion_expr::{Aggregate, Distinct, DistinctOn, Expr, LogicalPlan}; /// ``` /// Optimizer that replaces logical [[Distinct]] with a logical [[Aggregate]] -#[derive(Default)] +#[derive(Default, Debug)] pub struct ReplaceDistinctWithAggregate {} impl ReplaceDistinctWithAggregate { diff --git a/datafusion/optimizer/src/rewrite_disjunctive_predicate.rs b/datafusion/optimizer/src/rewrite_disjunctive_predicate.rs index 897afda267dc..a6b633fdb8fe 100644 --- a/datafusion/optimizer/src/rewrite_disjunctive_predicate.rs +++ b/datafusion/optimizer/src/rewrite_disjunctive_predicate.rs @@ -122,7 +122,7 @@ use datafusion_expr::{Expr, LogicalPlan, Operator}; /// ) /// ``` /// -#[derive(Default)] +#[derive(Default, Debug)] pub struct RewriteDisjunctivePredicate; impl RewriteDisjunctivePredicate { diff --git a/datafusion/optimizer/src/scalar_subquery_to_join.rs b/datafusion/optimizer/src/scalar_subquery_to_join.rs index e5c8757405cf..6409bb9e03f7 100644 --- a/datafusion/optimizer/src/scalar_subquery_to_join.rs +++ b/datafusion/optimizer/src/scalar_subquery_to_join.rs @@ -36,7 +36,7 @@ use datafusion_expr::utils::conjunction; use datafusion_expr::{expr, EmptyRelation, Expr, LogicalPlan, LogicalPlanBuilder}; /// Optimizer rule for rewriting subquery filters to joins -#[derive(Default)] +#[derive(Default, Debug)] pub struct ScalarSubqueryToJoin {} impl ScalarSubqueryToJoin { diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index c4dba4c8df73..c0142ae0fc5a 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -45,7 +45,7 @@ use super::ExprSimplifier; /// `Filter: b > 2` /// /// [`Expr`]: datafusion_expr::Expr -#[derive(Default)] +#[derive(Default, Debug)] pub struct SimplifyExpressions {} impl OptimizerRule for SimplifyExpressions { diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 2509e67dbec7..1c22c2a4375a 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -49,7 +49,7 @@ use hashbrown::HashSet; /// ) /// GROUP BY a /// ``` -#[derive(Default)] +#[derive(Default, Debug)] pub struct SingleDistinctToGroupBy {} const SINGLE_DISTINCT_ALIAS: &str = "alias1"; diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index 2118ae96a198..22e3c0ddd076 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -72,7 +72,7 @@ use datafusion_expr::{lit, Expr, ExprSchemable, LogicalPlan}; /// Filter: c1 > INT32(10) /// ``` /// -#[derive(Default)] +#[derive(Default, Debug)] pub struct UnwrapCastInComparison {} impl UnwrapCastInComparison {