From da3f6afec51e4272622cb74ec62a11d13df17267 Mon Sep 17 00:00:00 2001 From: Albert Skalt <133099191+askalt@users.noreply.github.com> Date: Mon, 26 Aug 2024 22:10:16 +0300 Subject: [PATCH 01/75] Add ability to return LogicalPlan by value from TableProvider (#12113) This patch changes the `get_logical_plan(...)` method signature. Now it returns a `Cow` to allow an implementation to return plan by value. --- datafusion/catalog/src/table.rs | 5 ++- datafusion/core/src/dataframe/mod.rs | 5 ++- .../core/src/datasource/cte_worktable.rs | 4 +- .../src/datasource/default_table_source.rs | 4 +- datafusion/core/src/datasource/view.rs | 6 +-- datafusion/expr/src/table_source.rs | 4 +- .../src/analyzer/inline_table_scan.rs | 43 +++++++++---------- 7 files changed, 36 insertions(+), 35 deletions(-) diff --git a/datafusion/catalog/src/table.rs b/datafusion/catalog/src/table.rs index 792315642a00..69fa81faf8e2 100644 --- a/datafusion/catalog/src/table.rs +++ b/datafusion/catalog/src/table.rs @@ -16,6 +16,7 @@ // under the License. use std::any::Any; +use std::borrow::Cow; use std::sync::Arc; use crate::session::Session; @@ -56,8 +57,8 @@ pub trait TableProvider: Sync + Send { None } - /// Get the [`LogicalPlan`] of this table, if available - fn get_logical_plan(&self) -> Option<&LogicalPlan> { + /// Get the [`LogicalPlan`] of this table, if available. + fn get_logical_plan(&self) -> Option> { None } diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index a38e7f45a6f1..c516c7985d54 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -21,6 +21,7 @@ mod parquet; use std::any::Any; +use std::borrow::Cow; use std::collections::HashMap; use std::sync::Arc; @@ -1648,8 +1649,8 @@ impl TableProvider for DataFrameTableProvider { self } - fn get_logical_plan(&self) -> Option<&LogicalPlan> { - Some(&self.plan) + fn get_logical_plan(&self) -> Option> { + Some(Cow::Borrowed(&self.plan)) } fn supports_filters_pushdown( diff --git a/datafusion/core/src/datasource/cte_worktable.rs b/datafusion/core/src/datasource/cte_worktable.rs index d7d224828dda..d2da15c64f52 100644 --- a/datafusion/core/src/datasource/cte_worktable.rs +++ b/datafusion/core/src/datasource/cte_worktable.rs @@ -17,8 +17,8 @@ //! CteWorkTable implementation used for recursive queries -use std::any::Any; use std::sync::Arc; +use std::{any::Any, borrow::Cow}; use arrow::datatypes::SchemaRef; use async_trait::async_trait; @@ -63,7 +63,7 @@ impl TableProvider for CteWorkTable { self } - fn get_logical_plan(&self) -> Option<&LogicalPlan> { + fn get_logical_plan(&self) -> Option> { None } diff --git a/datafusion/core/src/datasource/default_table_source.rs b/datafusion/core/src/datasource/default_table_source.rs index 977e681d6641..b4a5a76fc9ff 100644 --- a/datafusion/core/src/datasource/default_table_source.rs +++ b/datafusion/core/src/datasource/default_table_source.rs @@ -17,8 +17,8 @@ //! Default TableSource implementation used in DataFusion physical plans -use std::any::Any; use std::sync::Arc; +use std::{any::Any, borrow::Cow}; use crate::datasource::TableProvider; @@ -70,7 +70,7 @@ impl TableSource for DefaultTableSource { self.table_provider.supports_filters_pushdown(filter) } - fn get_logical_plan(&self) -> Option<&datafusion_expr::LogicalPlan> { + fn get_logical_plan(&self) -> Option> { self.table_provider.get_logical_plan() } diff --git a/datafusion/core/src/datasource/view.rs b/datafusion/core/src/datasource/view.rs index a81942bf769e..947714c1e4f9 100644 --- a/datafusion/core/src/datasource/view.rs +++ b/datafusion/core/src/datasource/view.rs @@ -17,7 +17,7 @@ //! View data source which uses a LogicalPlan as it's input. -use std::{any::Any, sync::Arc}; +use std::{any::Any, borrow::Cow, sync::Arc}; use crate::{ error::Result, @@ -90,8 +90,8 @@ impl TableProvider for ViewTable { self } - fn get_logical_plan(&self) -> Option<&LogicalPlan> { - Some(&self.logical_plan) + fn get_logical_plan(&self) -> Option> { + Some(Cow::Borrowed(&self.logical_plan)) } fn schema(&self) -> SchemaRef { diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index 2de3cc923315..8b8d2dfcf2df 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -22,7 +22,7 @@ use crate::{Expr, LogicalPlan}; use arrow::datatypes::SchemaRef; use datafusion_common::{Constraints, Result}; -use std::any::Any; +use std::{any::Any, borrow::Cow}; /// Indicates how a filter expression is handled by /// [`TableProvider::scan`]. @@ -122,7 +122,7 @@ pub trait TableSource: Sync + Send { } /// Get the Logical plan of this table provider, if available. - fn get_logical_plan(&self) -> Option<&LogicalPlan> { + fn get_logical_plan(&self) -> Option> { None } diff --git a/datafusion/optimizer/src/analyzer/inline_table_scan.rs b/datafusion/optimizer/src/analyzer/inline_table_scan.rs index b69b8410da49..d5b3648725b9 100644 --- a/datafusion/optimizer/src/analyzer/inline_table_scan.rs +++ b/datafusion/optimizer/src/analyzer/inline_table_scan.rs @@ -24,7 +24,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Column, Result}; use datafusion_expr::expr::WildcardOptions; -use datafusion_expr::{logical_plan::LogicalPlan, Expr, LogicalPlanBuilder, TableScan}; +use datafusion_expr::{logical_plan::LogicalPlan, Expr, LogicalPlanBuilder}; /// Analyzed rule that inlines TableScan that provide a [`LogicalPlan`] /// (DataFrame / ViewTable) @@ -56,24 +56,23 @@ fn analyze_internal(plan: LogicalPlan) -> Result> { match plan { // Match only on scans without filter / projection / fetch // Views and DataFrames won't have those added - // during the early stage of planning - LogicalPlan::TableScan(TableScan { - table_name, - source, - projection, - filters, - .. - }) if filters.is_empty() && source.get_logical_plan().is_some() => { - let sub_plan = source.get_logical_plan().unwrap(); - let projection_exprs = generate_projection_expr(&projection, sub_plan)?; - LogicalPlanBuilder::from(sub_plan.clone()) - .project(projection_exprs)? - // Ensures that the reference to the inlined table remains the - // same, meaning we don't have to change any of the parent nodes - // that reference this table. - .alias(table_name)? - .build() - .map(Transformed::yes) + // during the early stage of planning. + LogicalPlan::TableScan(table_scan) if table_scan.filters.is_empty() => { + if let Some(sub_plan) = table_scan.source.get_logical_plan() { + let sub_plan = sub_plan.into_owned(); + let projection_exprs = + generate_projection_expr(&table_scan.projection, &sub_plan)?; + LogicalPlanBuilder::from(sub_plan) + .project(projection_exprs)? + // Ensures that the reference to the inlined table remains the + // same, meaning we don't have to change any of the parent nodes + // that reference this table. + .alias(table_scan.table_name)? + .build() + .map(Transformed::yes) + } else { + Ok(Transformed::no(LogicalPlan::TableScan(table_scan))) + } } _ => Ok(Transformed::no(plan)), } @@ -104,7 +103,7 @@ fn generate_projection_expr( #[cfg(test)] mod tests { - use std::{sync::Arc, vec}; + use std::{borrow::Cow, sync::Arc, vec}; use crate::analyzer::inline_table_scan::InlineTableScan; use crate::test::assert_analyzed_plan_eq; @@ -167,8 +166,8 @@ mod tests { Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)])) } - fn get_logical_plan(&self) -> Option<&LogicalPlan> { - Some(&self.plan) + fn get_logical_plan(&self) -> Option> { + Some(Cow::Borrowed(&self.plan)) } } From 1b875f456a1d916d2c6826531dc1b3f46c42706f Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Mon, 26 Aug 2024 21:11:41 +0200 Subject: [PATCH 02/75] Implement groups accumulator for stddev and variance (#12095) * Add more stddev/var tests cases Add tests cases for stddev_samp/pop and var_smap/pop the includes a group_by clause. * Implement GroupsAccumulator for stddev and variance * Add cast to support all numeric types * Improve documenataion and comments --- .../groups_accumulator/accumulate.rs | 268 +++++++++-------- datafusion/functions-aggregate/src/stddev.rs | 83 +++++- .../functions-aggregate/src/variance.rs | 275 ++++++++++++++++-- .../sqllogictest/test_files/aggregate.slt | 79 +++++ 4 files changed, 555 insertions(+), 150 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs index 455fc5fec450..a0475fe8e446 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -91,36 +91,9 @@ impl NullState { /// * `opt_filter`: if present, only rows for which is Some(true) are included /// * `value_fn`: function invoked for (group_index, value) where value is non null /// - /// # Example + /// See [`accumulate`], for more details on how value_fn is called /// - /// ```text - /// ┌─────────┐ ┌─────────┐ ┌ ─ ─ ─ ─ ┐ - /// │ ┌─────┐ │ │ ┌─────┐ │ ┌─────┐ - /// │ │ 2 │ │ │ │ 200 │ │ │ │ t │ │ - /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ - /// │ │ 2 │ │ │ │ 100 │ │ │ │ f │ │ - /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ - /// │ │ 0 │ │ │ │ 200 │ │ │ │ t │ │ - /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ - /// │ │ 1 │ │ │ │ 200 │ │ │ │NULL │ │ - /// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ - /// │ │ 0 │ │ │ │ 300 │ │ │ │ t │ │ - /// │ └─────┘ │ │ └─────┘ │ └─────┘ - /// └─────────┘ └─────────┘ └ ─ ─ ─ ─ ┘ - /// - /// group_indices values opt_filter - /// ``` - /// - /// In the example above, `value_fn` is invoked for each (group_index, - /// value) pair where `opt_filter[i]` is true and values is non null - /// - /// ```text - /// value_fn(2, 200) - /// value_fn(0, 200) - /// value_fn(0, 300) - /// ``` - /// - /// It also sets + /// When value_fn is called it also sets /// /// 1. `self.seen_values[group_index]` to true for all rows that had a non null vale pub fn accumulate( @@ -134,105 +107,14 @@ impl NullState { T: ArrowPrimitiveType + Send, F: FnMut(usize, T::Native) + Send, { - let data: &[T::Native] = values.values(); - assert_eq!(data.len(), group_indices.len()); - // ensure the seen_values is big enough (start everything at // "not seen" valid) let seen_values = initialize_builder(&mut self.seen_values, total_num_groups, false); - - match (values.null_count() > 0, opt_filter) { - // no nulls, no filter, - (false, None) => { - let iter = group_indices.iter().zip(data.iter()); - for (&group_index, &new_value) in iter { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value); - } - } - // nulls, no filter - (true, None) => { - let nulls = values.nulls().unwrap(); - // This is based on (ahem, COPY/PASTE) arrow::compute::aggregate::sum - // iterate over in chunks of 64 bits for more efficient null checking - let group_indices_chunks = group_indices.chunks_exact(64); - let data_chunks = data.chunks_exact(64); - let bit_chunks = nulls.inner().bit_chunks(); - - let group_indices_remainder = group_indices_chunks.remainder(); - let data_remainder = data_chunks.remainder(); - - group_indices_chunks - .zip(data_chunks) - .zip(bit_chunks.iter()) - .for_each(|((group_index_chunk, data_chunk), mask)| { - // index_mask has value 1 << i in the loop - let mut index_mask = 1; - group_index_chunk.iter().zip(data_chunk.iter()).for_each( - |(&group_index, &new_value)| { - // valid bit was set, real value - let is_valid = (mask & index_mask) != 0; - if is_valid { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value); - } - index_mask <<= 1; - }, - ) - }); - - // handle any remaining bits (after the initial 64) - let remainder_bits = bit_chunks.remainder_bits(); - group_indices_remainder - .iter() - .zip(data_remainder.iter()) - .enumerate() - .for_each(|(i, (&group_index, &new_value))| { - let is_valid = remainder_bits & (1 << i) != 0; - if is_valid { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value); - } - }); - } - // no nulls, but a filter - (false, Some(filter)) => { - assert_eq!(filter.len(), group_indices.len()); - // The performance with a filter could be improved by - // iterating over the filter in chunks, rather than a single - // iterator. TODO file a ticket - group_indices - .iter() - .zip(data.iter()) - .zip(filter.iter()) - .for_each(|((&group_index, &new_value), filter_value)| { - if let Some(true) = filter_value { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value); - } - }) - } - // both null values and filters - (true, Some(filter)) => { - assert_eq!(filter.len(), group_indices.len()); - // The performance with a filter could be improved by - // iterating over the filter in chunks, rather than using - // iterators. TODO file a ticket - filter - .iter() - .zip(group_indices.iter()) - .zip(values.iter()) - .for_each(|((filter_value, &group_index), new_value)| { - if let Some(true) = filter_value { - if let Some(new_value) = new_value { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value) - } - } - }) - } - } + accumulate(group_indices, values, opt_filter, |group_index, value| { + seen_values.set_bit(group_index, true); + value_fn(group_index, value); + }); } /// Invokes `value_fn(group_index, value)` for each non null, non @@ -351,6 +233,144 @@ impl NullState { } } +/// Invokes `value_fn(group_index, value)` for each non null, non +/// filtered value of `value`, +/// +/// # Arguments: +/// +/// * `group_indices`: To which groups do the rows in `values` belong, (aka group_index) +/// * `values`: the input arguments to the accumulator +/// * `opt_filter`: if present, only rows for which is Some(true) are included +/// * `value_fn`: function invoked for (group_index, value) where value is non null +/// +/// # Example +/// +/// ```text +/// ┌─────────┐ ┌─────────┐ ┌ ─ ─ ─ ─ ┐ +/// │ ┌─────┐ │ │ ┌─────┐ │ ┌─────┐ +/// │ │ 2 │ │ │ │ 200 │ │ │ │ t │ │ +/// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ +/// │ │ 2 │ │ │ │ 100 │ │ │ │ f │ │ +/// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ +/// │ │ 0 │ │ │ │ 200 │ │ │ │ t │ │ +/// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ +/// │ │ 1 │ │ │ │ 200 │ │ │ │NULL │ │ +/// │ ├─────┤ │ │ ├─────┤ │ ├─────┤ +/// │ │ 0 │ │ │ │ 300 │ │ │ │ t │ │ +/// │ └─────┘ │ │ └─────┘ │ └─────┘ +/// └─────────┘ └─────────┘ └ ─ ─ ─ ─ ┘ +/// +/// group_indices values opt_filter +/// ``` +/// +/// In the example above, `value_fn` is invoked for each (group_index, +/// value) pair where `opt_filter[i]` is true and values is non null +/// +/// ```text +/// value_fn(2, 200) +/// value_fn(0, 200) +/// value_fn(0, 300) +/// ``` +pub fn accumulate( + group_indices: &[usize], + values: &PrimitiveArray, + opt_filter: Option<&BooleanArray>, + mut value_fn: F, +) where + T: ArrowPrimitiveType + Send, + F: FnMut(usize, T::Native) + Send, +{ + let data: &[T::Native] = values.values(); + assert_eq!(data.len(), group_indices.len()); + + match (values.null_count() > 0, opt_filter) { + // no nulls, no filter, + (false, None) => { + let iter = group_indices.iter().zip(data.iter()); + for (&group_index, &new_value) in iter { + value_fn(group_index, new_value); + } + } + // nulls, no filter + (true, None) => { + let nulls = values.nulls().unwrap(); + // This is based on (ahem, COPY/PASTE) arrow::compute::aggregate::sum + // iterate over in chunks of 64 bits for more efficient null checking + let group_indices_chunks = group_indices.chunks_exact(64); + let data_chunks = data.chunks_exact(64); + let bit_chunks = nulls.inner().bit_chunks(); + + let group_indices_remainder = group_indices_chunks.remainder(); + let data_remainder = data_chunks.remainder(); + + group_indices_chunks + .zip(data_chunks) + .zip(bit_chunks.iter()) + .for_each(|((group_index_chunk, data_chunk), mask)| { + // index_mask has value 1 << i in the loop + let mut index_mask = 1; + group_index_chunk.iter().zip(data_chunk.iter()).for_each( + |(&group_index, &new_value)| { + // valid bit was set, real value + let is_valid = (mask & index_mask) != 0; + if is_valid { + value_fn(group_index, new_value); + } + index_mask <<= 1; + }, + ) + }); + + // handle any remaining bits (after the initial 64) + let remainder_bits = bit_chunks.remainder_bits(); + group_indices_remainder + .iter() + .zip(data_remainder.iter()) + .enumerate() + .for_each(|(i, (&group_index, &new_value))| { + let is_valid = remainder_bits & (1 << i) != 0; + if is_valid { + value_fn(group_index, new_value); + } + }); + } + // no nulls, but a filter + (false, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than a single + // iterator. TODO file a ticket + group_indices + .iter() + .zip(data.iter()) + .zip(filter.iter()) + .for_each(|((&group_index, &new_value), filter_value)| { + if let Some(true) = filter_value { + value_fn(group_index, new_value); + } + }) + } + // both null values and filters + (true, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than using + // iterators. TODO file a ticket + filter + .iter() + .zip(group_indices.iter()) + .zip(values.iter()) + .for_each(|((filter_value, &group_index), new_value)| { + if let Some(true) = filter_value { + if let Some(new_value) = new_value { + value_fn(group_index, new_value) + } + } + }) + } + } +} + /// This function is called to update the accumulator state per row /// when the value is not needed (e.g. COUNT) /// diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index 180f4ad3cf37..3534fb5b4d26 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -19,17 +19,21 @@ use std::any::Any; use std::fmt::{Debug, Formatter}; +use std::sync::Arc; +use arrow::array::Float64Array; use arrow::{array::ArrayRef, datatypes::DataType, datatypes::Field}; use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_common::{plan_err, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::format_state_name; -use datafusion_expr::{Accumulator, AggregateUDFImpl, Signature, Volatility}; +use datafusion_expr::{ + Accumulator, AggregateUDFImpl, GroupsAccumulator, Signature, Volatility, +}; use datafusion_functions_aggregate_common::stats::StatsType; -use crate::variance::VarianceAccumulator; +use crate::variance::{VarianceAccumulator, VarianceGroupsAccumulator}; make_udaf_expr_and_func!( Stddev, @@ -118,6 +122,17 @@ impl AggregateUDFImpl for Stddev { fn aliases(&self) -> &[String] { &self.alias } + + fn groups_accumulator_supported(&self, acc_args: AccumulatorArgs) -> bool { + !acc_args.is_distinct + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + Ok(Box::new(StddevGroupsAccumulator::new(StatsType::Sample))) + } } make_udaf_expr_and_func!( @@ -201,6 +216,19 @@ impl AggregateUDFImpl for StddevPop { Ok(DataType::Float64) } + + fn groups_accumulator_supported(&self, acc_args: AccumulatorArgs) -> bool { + !acc_args.is_distinct + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + Ok(Box::new(StddevGroupsAccumulator::new( + StatsType::Population, + ))) + } } /// An accumulator to compute the average @@ -267,6 +295,57 @@ impl Accumulator for StddevAccumulator { } } +#[derive(Debug)] +pub struct StddevGroupsAccumulator { + variance: VarianceGroupsAccumulator, +} + +impl StddevGroupsAccumulator { + pub fn new(s_type: StatsType) -> Self { + Self { + variance: VarianceGroupsAccumulator::new(s_type), + } + } +} + +impl GroupsAccumulator for StddevGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow::array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + self.variance + .update_batch(values, group_indices, opt_filter, total_num_groups) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow::array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + self.variance + .merge_batch(values, group_indices, opt_filter, total_num_groups) + } + + fn evaluate(&mut self, emit_to: datafusion_expr::EmitTo) -> Result { + let (mut variances, nulls) = self.variance.variance(emit_to); + variances.iter_mut().for_each(|v| *v = v.sqrt()); + Ok(Arc::new(Float64Array::new(variances.into(), Some(nulls)))) + } + + fn state(&mut self, emit_to: datafusion_expr::EmitTo) -> Result> { + self.variance.state(emit_to) + } + + fn size(&self) -> usize { + self.variance.size() + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/functions-aggregate/src/variance.rs b/datafusion/functions-aggregate/src/variance.rs index 4c78a42ea494..f5f2d06e3837 100644 --- a/datafusion/functions-aggregate/src/variance.rs +++ b/datafusion/functions-aggregate/src/variance.rs @@ -18,10 +18,11 @@ //! [`VarianceSample`]: variance sample aggregations. //! [`VariancePopulation`]: variance population aggregations. -use std::fmt::Debug; +use std::{fmt::Debug, sync::Arc}; use arrow::{ - array::{ArrayRef, Float64Array, UInt64Array}, + array::{Array, ArrayRef, BooleanArray, Float64Array, UInt64Array}, + buffer::NullBuffer, compute::kernels::cast, datatypes::{DataType, Field}, }; @@ -32,9 +33,11 @@ use datafusion_common::{ use datafusion_expr::{ function::{AccumulatorArgs, StateFieldsArgs}, utils::format_state_name, - Accumulator, AggregateUDFImpl, Signature, Volatility, + Accumulator, AggregateUDFImpl, GroupsAccumulator, Signature, Volatility, +}; +use datafusion_functions_aggregate_common::{ + aggregate::groups_accumulator::accumulate::accumulate, stats::StatsType, }; -use datafusion_functions_aggregate_common::stats::StatsType; make_udaf_expr_and_func!( VarianceSample, @@ -122,6 +125,17 @@ impl AggregateUDFImpl for VarianceSample { fn aliases(&self) -> &[String] { &self.aliases } + + fn groups_accumulator_supported(&self, acc_args: AccumulatorArgs) -> bool { + !acc_args.is_distinct + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + Ok(Box::new(VarianceGroupsAccumulator::new(StatsType::Sample))) + } } pub struct VariancePopulation { @@ -196,6 +210,19 @@ impl AggregateUDFImpl for VariancePopulation { fn aliases(&self) -> &[String] { &self.aliases } + + fn groups_accumulator_supported(&self, acc_args: AccumulatorArgs) -> bool { + !acc_args.is_distinct + } + + fn create_groups_accumulator( + &self, + _args: AccumulatorArgs, + ) -> Result> { + Ok(Box::new(VarianceGroupsAccumulator::new( + StatsType::Population, + ))) + } } /// An accumulator to compute variance @@ -239,6 +266,36 @@ impl VarianceAccumulator { } } +#[inline] +fn merge( + count: u64, + mean: f64, + m2: f64, + count2: u64, + mean2: f64, + m22: f64, +) -> (u64, f64, f64) { + let new_count = count + count2; + let new_mean = + mean * count as f64 / new_count as f64 + mean2 * count2 as f64 / new_count as f64; + let delta = mean - mean2; + let new_m2 = + m2 + m22 + delta * delta * count as f64 * count2 as f64 / new_count as f64; + + (new_count, new_mean, new_m2) +} + +#[inline] +fn update(count: u64, mean: f64, m2: f64, value: f64) -> (u64, f64, f64) { + let new_count = count + 1; + let delta1 = value - mean; + let new_mean = delta1 / new_count as f64 + mean; + let delta2 = value - new_mean; + let new_m2 = m2 + delta1 * delta2; + + (new_count, new_mean, new_m2) +} + impl Accumulator for VarianceAccumulator { fn state(&mut self) -> Result> { Ok(vec![ @@ -253,15 +310,8 @@ impl Accumulator for VarianceAccumulator { let arr = downcast_value!(values, Float64Array).iter().flatten(); for value in arr { - let new_count = self.count + 1; - let delta1 = value - self.mean; - let new_mean = delta1 / new_count as f64 + self.mean; - let delta2 = value - new_mean; - let new_m2 = self.m2 + delta1 * delta2; - - self.count += 1; - self.mean = new_mean; - self.m2 = new_m2; + (self.count, self.mean, self.m2) = + update(self.count, self.mean, self.m2, value) } Ok(()) @@ -296,17 +346,14 @@ impl Accumulator for VarianceAccumulator { if c == 0_u64 { continue; } - let new_count = self.count + c; - let new_mean = self.mean * self.count as f64 / new_count as f64 - + means.value(i) * c as f64 / new_count as f64; - let delta = self.mean - means.value(i); - let new_m2 = self.m2 - + m2s.value(i) - + delta * delta * self.count as f64 * c as f64 / new_count as f64; - - self.count = new_count; - self.mean = new_mean; - self.m2 = new_m2; + (self.count, self.mean, self.m2) = merge( + self.count, + self.mean, + self.m2, + c, + means.value(i), + m2s.value(i), + ) } Ok(()) } @@ -344,3 +391,183 @@ impl Accumulator for VarianceAccumulator { true } } + +#[derive(Debug)] +pub struct VarianceGroupsAccumulator { + m2s: Vec, + means: Vec, + counts: Vec, + stats_type: StatsType, +} + +impl VarianceGroupsAccumulator { + pub fn new(s_type: StatsType) -> Self { + Self { + m2s: Vec::new(), + means: Vec::new(), + counts: Vec::new(), + stats_type: s_type, + } + } + + fn resize(&mut self, total_num_groups: usize) { + self.m2s.resize(total_num_groups, 0.0); + self.means.resize(total_num_groups, 0.0); + self.counts.resize(total_num_groups, 0); + } + + fn merge( + group_indices: &[usize], + counts: &UInt64Array, + means: &Float64Array, + m2s: &Float64Array, + opt_filter: Option<&BooleanArray>, + mut value_fn: F, + ) where + F: FnMut(usize, u64, f64, f64) + Send, + { + assert_eq!(counts.null_count(), 0); + assert_eq!(means.null_count(), 0); + assert_eq!(m2s.null_count(), 0); + + match opt_filter { + None => { + group_indices + .iter() + .zip(counts.values().iter()) + .zip(means.values().iter()) + .zip(m2s.values().iter()) + .for_each(|(((&group_index, &count), &mean), &m2)| { + value_fn(group_index, count, mean, m2); + }); + } + Some(filter) => { + group_indices + .iter() + .zip(counts.values().iter()) + .zip(means.values().iter()) + .zip(m2s.values().iter()) + .zip(filter.iter()) + .for_each( + |((((&group_index, &count), &mean), &m2), filter_value)| { + if let Some(true) = filter_value { + value_fn(group_index, count, mean, m2); + } + }, + ); + } + } + } + + pub fn variance( + &mut self, + emit_to: datafusion_expr::EmitTo, + ) -> (Vec, NullBuffer) { + let mut counts = emit_to.take_needed(&mut self.counts); + // means are only needed for updating m2s and are not needed for the final result. + // But we still need to take them to ensure the internal state is consistent. + let _ = emit_to.take_needed(&mut self.means); + let m2s = emit_to.take_needed(&mut self.m2s); + + if let StatsType::Sample = self.stats_type { + counts.iter_mut().for_each(|count| { + *count -= 1; + }); + } + let nulls = NullBuffer::from_iter(counts.iter().map(|&count| count != 0)); + let variance = m2s + .iter() + .zip(counts) + .map(|(m2, count)| m2 / count as f64) + .collect(); + (variance, nulls) + } +} + +impl GroupsAccumulator for VarianceGroupsAccumulator { + fn update_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow::array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 1, "single argument to update_batch"); + let values = &cast(&values[0], &DataType::Float64)?; + let values = downcast_value!(values, Float64Array); + + self.resize(total_num_groups); + accumulate(group_indices, values, opt_filter, |group_index, value| { + let (new_count, new_mean, new_m2) = update( + self.counts[group_index], + self.means[group_index], + self.m2s[group_index], + value, + ); + self.counts[group_index] = new_count; + self.means[group_index] = new_mean; + self.m2s[group_index] = new_m2; + }); + Ok(()) + } + + fn merge_batch( + &mut self, + values: &[ArrayRef], + group_indices: &[usize], + opt_filter: Option<&arrow::array::BooleanArray>, + total_num_groups: usize, + ) -> Result<()> { + assert_eq!(values.len(), 3, "two arguments to merge_batch"); + // first batch is counts, second is partial means, third is partial m2s + let partial_counts = downcast_value!(values[0], UInt64Array); + let partial_means = downcast_value!(values[1], Float64Array); + let partial_m2s = downcast_value!(values[2], Float64Array); + + self.resize(total_num_groups); + Self::merge( + group_indices, + partial_counts, + partial_means, + partial_m2s, + opt_filter, + |group_index, partial_count, partial_mean, partial_m2| { + let (new_count, new_mean, new_m2) = merge( + self.counts[group_index], + self.means[group_index], + self.m2s[group_index], + partial_count, + partial_mean, + partial_m2, + ); + self.counts[group_index] = new_count; + self.means[group_index] = new_mean; + self.m2s[group_index] = new_m2; + }, + ); + Ok(()) + } + + fn evaluate(&mut self, emit_to: datafusion_expr::EmitTo) -> Result { + let (variances, nulls) = self.variance(emit_to); + Ok(Arc::new(Float64Array::new(variances.into(), Some(nulls)))) + } + + fn state(&mut self, emit_to: datafusion_expr::EmitTo) -> Result> { + let counts = emit_to.take_needed(&mut self.counts); + let means = emit_to.take_needed(&mut self.means); + let m2s = emit_to.take_needed(&mut self.m2s); + + Ok(vec![ + Arc::new(UInt64Array::new(counts.into(), None)), + Arc::new(Float64Array::new(means.into(), None)), + Arc::new(Float64Array::new(m2s.into(), None)), + ]) + } + + fn size(&self) -> usize { + self.m2s.capacity() * std::mem::size_of::() + + self.means.capacity() * std::mem::size_of::() + + self.counts.capacity() * std::mem::size_of::() + } +} diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index b826dd31f4d1..45cb4d4615d7 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -511,6 +511,85 @@ select stddev(sq.column1) from (values (1.1), (2.0), (3.0)) as sq ---- 0.950438495292 +# csv_query_stddev_7 +query IR +SELECT c2, stddev_samp(c12) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2 +---- +1 0.303641032262 +2 0.284581967411 +3 0.296002660506 +4 0.284324609109 +5 0.331034486752 + +# csv_query_stddev_8 +query IR +SELECT c2, stddev_pop(c12) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2 +---- +1 0.296659845456 +2 0.278038978602 +3 0.288107833475 +4 0.278074953424 +5 0.318992813225 + +# csv_query_stddev_9 +query IR +SELECT c2, var_pop(c12) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2 +---- +1 0.088007063906 +2 0.077305673622 +3 0.083006123709 +4 0.077325679722 +5 0.101756414889 + +# csv_query_stddev_10 +query IR +SELECT c2, var_samp(c12) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2 +---- +1 0.092197876473 +2 0.080986896176 +3 0.087617575027 +4 0.080840483345 +5 0.109583831419 + +# csv_query_stddev_11 +query IR +SELECT c2, var_samp(c12) FROM aggregate_test_100 WHERE c12 > 0.90 GROUP BY c2 ORDER BY c2 +---- +1 0.000889240174 +2 0.000785878272 +3 NULL +4 NULL +5 0.000269544643 + +# Use PostgresSQL dialect +statement ok +set datafusion.sql_parser.dialect = 'Postgres'; + +# csv_query_stddev_12 +query IR +SELECT c2, var_samp(c12) FILTER (WHERE c12 > 0.90) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2 +---- +1 0.000889240174 +2 0.000785878272 +3 NULL +4 NULL +5 0.000269544643 + +# Restore the default dialect +statement ok +set datafusion.sql_parser.dialect = 'Generic'; + +# csv_query_stddev_13 +query IR +SELECT c2, var_samp(CASE WHEN c12 > 0.90 THEN c12 ELSE null END) FROM aggregate_test_100 GROUP BY c2 ORDER BY c2 +---- +1 0.000889240174 +2 0.000785878272 +3 NULL +4 NULL +5 0.000269544643 + + # csv_query_approx_median_1 query I SELECT approx_median(c2) FROM aggregate_test_100 From 7d49fb308d638bef2c1ed1fa96fc87e3f740c72e Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 26 Aug 2024 15:12:23 -0400 Subject: [PATCH 03/75] Minor: refine Partitioning documentation (#12145) --- datafusion/expr/src/logical_plan/plan.rs | 6 +++++- datafusion/physical-expr/src/partitioning.rs | 6 ++++-- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 3ede7f25b753..d6574040c596 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -2932,7 +2932,11 @@ impl Debug for Subquery { } } -/// Logical partitioning schemes supported by the repartition operator. +/// Logical partitioning schemes supported by [`LogicalPlan::Repartition`] +/// +/// 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)] pub enum Partitioning { /// Allocate batches using a round-robin algorithm and the specified number of partitions diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 6472dd47489c..45beeb7b81af 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -24,8 +24,8 @@ use crate::{physical_exprs_equal, EquivalenceProperties, PhysicalExpr}; /// Output partitioning supported by [`ExecutionPlan`]s. /// -/// When `executed`, `ExecutionPlan`s produce one or more independent stream of -/// data batches in parallel, referred to as partitions. The streams are Rust +/// Calling [`ExecutionPlan::execute`] produce one or more independent streams of +/// [`RecordBatch`]es in parallel, referred to as partitions. The streams are Rust /// `async` [`Stream`]s (a special kind of future). The number of output /// partitions varies based on the input and the operation performed. /// @@ -102,6 +102,8 @@ use crate::{physical_exprs_equal, EquivalenceProperties, PhysicalExpr}; /// Plans such as `FilterExec` produce the same number of output streams /// (partitions) as input streams (partitions). /// +/// [`RecordBatch`]: arrow::record_batch::RecordBatch +/// [`ExecutionPlan::execute`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#tymethod.execute /// [`ExecutionPlan`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html /// [`Stream`]: https://docs.rs/futures/latest/futures/stream/trait.Stream.html #[derive(Debug, Clone)] From ed12f113cb1d33a1eea80c41787a93bc52f196c7 Mon Sep 17 00:00:00 2001 From: theirix Date: Mon, 26 Aug 2024 20:13:34 +0100 Subject: [PATCH 04/75] Minor: allow to build RuntimeEnv from RuntimeConfig (#12151) * Allow to build RuntimeEnv from RuntimeConfig * Fix formatting --- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 9 +++++---- datafusion/execution/src/runtime_env.rs | 5 +++++ 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index f4b4f16aa160..eda306dd3de5 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -22,7 +22,7 @@ use arrow::{ compute::SortOptions, record_batch::RecordBatch, }; -use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; +use datafusion::execution::runtime_env::RuntimeConfig; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::sorts::sort::SortExec; @@ -136,9 +136,10 @@ impl SortTest { .sort_spill_reservation_bytes, ); - let runtime_config = RuntimeConfig::new() - .with_memory_pool(Arc::new(GreedyMemoryPool::new(pool_size))); - let runtime = Arc::new(RuntimeEnv::new(runtime_config).unwrap()); + let runtime_env = RuntimeConfig::new() + .with_memory_pool(Arc::new(GreedyMemoryPool::new(pool_size))) + .build(); + let runtime = Arc::new(runtime_env.unwrap()); SessionContext::new_with_config_rt(session_config, runtime) } else { SessionContext::new_with_config(session_config) diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index 420246595558..bddce81f537b 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -228,4 +228,9 @@ impl RuntimeConfig { pub fn with_temp_file_path(self, path: impl Into) -> Self { self.with_disk_manager(DiskManagerConfig::new_specified(vec![path.into()])) } + + /// Build a `RuntimeEnv` object from the configuration + pub fn build(self) -> Result { + RuntimeEnv::new(self) + } } From 55a1459b123c0bb16e2cbe41cc59cf3582115932 Mon Sep 17 00:00:00 2001 From: Trent Hauck Date: Mon, 26 Aug 2024 12:19:27 -0700 Subject: [PATCH 05/75] benches: add lower benches (#12152) --- datafusion/functions/benches/lower.rs | 92 ++++++++++++++++++++++++++- 1 file changed, 90 insertions(+), 2 deletions(-) diff --git a/datafusion/functions/benches/lower.rs b/datafusion/functions/benches/lower.rs index fa963f174e46..934c1c6bd189 100644 --- a/datafusion/functions/benches/lower.rs +++ b/datafusion/functions/benches/lower.rs @@ -17,8 +17,10 @@ extern crate criterion; -use arrow::array::{ArrayRef, StringArray}; -use arrow::util::bench_util::create_string_array_with_len; +use arrow::array::{ArrayRef, StringArray, StringViewBuilder}; +use arrow::util::bench_util::{ + create_string_array_with_len, create_string_view_array_with_len, +}; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_expr::ColumnarValue; use datafusion_functions::string; @@ -65,6 +67,58 @@ fn create_args3(size: usize) -> Vec { vec![ColumnarValue::Array(array)] } +/// Create an array of args containing StringViews, where all the values in the +/// StringViews are ASCII. +/// * `size` - the length of the StringViews, and +/// * `str_len` - the length of the strings within the array. +/// * `null_density` - the density of null values in the array. +/// * `mixed` - whether the array is mixed between inlined and referenced strings. +fn create_args4( + size: usize, + str_len: usize, + null_density: f32, + mixed: bool, +) -> Vec { + let array = Arc::new(create_string_view_array_with_len( + size, + null_density, + str_len, + mixed, + )); + + vec![ColumnarValue::Array(array)] +} + +/// Create an array of args containing a StringViewArray, where some of the values in the +/// array are non-ASCII. +/// * `size` - the length of the StringArray, and +/// * `non_ascii_density` - the density of non-ASCII values in the array. +/// * `null_density` - the density of null values in the array. +fn create_args5( + size: usize, + non_ascii_density: f32, + null_density: f32, +) -> Vec { + let mut string_view_builder = StringViewBuilder::with_capacity(size); + for _ in 0..size { + // sample null_density to determine if the value should be null + if rand::random::() < null_density { + string_view_builder.append_null(); + continue; + } + + // sample non_ascii_density to determine if the value should be non-ASCII + if rand::random::() < non_ascii_density { + string_view_builder.append_value("农历新年农历新年农历新年农历新年农历新年"); + } else { + string_view_builder.append_value("DATAFUSIONDATAFUSIONDATAFUSION"); + } + } + + let array = Arc::new(string_view_builder.finish()) as ArrayRef; + vec![ColumnarValue::Array(array)] +} + fn criterion_benchmark(c: &mut Criterion) { let lower = string::lower(); for size in [1024, 4096, 8192] { @@ -85,6 +139,40 @@ fn criterion_benchmark(c: &mut Criterion) { |b| b.iter(|| black_box(lower.invoke(&args))), ); } + + let sizes = [4096, 8192]; + let str_lens = [10, 64, 128]; + let mixes = [true, false]; + let null_densities = [0.0f32, 0.1f32]; + + for null_density in &null_densities { + for &mixed in &mixes { + for &str_len in &str_lens { + for &size in &sizes { + let args = create_args4(size, str_len, *null_density, mixed); + c.bench_function( + &format!("lower_all_values_are_ascii_string_views: size: {}, str_len: {}, null_density: {}, mixed: {}", + size, str_len, null_density, mixed), + |b| b.iter(|| black_box(lower.invoke(&args))), + ); + + let args = create_args4(size, str_len, *null_density, mixed); + c.bench_function( + &format!("lower_all_values_are_ascii_string_views: size: {}, str_len: {}, null_density: {}, mixed: {}", + size, str_len, null_density, mixed), + |b| b.iter(|| black_box(lower.invoke(&args))), + ); + + let args = create_args5(size, 0.1, *null_density); + c.bench_function( + &format!("lower_some_values_are_nonascii_string_views: size: {}, str_len: {}, non_ascii_density: {}, null_density: {}, mixed: {}", + size, str_len, 0.1, null_density, mixed), + |b| b.iter(|| black_box(lower.invoke(&args))), + ); + } + } + } + } } criterion_group!(benches, criterion_benchmark); From 0f96af5b500efff72314f840a59a736787cc3def Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 26 Aug 2024 21:24:51 +0200 Subject: [PATCH 06/75] Replace Arc::try_unwrap with Arc::unwrap_or_clone where cloning anyway (#12173) --- datafusion/core/src/execution/context/mod.rs | 2 +- datafusion/expr/src/expr_fn.rs | 8 ++++---- datafusion/expr/src/logical_plan/plan.rs | 5 +---- datafusion/sql/src/unparser/rewrite.rs | 5 +---- 4 files changed, 7 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 49cbf9ebde9d..c2a072eef756 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -688,7 +688,7 @@ impl SessionContext { column_defaults, } = cmd; - let input = Arc::try_unwrap(input).unwrap_or_else(|e| e.as_ref().clone()); + let input = Arc::unwrap_or_clone(input); let input = self.state().optimize(&input)?; let table = self.table(name.clone()).await; match (if_not_exists, or_replace, table) { diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 4e6022399653..1e0b601146dd 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -394,7 +394,7 @@ pub fn create_udf( volatility: Volatility, fun: ScalarFunctionImplementation, ) -> ScalarUDF { - let return_type = Arc::try_unwrap(return_type).unwrap_or_else(|t| t.as_ref().clone()); + let return_type = Arc::unwrap_or_clone(return_type); ScalarUDF::from(SimpleScalarUDF::new( name, input_types, @@ -476,8 +476,8 @@ pub fn create_udaf( accumulator: AccumulatorFactoryFunction, state_type: Arc>, ) -> AggregateUDF { - let return_type = Arc::try_unwrap(return_type).unwrap_or_else(|t| t.as_ref().clone()); - let state_type = Arc::try_unwrap(state_type).unwrap_or_else(|t| t.as_ref().clone()); + let return_type = Arc::unwrap_or_clone(return_type); + let state_type = Arc::unwrap_or_clone(state_type); let state_fields = state_type .into_iter() .enumerate() @@ -594,7 +594,7 @@ pub fn create_udwf( volatility: Volatility, partition_evaluator_factory: PartitionEvaluatorFactory, ) -> WindowUDF { - let return_type = Arc::try_unwrap(return_type).unwrap_or_else(|t| t.as_ref().clone()); + let return_type = Arc::unwrap_or_clone(return_type); WindowUDF::from(SimpleWindowUDF::new( name, input_type, diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index d6574040c596..096b65896b69 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1160,10 +1160,7 @@ impl LogicalPlan { Ok(if let LogicalPlan::Prepare(prepare_lp) = plan_with_values { param_values.verify(&prepare_lp.data_types)?; // try and take ownership of the input if is not shared, clone otherwise - match Arc::try_unwrap(prepare_lp.input) { - Ok(input) => input, - Err(arc_input) => arc_input.as_ref().clone(), - } + Arc::unwrap_or_clone(prepare_lp.input) } else { plan_with_values }) diff --git a/datafusion/sql/src/unparser/rewrite.rs b/datafusion/sql/src/unparser/rewrite.rs index f6725485f920..fed3f2c39f9d 100644 --- a/datafusion/sql/src/unparser/rewrite.rs +++ b/datafusion/sql/src/unparser/rewrite.rs @@ -59,10 +59,7 @@ pub(super) fn normalize_union_schema(plan: &LogicalPlan) -> Result let transformed_plan = plan.transform_up(|plan| match plan { LogicalPlan::Union(mut union) => { - let schema = match Arc::try_unwrap(union.schema) { - Ok(inner) => inner, - Err(schema) => (*schema).clone(), - }; + let schema = Arc::unwrap_or_clone(union.schema); let schema = schema.strip_qualifiers(); union.schema = Arc::new(schema); From dff590bfd2bb9993b2c8ce6f76a3bdd973e520a8 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 26 Aug 2024 13:57:17 -0700 Subject: [PATCH 07/75] fix: Skip buffered rows which are not joined with streamed side when checking join filter results (#12159) --- datafusion/physical-plan/src/joins/sort_merge_join.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 56cd699bf6e7..511cb4c55fcd 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -1474,6 +1474,12 @@ impl SMJStream { [chunk.buffered_batch_idx.unwrap()]; for i in 0..pre_mask.len() { + // If the buffered row is not joined with streamed side, + // skip it. + if buffered_indices.is_null(i) { + continue; + } + let buffered_index = buffered_indices.value(i); buffered_batch.join_filter_failed_map.insert( From 1e67cb6239dbaa896344335678356c24ef1df509 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Tue, 27 Aug 2024 08:22:42 +0800 Subject: [PATCH 08/75] enable the null map tests (#12176) --- datafusion/sqllogictest/test_files/map.slt | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index b7a0a74913b0..270e4beccc52 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -376,11 +376,10 @@ SELECT MAP {'a':1, 'b':2, 'c':3 }['a'] FROM t; 1 1 -# TODO(https://github.com/sqlparser-rs/sqlparser-rs/pull/1361): support parsing an empty map. Enable this after upgrading sqlparser-rs. -# query ? -# SELECT MAP {}; -# ---- -# {} +query ? +SELECT MAP {}; +---- +{} # values contain null query ? From a723b79eca998dfa61cd2aefa122779336b153c7 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 03:34:30 +0200 Subject: [PATCH 09/75] Remove unwrap_arc helper (#12172) It can now be replaced with single call `Arc::unwrap_or_clone`, with added bonus of slightly better name. --- datafusion/expr/src/logical_plan/builder.rs | 37 +++++++++++-------- datafusion/expr/src/logical_plan/plan.rs | 3 +- datafusion/expr/src/logical_plan/tree_node.rs | 11 +----- .../optimizer/src/analyzer/type_coercion.rs | 18 ++++++--- .../optimizer/src/common_subexpr_eliminate.rs | 11 +++--- .../src/decorrelate_predicate_subquery.rs | 9 +++-- .../optimizer/src/eliminate_cross_join.rs | 27 +++++++++++--- datafusion/optimizer/src/eliminate_filter.rs | 3 +- datafusion/optimizer/src/eliminate_limit.rs | 6 ++- .../optimizer/src/eliminate_nested_union.rs | 14 +++---- .../optimizer/src/eliminate_one_union.rs | 10 +++-- .../optimizer/src/eliminate_outer_join.rs | 3 +- .../optimizer/src/optimize_projections/mod.rs | 14 +++---- .../optimizer/src/propagate_empty_relation.rs | 3 +- datafusion/optimizer/src/push_down_filter.rs | 23 ++++++------ datafusion/optimizer/src/push_down_limit.rs | 3 +- datafusion/sql/src/select.rs | 10 ++--- 17 files changed, 110 insertions(+), 95 deletions(-) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index fdd07da023e0..6abcfe4279bc 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -41,9 +41,8 @@ use crate::utils::{ find_valid_equijoin_key_pair, group_window_expr_by_sort_keys, }; use crate::{ - and, binary_expr, logical_plan::tree_node::unwrap_arc, DmlStatement, Expr, - ExprSchemable, Operator, RecursiveQuery, TableProviderFilterPushDown, TableSource, - WriteOp, + and, binary_expr, DmlStatement, Expr, ExprSchemable, Operator, RecursiveQuery, + TableProviderFilterPushDown, TableSource, WriteOp, }; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; @@ -376,7 +375,7 @@ impl LogicalPlanBuilder { self, expr: impl IntoIterator>, ) -> Result { - project(unwrap_arc(self.plan), expr).map(Self::new) + project(Arc::unwrap_or_clone(self.plan), expr).map(Self::new) } /// Select the given column indices @@ -429,7 +428,7 @@ impl LogicalPlanBuilder { /// Apply an alias pub fn alias(self, alias: impl Into) -> Result { - subquery_alias(unwrap_arc(self.plan), alias).map(Self::new) + subquery_alias(Arc::unwrap_or_clone(self.plan), alias).map(Self::new) } /// Add missing sort columns to all downstream projection @@ -484,7 +483,7 @@ impl LogicalPlanBuilder { Self::ambiguous_distinct_check(&missing_exprs, missing_cols, &expr)?; } expr.extend(missing_exprs); - project(unwrap_arc(input), expr) + project(Arc::unwrap_or_clone(input), expr) } _ => { let is_distinct = @@ -580,8 +579,11 @@ impl LogicalPlanBuilder { let new_expr = schema.columns().into_iter().map(Expr::Column).collect(); let is_distinct = false; - let plan = - Self::add_missing_columns(unwrap_arc(self.plan), &missing_cols, is_distinct)?; + let plan = Self::add_missing_columns( + Arc::unwrap_or_clone(self.plan), + &missing_cols, + is_distinct, + )?; let sort_plan = LogicalPlan::Sort(Sort { expr: normalize_cols(exprs, &plan)?, input: Arc::new(plan), @@ -595,12 +597,12 @@ impl LogicalPlanBuilder { /// Apply a union, preserving duplicate rows pub fn union(self, plan: LogicalPlan) -> Result { - union(unwrap_arc(self.plan), plan).map(Self::new) + union(Arc::unwrap_or_clone(self.plan), plan).map(Self::new) } /// Apply a union, removing duplicate rows pub fn union_distinct(self, plan: LogicalPlan) -> Result { - let left_plan: LogicalPlan = unwrap_arc(self.plan); + let left_plan: LogicalPlan = Arc::unwrap_or_clone(self.plan); let right_plan: LogicalPlan = plan; Ok(Self::new(LogicalPlan::Distinct(Distinct::All(Arc::new( @@ -1064,7 +1066,7 @@ impl LogicalPlanBuilder { /// Build the plan pub fn build(self) -> Result { - Ok(unwrap_arc(self.plan)) + Ok(Arc::unwrap_or_clone(self.plan)) } /// Apply a join with the expression on constraint. @@ -1138,7 +1140,7 @@ impl LogicalPlanBuilder { /// Unnest the given column. pub fn unnest_column(self, column: impl Into) -> Result { - unnest(unwrap_arc(self.plan), vec![column.into()]).map(Self::new) + unnest(Arc::unwrap_or_clone(self.plan), vec![column.into()]).map(Self::new) } /// Unnest the given column given [`UnnestOptions`] @@ -1147,8 +1149,12 @@ impl LogicalPlanBuilder { column: impl Into, options: UnnestOptions, ) -> Result { - unnest_with_options(unwrap_arc(self.plan), vec![column.into()], options) - .map(Self::new) + unnest_with_options( + Arc::unwrap_or_clone(self.plan), + vec![column.into()], + options, + ) + .map(Self::new) } /// Unnest the given columns with the given [`UnnestOptions`] @@ -1157,7 +1163,8 @@ impl LogicalPlanBuilder { columns: Vec, options: UnnestOptions, ) -> Result { - unnest_with_options(unwrap_arc(self.plan), columns, options).map(Self::new) + unnest_with_options(Arc::unwrap_or_clone(self.plan), columns, options) + .map(Self::new) } } diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 096b65896b69..359de2d30a57 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -51,7 +51,6 @@ use datafusion_common::{ // backwards compatibility use crate::display::PgJsonVisitor; -use crate::logical_plan::tree_node::unwrap_arc; pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; @@ -770,7 +769,7 @@ impl LogicalPlan { .. }) => { // Update schema with unnested column type. - unnest_with_options(unwrap_arc(input), exec_columns, options) + unnest_with_options(Arc::unwrap_or_clone(input), exec_columns, options) } } } diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index 539cb1cf5fb2..273404c8df31 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -379,21 +379,12 @@ impl TreeNode for LogicalPlan { } } -/// Converts a `Arc` without copying, if possible. Copies the plan -/// if there is a shared reference -pub fn unwrap_arc(plan: Arc) -> LogicalPlan { - Arc::try_unwrap(plan) - // if None is returned, there is another reference to this - // LogicalPlan, so we can not own it, and must clone instead - .unwrap_or_else(|node| node.as_ref().clone()) -} - /// Applies `f` to rewrite a `Arc` without copying, if possible fn rewrite_arc Result>>( plan: Arc, mut f: F, ) -> Result>> { - f(unwrap_arc(plan))?.map_data(|new_plan| Ok(Arc::new(new_plan))) + f(Arc::unwrap_or_clone(plan))?.map_data(|new_plan| Ok(Arc::new(new_plan))) } /// rewrite a `Vec` of `Arc` without copying, if possible diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 315284c50839..a6b9bad6c5d9 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -37,7 +37,6 @@ use datafusion_expr::expr::{ }; use datafusion_expr::expr_rewriter::coerce_plan_expr_for_schema; use datafusion_expr::expr_schema::cast_subquery; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::logical_plan::Subquery; use datafusion_expr::type_coercion::binary::{ comparison_coercion, get_input_types, like_coercion, @@ -250,15 +249,19 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { subquery, outer_ref_columns, }) => { - let new_plan = analyze_internal(self.schema, unwrap_arc(subquery))?.data; + let new_plan = + analyze_internal(self.schema, Arc::unwrap_or_clone(subquery))?.data; Ok(Transformed::yes(Expr::ScalarSubquery(Subquery { subquery: Arc::new(new_plan), outer_ref_columns, }))) } Expr::Exists(Exists { subquery, negated }) => { - let new_plan = - analyze_internal(self.schema, unwrap_arc(subquery.subquery))?.data; + let new_plan = analyze_internal( + self.schema, + Arc::unwrap_or_clone(subquery.subquery), + )? + .data; Ok(Transformed::yes(Expr::Exists(Exists { subquery: Subquery { subquery: Arc::new(new_plan), @@ -272,8 +275,11 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { subquery, negated, }) => { - let new_plan = - analyze_internal(self.schema, unwrap_arc(subquery.subquery))?.data; + let new_plan = analyze_internal( + self.schema, + Arc::unwrap_or_clone(subquery.subquery), + )? + .data; let expr_type = expr.get_type(self.schema)?; let subquery_type = new_plan.schema().field(0).data_type(); let common_type = comparison_coercion(&expr_type, subquery_type).ok_or(plan_datafusion_err!( diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index feccf5679efb..3fcee4123b76 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -33,7 +33,6 @@ use datafusion_common::tree_node::{ }; use datafusion_common::{qualified_name, Column, DFSchema, DFSchemaRef, Result}; use datafusion_expr::expr::{Alias, ScalarFunction}; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::logical_plan::{ Aggregate, Filter, LogicalPlan, Projection, Sort, Window, }; @@ -314,7 +313,7 @@ impl CommonSubexprEliminate { schema, .. } = projection; - let input = unwrap_arc(input); + let input = Arc::unwrap_or_clone(input); self.try_unary_plan(expr, input, config)? .map_data(|(new_expr, new_input)| { Projection::try_new_with_schema(new_expr, Arc::new(new_input), schema) @@ -327,7 +326,7 @@ impl CommonSubexprEliminate { config: &dyn OptimizerConfig, ) -> Result> { let Sort { expr, input, fetch } = sort; - let input = unwrap_arc(input); + let input = Arc::unwrap_or_clone(input); let new_sort = self.try_unary_plan(expr, input, config)?.update_data( |(new_expr, new_input)| { LogicalPlan::Sort(Sort { @@ -348,7 +347,7 @@ impl CommonSubexprEliminate { let Filter { predicate, input, .. } = filter; - let input = unwrap_arc(input); + let input = Arc::unwrap_or_clone(input); let expr = vec![predicate]; self.try_unary_plan(expr, input, config)? .map_data(|(mut new_expr, new_input)| { @@ -458,7 +457,7 @@ impl CommonSubexprEliminate { schema, .. } = aggregate; - let input = unwrap_arc(input); + let input = Arc::unwrap_or_clone(input); // Extract common sub-expressions from the aggregate and grouping expressions. self.find_common_exprs(vec![group_expr, aggr_expr], config, ExprMask::Normal)? .map_data(|common| { @@ -729,7 +728,7 @@ fn get_consecutive_window_exprs( window_expr_list.push(window_expr); window_schemas.push(schema); - plan = unwrap_arc(input); + plan = Arc::unwrap_or_clone(input); } (window_expr_list, window_schemas, plan) } diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index b6d49490d437..f1cae1099a4d 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -37,7 +37,6 @@ use datafusion_expr::{ LogicalPlan, LogicalPlanBuilder, Operator, }; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use log::debug; /// Optimizer rule for rewriting predicate(IN/EXISTS) subquery to left semi/anti joins @@ -55,8 +54,10 @@ impl DecorrelatePredicateSubquery { mut subquery: Subquery, config: &dyn OptimizerConfig, ) -> Result { - subquery.subquery = - Arc::new(self.rewrite(unwrap_arc(subquery.subquery), config)?.data); + subquery.subquery = Arc::new( + self.rewrite(Arc::unwrap_or_clone(subquery.subquery), config)? + .data, + ); Ok(subquery) } @@ -164,7 +165,7 @@ impl OptimizerRule for DecorrelatePredicateSubquery { } // iterate through all exists clauses in predicate, turning each into a join - let mut cur_input = unwrap_arc(input); + let mut cur_input = Arc::unwrap_or_clone(input); for subquery in subqueries { if let Some(plan) = build_join(&subquery, &cur_input, config.alias_generator())? diff --git a/datafusion/optimizer/src/eliminate_cross_join.rs b/datafusion/optimizer/src/eliminate_cross_join.rs index fc4eaef80903..20e6641e4d62 100644 --- a/datafusion/optimizer/src/eliminate_cross_join.rs +++ b/datafusion/optimizer/src/eliminate_cross_join.rs @@ -24,7 +24,6 @@ use crate::join_key_set::JoinKeySet; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{internal_err, Result}; use datafusion_expr::expr::{BinaryExpr, Expr}; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::logical_plan::{ CrossJoin, Filter, Join, JoinConstraint, JoinType, LogicalPlan, Projection, }; @@ -114,7 +113,7 @@ impl OptimizerRule for EliminateCrossJoin { input, predicate, .. } = filter; flatten_join_inputs( - unwrap_arc(input), + Arc::unwrap_or_clone(input), &mut possible_join_keys, &mut all_inputs, )?; @@ -217,12 +216,28 @@ fn flatten_join_inputs( ); } possible_join_keys.insert_all_owned(join.on); - flatten_join_inputs(unwrap_arc(join.left), possible_join_keys, all_inputs)?; - flatten_join_inputs(unwrap_arc(join.right), possible_join_keys, all_inputs)?; + flatten_join_inputs( + Arc::unwrap_or_clone(join.left), + possible_join_keys, + all_inputs, + )?; + flatten_join_inputs( + Arc::unwrap_or_clone(join.right), + possible_join_keys, + all_inputs, + )?; } LogicalPlan::CrossJoin(join) => { - flatten_join_inputs(unwrap_arc(join.left), possible_join_keys, all_inputs)?; - flatten_join_inputs(unwrap_arc(join.right), possible_join_keys, all_inputs)?; + flatten_join_inputs( + Arc::unwrap_or_clone(join.left), + possible_join_keys, + all_inputs, + )?; + flatten_join_inputs( + Arc::unwrap_or_clone(join.right), + possible_join_keys, + all_inputs, + )?; } _ => { all_inputs.push(plan); diff --git a/datafusion/optimizer/src/eliminate_filter.rs b/datafusion/optimizer/src/eliminate_filter.rs index 84bb8e782142..bb2b4547e9c2 100644 --- a/datafusion/optimizer/src/eliminate_filter.rs +++ b/datafusion/optimizer/src/eliminate_filter.rs @@ -19,7 +19,6 @@ use datafusion_common::tree_node::Transformed; use datafusion_common::{Result, ScalarValue}; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::{EmptyRelation, Expr, Filter, LogicalPlan}; use std::sync::Arc; @@ -65,7 +64,7 @@ impl OptimizerRule for EliminateFilter { input, .. }) => match v { - Some(true) => Ok(Transformed::yes(unwrap_arc(input))), + Some(true) => Ok(Transformed::yes(Arc::unwrap_or_clone(input))), Some(false) | None => Ok(Transformed::yes(LogicalPlan::EmptyRelation( EmptyRelation { produce_one_row: false, diff --git a/datafusion/optimizer/src/eliminate_limit.rs b/datafusion/optimizer/src/eliminate_limit.rs index a42fe6a6f95b..e48f37a77cd3 100644 --- a/datafusion/optimizer/src/eliminate_limit.rs +++ b/datafusion/optimizer/src/eliminate_limit.rs @@ -20,7 +20,7 @@ use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::tree_node::Transformed; use datafusion_common::Result; -use datafusion_expr::logical_plan::{tree_node::unwrap_arc, EmptyRelation, LogicalPlan}; +use datafusion_expr::logical_plan::{EmptyRelation, LogicalPlan}; use std::sync::Arc; /// Optimizer rule to replace `LIMIT 0` or `LIMIT` whose ancestor LIMIT's skip is @@ -74,7 +74,9 @@ impl OptimizerRule for EliminateLimit { } } else if limit.skip == 0 { // input also can be Limit, so we should apply again. - return Ok(self.rewrite(unwrap_arc(limit.input), _config).unwrap()); + return Ok(self + .rewrite(Arc::unwrap_or_clone(limit.input), _config) + .unwrap()); } Ok(Transformed::no(LogicalPlan::Limit(limit))) } diff --git a/datafusion/optimizer/src/eliminate_nested_union.rs b/datafusion/optimizer/src/eliminate_nested_union.rs index 5d7895bba4d8..e9b38567a982 100644 --- a/datafusion/optimizer/src/eliminate_nested_union.rs +++ b/datafusion/optimizer/src/eliminate_nested_union.rs @@ -21,7 +21,6 @@ use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::tree_node::Transformed; use datafusion_common::Result; use datafusion_expr::expr_rewriter::coerce_plan_expr_for_schema; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::{Distinct, LogicalPlan, Union}; use itertools::Itertools; use std::sync::Arc; @@ -69,7 +68,7 @@ impl OptimizerRule for EliminateNestedUnion { }))) } LogicalPlan::Distinct(Distinct::All(nested_plan)) => { - match unwrap_arc(nested_plan) { + match Arc::unwrap_or_clone(nested_plan) { LogicalPlan::Union(Union { inputs, schema }) => { let inputs = inputs .into_iter() @@ -96,16 +95,17 @@ impl OptimizerRule for EliminateNestedUnion { } fn extract_plans_from_union(plan: Arc) -> Vec { - match unwrap_arc(plan) { - LogicalPlan::Union(Union { inputs, .. }) => { - inputs.into_iter().map(unwrap_arc).collect::>() - } + match Arc::unwrap_or_clone(plan) { + LogicalPlan::Union(Union { inputs, .. }) => inputs + .into_iter() + .map(Arc::unwrap_or_clone) + .collect::>(), plan => vec![plan], } } fn extract_plan_from_distinct(plan: Arc) -> Arc { - match unwrap_arc(plan) { + match Arc::unwrap_or_clone(plan) { LogicalPlan::Distinct(Distinct::All(plan)) => plan, plan => Arc::new(plan), } diff --git a/datafusion/optimizer/src/eliminate_one_union.rs b/datafusion/optimizer/src/eliminate_one_union.rs index 43024107c4f8..383b33637f6f 100644 --- a/datafusion/optimizer/src/eliminate_one_union.rs +++ b/datafusion/optimizer/src/eliminate_one_union.rs @@ -16,9 +16,11 @@ // under the License. //! [`EliminateOneUnion`] eliminates single element `Union` + use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::{tree_node::Transformed, Result}; -use datafusion_expr::logical_plan::{tree_node::unwrap_arc, LogicalPlan, Union}; +use datafusion_expr::logical_plan::{LogicalPlan, Union}; +use std::sync::Arc; use crate::optimizer::ApplyOrder; @@ -48,9 +50,9 @@ impl OptimizerRule for EliminateOneUnion { _config: &dyn OptimizerConfig, ) -> Result> { match plan { - LogicalPlan::Union(Union { mut inputs, .. }) if inputs.len() == 1 => { - Ok(Transformed::yes(unwrap_arc(inputs.pop().unwrap()))) - } + LogicalPlan::Union(Union { mut inputs, .. }) if inputs.len() == 1 => Ok( + Transformed::yes(Arc::unwrap_or_clone(inputs.pop().unwrap())), + ), _ => Ok(Transformed::no(plan)), } } diff --git a/datafusion/optimizer/src/eliminate_outer_join.rs b/datafusion/optimizer/src/eliminate_outer_join.rs index 12534e058152..e7c88df55122 100644 --- a/datafusion/optimizer/src/eliminate_outer_join.rs +++ b/datafusion/optimizer/src/eliminate_outer_join.rs @@ -18,7 +18,6 @@ //! [`EliminateOuterJoin`] converts `LEFT/RIGHT/FULL` joins to `INNER` joins use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::{Column, DFSchema, Result}; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::logical_plan::{Join, JoinType, LogicalPlan}; use datafusion_expr::{Expr, Filter, Operator}; @@ -79,7 +78,7 @@ impl OptimizerRule for EliminateOuterJoin { _config: &dyn OptimizerConfig, ) -> Result> { match plan { - LogicalPlan::Filter(mut filter) => match unwrap_arc(filter.input) { + LogicalPlan::Filter(mut filter) => match Arc::unwrap_or_clone(filter.input) { LogicalPlan::Join(join) => { let mut non_nullable_cols: Vec = vec![]; diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index ac4ed87a4a1a..5db5afd11062 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -41,7 +41,6 @@ use crate::utils::NamePreserver; use datafusion_common::tree_node::{ Transformed, TreeNode, TreeNodeIterator, TreeNodeRecursion, }; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; /// Optimizer rule to prune unnecessary columns from intermediate schemas /// inside the [`LogicalPlan`]. This rule: @@ -181,7 +180,7 @@ fn optimize_projections( let necessary_exprs = necessary_indices.get_required_exprs(schema); return optimize_projections( - unwrap_arc(aggregate.input), + Arc::unwrap_or_clone(aggregate.input), config, necessary_indices, )? @@ -221,7 +220,7 @@ fn optimize_projections( child_reqs.with_exprs(&input_schema, &new_window_expr)?; return optimize_projections( - unwrap_arc(window.input), + Arc::unwrap_or_clone(window.input), config, required_indices.clone(), )? @@ -488,7 +487,7 @@ fn merge_consecutive_projections(proj: Projection) -> Result { let parents_predicates = split_conjunction_owned(filter.predicate); @@ -1139,19 +1138,19 @@ fn convert_to_cross_join_if_beneficial( match plan { // Can be converted back to cross join LogicalPlan::Join(join) if join.on.is_empty() && join.filter.is_none() => { - LogicalPlanBuilder::from(unwrap_arc(join.left)) - .cross_join(unwrap_arc(join.right))? + LogicalPlanBuilder::from(Arc::unwrap_or_clone(join.left)) + .cross_join(Arc::unwrap_or_clone(join.right))? .build() .map(Transformed::yes) } - LogicalPlan::Filter(filter) => convert_to_cross_join_if_beneficial(unwrap_arc( - filter.input, - ))? - .transform_data(|child_plan| { - Filter::try_new(filter.predicate, Arc::new(child_plan)) - .map(LogicalPlan::Filter) - .map(Transformed::yes) - }), + LogicalPlan::Filter(filter) => { + convert_to_cross_join_if_beneficial(Arc::unwrap_or_clone(filter.input))? + .transform_data(|child_plan| { + Filter::try_new(filter.predicate, Arc::new(child_plan)) + .map(LogicalPlan::Filter) + .map(Transformed::yes) + }) + } plan => Ok(Transformed::no(plan)), } } diff --git a/datafusion/optimizer/src/push_down_limit.rs b/datafusion/optimizer/src/push_down_limit.rs index 290b893577b8..dff0b61c6b22 100644 --- a/datafusion/optimizer/src/push_down_limit.rs +++ b/datafusion/optimizer/src/push_down_limit.rs @@ -26,7 +26,6 @@ use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::tree_node::Transformed; use datafusion_common::utils::combine_limit; use datafusion_common::Result; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::logical_plan::{Join, JoinType, Limit, LogicalPlan}; /// Optimization rule that tries to push down `LIMIT`. @@ -83,7 +82,7 @@ impl OptimizerRule for PushDownLimit { }))); }; - match unwrap_arc(input) { + match Arc::unwrap_or_clone(input) { LogicalPlan::TableScan(mut scan) => { let rows_needed = if fetch != 0 { fetch + skip } else { 0 }; let new_fetch = scan diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 45fda094557b..384893bfa94c 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -33,7 +33,6 @@ use datafusion_expr::expr::{Alias, PlannedReplaceSelectItem, WildcardOptions}; use datafusion_expr::expr_rewriter::{ normalize_col, normalize_col_with_schemas_and_ambiguity_check, normalize_cols, }; -use datafusion_expr::logical_plan::tree_node::unwrap_arc; use datafusion_expr::utils::{ expr_as_column_expr, expr_to_columns, find_aggregate_exprs, find_window_exprs, }; @@ -361,9 +360,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .build() } LogicalPlan::Filter(mut filter) => { - filter.input = Arc::new( - self.try_process_aggregate_unnest(unwrap_arc(filter.input))?, - ); + filter.input = + Arc::new(self.try_process_aggregate_unnest(Arc::unwrap_or_clone( + filter.input, + ))?); Ok(LogicalPlan::Filter(filter)) } _ => Ok(input), @@ -401,7 +401,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Projection: tab.array_col AS unnest(tab.array_col) // TableScan: tab // ``` - let mut intermediate_plan = unwrap_arc(input); + let mut intermediate_plan = Arc::unwrap_or_clone(input); let mut intermediate_select_exprs = group_expr; loop { From 9ef2f95241fc761929b71bdf9148ad221a6f34c2 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 03:37:09 +0200 Subject: [PATCH 10/75] Fix typo (#12169) --- datafusion/expr/src/utils.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 11a244a944f8..8f04d3ec3067 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -463,7 +463,7 @@ pub fn expand_qualified_wildcard( /// if bool is true SortExpr comes from `PARTITION BY` column, if false comes from `ORDER BY` column type WindowSortKey = Vec<(Expr, bool)>; -/// Generate a sort key for a given window expr's partition_by and order_bu expr +/// Generate a sort key for a given window expr's partition_by and order_by expr pub fn generate_sort_key( partition_by: &[Expr], order_by: &[Expr], From 7e9ea3ad59071d56093c197c5ecd5c50021deb94 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 26 Aug 2024 21:38:43 -0400 Subject: [PATCH 11/75] Minor: remove vestigal github workflow (#12182) --- .github/workflows/pr_comment.yml | 53 -------------------------------- 1 file changed, 53 deletions(-) delete mode 100644 .github/workflows/pr_comment.yml diff --git a/.github/workflows/pr_comment.yml b/.github/workflows/pr_comment.yml deleted file mode 100644 index 8b6df1c75687..000000000000 --- a/.github/workflows/pr_comment.yml +++ /dev/null @@ -1,53 +0,0 @@ -# Downloads any `message` artifacts created by other jobs -# and posts them as comments to the PR -name: PR Comment - -on: - workflow_run: - workflows: ["Benchmarks"] - types: - - completed - -jobs: - comment: - name: PR Comment - runs-on: ubuntu-latest - if: github.event.workflow_run.conclusion == 'success' - steps: - - name: Dump GitHub context - env: - GITHUB_CONTEXT: ${{ toJSON(github) }} - run: echo "$GITHUB_CONTEXT" - - - name: Download comment message - uses: actions/download-artifact@v4 - with: - name: message - run-id: ${{ github.event.workflow_run.id }} - github-token: ${{ secrets.GITHUB_TOKEN }} - - - name: Download pr number - uses: actions/download-artifact@v4 - with: - name: pr - run-id: ${{ github.event.workflow_run.id }} - github-token: ${{ secrets.GITHUB_TOKEN }} - - - name: Print message and pr number - run: | - cat pr - echo "PR_NUMBER=$(cat pr)" >> "$GITHUB_ENV" - cat message.md - - - name: Post the comment - uses: actions/github-script@v7 - with: - script: | - const fs = require('fs'); - const content = fs.readFileSync('message.md', 'utf8'); - github.rest.issues.createComment({ - issue_number: process.env.PR_NUMBER, - owner: context.repo.owner, - repo: context.repo.repo, - body: content, - }) From 2ac0842954a1ee5605be54810ed668e63379dfdd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Tue, 27 Aug 2024 21:09:58 +0800 Subject: [PATCH 12/75] Remove `AggregateExpr` trait (#12096) * save * Move AggregateFunctionExpr into physical-expr crate * Move AggregateExpr trait into physical-expr crate * Remove AggregateExpr trait * Fix doc and tests * Remove empty crate physical-expr-functions-aggregate * Use func name instead of expr name * Fix tests * Fix tests and clippy lints * Cargo update for datafusion-cli * Add docs * Add functions back * Fix doc --- Cargo.toml | 2 - datafusion-cli/Cargo.lock | 160 ++++++---- datafusion/core/Cargo.toml | 1 - datafusion/core/src/lib.rs | 6 - .../combine_partial_final_agg.rs | 13 +- .../physical_optimizer/update_aggr_exprs.rs | 9 +- datafusion/core/src/physical_planner.rs | 10 +- datafusion/core/src/test_util/mod.rs | 8 +- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 2 +- .../src/aggregate.rs | 167 ---------- .../functions-aggregate-common/src/utils.rs | 21 +- .../Cargo.toml | 48 --- .../src/lib.rs | 20 -- .../src/aggregate.rs | 294 ++++++++++++------ datafusion/physical-expr/src/lib.rs | 25 +- .../physical-expr/src/window/aggregate.rs | 11 +- .../src/window/sliding_aggregate.rs | 13 +- .../src/aggregate_statistics.rs | 38 +-- datafusion/physical-plan/Cargo.toml | 1 - .../physical-plan/src/aggregates/mod.rs | 110 +++---- .../physical-plan/src/aggregates/row_hash.rs | 9 +- .../physical-plan/src/execution_plan.rs | 2 +- datafusion/physical-plan/src/lib.rs | 4 +- datafusion/physical-plan/src/windows/mod.rs | 7 +- datafusion/proto/src/physical_plan/mod.rs | 7 +- .../proto/src/physical_plan/to_proto.rs | 72 ++--- .../tests/cases/roundtrip_physical_plan.rs | 20 +- 27 files changed, 442 insertions(+), 638 deletions(-) delete mode 100644 datafusion/physical-expr-functions-aggregate/Cargo.toml delete mode 100644 datafusion/physical-expr-functions-aggregate/src/lib.rs rename datafusion/{physical-expr-functions-aggregate => physical-expr}/src/aggregate.rs (69%) diff --git a/Cargo.toml b/Cargo.toml index 124747999041..479d2cadc65b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -33,7 +33,6 @@ members = [ "datafusion/optimizer", "datafusion/physical-expr", "datafusion/physical-expr-common", - "datafusion/physical-expr-functions-aggregate", "datafusion/physical-optimizer", "datafusion/physical-plan", "datafusion/proto", @@ -106,7 +105,6 @@ datafusion-functions-window = { path = "datafusion/functions-window", version = datafusion-optimizer = { path = "datafusion/optimizer", version = "41.0.0", default-features = false } datafusion-physical-expr = { path = "datafusion/physical-expr", version = "41.0.0", default-features = false } datafusion-physical-expr-common = { path = "datafusion/physical-expr-common", version = "41.0.0", default-features = false } -datafusion-physical-expr-functions-aggregate = { path = "datafusion/physical-expr-functions-aggregate", version = "41.0.0" } datafusion-physical-optimizer = { path = "datafusion/physical-optimizer", version = "41.0.0" } datafusion-physical-plan = { path = "datafusion/physical-plan", version = "41.0.0" } datafusion-proto = { path = "datafusion/proto", version = "41.0.0" } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 50333d17ca8d..1e89bb3af87e 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -17,6 +17,12 @@ version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" +[[package]] +name = "adler2" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "512761e0bb2578dd7380c6baaa0f4ce03e84f95e960231d1dec8bf4d7d6e2627" + [[package]] name = "adler32" version = "1.2.0" @@ -167,9 +173,9 @@ checksum = "9d151e35f61089500b617991b791fc8bfd237ae50cd5950803758a179b41e67a" [[package]] name = "arrayvec" -version = "0.7.4" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" +checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" @@ -430,7 +436,7 @@ checksum = "6e0c28dcc82d7c8ead5cb13beb15405b57b8546e93215673ff8ca0349a028107" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -765,7 +771,7 @@ dependencies = [ "cc", "cfg-if", "libc", - "miniz_oxide", + "miniz_oxide 0.7.4", "object", "rustc-demangle", ] @@ -815,9 +821,9 @@ dependencies = [ [[package]] name = "blake3" -version = "1.5.3" +version = "1.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9ec96fe9a81b5e365f9db71fe00edc4fe4ca2cc7dcb7861f0603012a7caa210" +checksum = "d82033247fd8e890df8f740e407ad4d038debb9eb1f40533fffb32e7d17dc6f7" dependencies = [ "arrayref", "arrayvec", @@ -999,7 +1005,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -1155,7 +1161,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "edb49164822f3ee45b17acd4a208cfc1251410cf0cad9a833234c9890774dd9f" dependencies = [ "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -1206,7 +1212,6 @@ dependencies = [ "datafusion-optimizer", "datafusion-physical-expr", "datafusion-physical-expr-common", - "datafusion-physical-expr-functions-aggregate", "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-sql", @@ -1501,20 +1506,6 @@ dependencies = [ "rand", ] -[[package]] -name = "datafusion-physical-expr-functions-aggregate" -version = "41.0.0" -dependencies = [ - "ahash", - "arrow", - "datafusion-common", - "datafusion-expr", - "datafusion-expr-common", - "datafusion-functions-aggregate-common", - "datafusion-physical-expr-common", - "rand", -] - [[package]] name = "datafusion-physical-optimizer" version = "41.0.0" @@ -1546,7 +1537,6 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-physical-expr", "datafusion-physical-expr-common", - "datafusion-physical-expr-functions-aggregate", "futures", "half", "hashbrown", @@ -1743,12 +1733,12 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.31" +version = "1.0.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f211bbe8e69bbd0cfdea405084f128ae8b4aaa6b0b522fc8f2b009084797920" +checksum = "9c0596c1eac1f9e04ed902702e9878208b336edc9d6fddc8a48387349bab3666" dependencies = [ "crc32fast", - "miniz_oxide", + "miniz_oxide 0.8.0", ] [[package]] @@ -1831,7 +1821,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -1924,9 +1914,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa82e28a107a8cc405f0839610bdc9b15f1e25ec7d696aa5cf173edbcb1486ab" +checksum = "524e8ac6999421f49a846c2d4411f337e53497d8ec55d67753beffa43c5d9205" dependencies = [ "atomic-waker", "bytes", @@ -2111,7 +2101,7 @@ dependencies = [ "bytes", "futures-channel", "futures-util", - "h2 0.4.5", + "h2 0.4.6", "http 1.1.0", "http-body 1.0.1", "httparse", @@ -2148,7 +2138,7 @@ dependencies = [ "hyper 1.4.1", "hyper-util", "rustls 0.23.12", - "rustls-native-certs 0.7.1", + "rustls-native-certs 0.7.2", "rustls-pki-types", "tokio", "tokio-rustls 0.26.0", @@ -2353,9 +2343,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.156" +version = "0.2.158" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5f43f184355eefb8d17fc948dbecf6c13be3c141f20d834ae842193a448c72a" +checksum = "d8adc4bb1803a324070e64a98ae98f38934d91957a99cfb3a43dcbc01bc56439" [[package]] name = "libflate" @@ -2489,6 +2479,15 @@ dependencies = [ "adler", ] +[[package]] +name = "miniz_oxide" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e2d80299ef12ff69b16a84bb182e3b9df68b5a91574d3d4fa6e41b65deec4df1" +dependencies = [ + "adler2", +] + [[package]] name = "mio" version = "1.0.2" @@ -2829,7 +2828,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3028,9 +3027,9 @@ dependencies = [ [[package]] name = "redox_users" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd283d9651eeda4b2a83a43c1c91b266c40fd76ecd39a50a8c630ae69dc72891" +checksum = "ba009ff324d1fc1b900bd1fdb31564febe58a8ccc8a6fdbb93b543d33b13ca43" dependencies = [ "getrandom", "libredox", @@ -3074,15 +3073,15 @@ checksum = "7a66a03ae7c801facd77a29370b4faec201768915ac14a721ba36f20bc9c209b" [[package]] name = "reqwest" -version = "0.12.5" +version = "0.12.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7d6d2a27d57148378eb5e111173f4276ad26340ecc5c49a4a2152167a2d6a37" +checksum = "f8f4955649ef5c38cc7f9e8aa41761d48fb9677197daea9984dc54f56aad5e63" dependencies = [ "base64 0.22.1", "bytes", "futures-core", "futures-util", - "h2 0.4.5", + "h2 0.4.6", "http 1.1.0", "http-body 1.0.1", "http-body-util", @@ -3098,7 +3097,7 @@ dependencies = [ "pin-project-lite", "quinn", "rustls 0.23.12", - "rustls-native-certs 0.7.1", + "rustls-native-certs 0.7.2", "rustls-pemfile 2.1.3", "rustls-pki-types", "serde", @@ -3114,7 +3113,7 @@ dependencies = [ "wasm-bindgen-futures", "wasm-streams", "web-sys", - "winreg", + "windows-registry", ] [[package]] @@ -3253,9 +3252,9 @@ dependencies = [ [[package]] name = "rustls-native-certs" -version = "0.7.1" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a88d6d420651b496bdd98684116959239430022a115c1240e6c3993be0b15fba" +checksum = "04182dffc9091a404e0fc069ea5cd60e5b866c3adf881eff99a32d048242dffa" dependencies = [ "openssl-probe", "rustls-pemfile 2.1.3", @@ -3421,7 +3420,7 @@ checksum = "24008e81ff7613ed8e5ba0cfaf24e2c2f1e5b8a0495711e44fcd4882fca62bcf" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3563,7 +3562,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3609,7 +3608,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3622,7 +3621,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3644,9 +3643,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.74" +version = "2.0.75" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fceb41e3d546d0bd83421d3409b1460cc7444cd389341a4c880fe7a042cb3d7" +checksum = "f6af063034fc1935ede7be0122941bafa9bacb949334d090b77ca98b5817c7d9" dependencies = [ "proc-macro2", "quote", @@ -3658,6 +3657,9 @@ name = "sync_wrapper" version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a7065abeca94b6a8a577f9bd45aa0867a2238b74e8eb67cf10d492bc39351394" +dependencies = [ + "futures-core", +] [[package]] name = "tempfile" @@ -3704,7 +3706,7 @@ checksum = "a4558b58466b9ad7ca0f102865eccc95938dca1a74a856f2b57b6629050da261" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3774,9 +3776,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.39.2" +version = "1.39.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "daa4fb1bc778bd6f04cbfc4bb2d06a7396a8f299dc33ea1900cedaa316f467b1" +checksum = "9babc99b9923bfa4804bd74722ff02c0381021eafa4db9949217e3be8e84fff5" dependencies = [ "backtrace", "bytes", @@ -3798,7 +3800,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3895,7 +3897,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -3940,7 +3942,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] @@ -4095,7 +4097,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", "wasm-bindgen-shared", ] @@ -4129,7 +4131,7 @@ checksum = "afc340c74d9005395cf9dd098506f7f44e38f2b4a21c6aaacf9a105ea5e1e836" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -4213,6 +4215,36 @@ dependencies = [ "windows-targets 0.52.6", ] +[[package]] +name = "windows-registry" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e400001bb720a623c1c69032f8e3e4cf09984deec740f007dd2b03ec864804b0" +dependencies = [ + "windows-result", + "windows-strings", + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-result" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d1043d8214f791817bab27572aaa8af63732e11bf84aa21a45a78d6c317ae0e" +dependencies = [ + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-strings" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4cd9b125c486025df0eabcb585e62173c6c9eddcec5d117d3b6e8c30e2ee4d10" +dependencies = [ + "windows-result", + "windows-targets 0.52.6", +] + [[package]] name = "windows-sys" version = "0.48.0" @@ -4361,16 +4393,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" -[[package]] -name = "winreg" -version = "0.52.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a277a57398d4bfa075df44f501a17cfdf8542d224f0d36095a2adc7aee4ef0a5" -dependencies = [ - "cfg-if", - "windows-sys 0.48.0", -] - [[package]] name = "xmlparser" version = "0.13.6" @@ -4404,7 +4426,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.74", + "syn 2.0.75", ] [[package]] diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index adbba3eb31d6..de228e058096 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -110,7 +110,6 @@ datafusion-functions-window = { workspace = true } datafusion-optimizer = { workspace = true } datafusion-physical-expr = { workspace = true } datafusion-physical-expr-common = { workspace = true } -datafusion-physical-expr-functions-aggregate = { workspace = true } datafusion-physical-optimizer = { workspace = true } datafusion-physical-plan = { workspace = true } datafusion-sql = { workspace = true } diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index 735a381586ad..67f3cb01c0a4 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -490,7 +490,6 @@ //! [`PhysicalOptimizerRule`]: datafusion::physical_optimizer::optimizer::PhysicalOptimizerRule //! [`Schema`]: arrow::datatypes::Schema //! [`PhysicalExpr`]: physical_plan::PhysicalExpr -//! [`AggregateExpr`]: physical_plan::AggregateExpr //! [`RecordBatch`]: arrow::record_batch::RecordBatch //! [`RecordBatchReader`]: arrow::record_batch::RecordBatchReader //! [`Array`]: arrow::array::Array @@ -556,11 +555,6 @@ pub mod physical_expr_common { pub use datafusion_physical_expr_common::*; } -/// re-export of [`datafusion_physical_expr_functions_aggregate`] crate -pub mod physical_expr_functions_aggregate { - pub use datafusion_physical_expr_functions_aggregate::*; -} - /// re-export of [`datafusion_physical_expr`] crate pub mod physical_expr { pub use datafusion_physical_expr::*; diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 8cbb187f7bd2..1a12fc7de888 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -26,7 +26,8 @@ use crate::physical_plan::ExecutionPlan; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_physical_expr::{physical_exprs_equal, AggregateExpr, PhysicalExpr}; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; +use datafusion_physical_expr::{physical_exprs_equal, PhysicalExpr}; use datafusion_physical_optimizer::PhysicalOptimizerRule; /// CombinePartialFinalAggregate optimizer rule combines the adjacent Partial and Final AggregateExecs @@ -122,7 +123,7 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { type GroupExprsRef<'a> = ( &'a PhysicalGroupBy, - &'a [Arc], + &'a [Arc], &'a [Option>], ); @@ -171,8 +172,8 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::sum::sum_udaf; + use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; /// Runs the CombinePartialFinalAggregate optimizer and asserts the plan against the expected macro_rules! assert_optimized { @@ -224,7 +225,7 @@ mod tests { fn partial_aggregate_exec( input: Arc, group_by: PhysicalGroupBy, - aggr_expr: Vec>, + aggr_expr: Vec>, ) -> Arc { let schema = input.schema(); let n_aggr = aggr_expr.len(); @@ -244,7 +245,7 @@ mod tests { fn final_aggregate_exec( input: Arc, group_by: PhysicalGroupBy, - aggr_expr: Vec>, + aggr_expr: Vec>, ) -> Arc { let schema = input.schema(); let n_aggr = aggr_expr.len(); @@ -272,7 +273,7 @@ mod tests { expr: Arc, name: &str, schema: &Schema, - ) -> Arc { + ) -> Arc { AggregateExprBuilder::new(count_udaf(), vec![expr]) .schema(Arc::new(schema.clone())) .alias(name) diff --git a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index f8edf73e3d2a..a2726d62e9f6 100644 --- a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -23,8 +23,9 @@ use std::sync::Arc; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{plan_datafusion_err, Result}; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr::{ - reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalSortRequirement, + reverse_order_bys, EquivalenceProperties, PhysicalSortRequirement, }; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::aggregates::concat_slices; @@ -117,7 +118,7 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { /// /// # Parameters /// -/// * `aggr_exprs` - A vector of `Arc` representing the +/// * `aggr_exprs` - A vector of `Arc` representing the /// aggregate expressions to be optimized. /// * `prefix_requirement` - An array slice representing the ordering /// requirements preceding the aggregate expressions. @@ -130,10 +131,10 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { /// successfully. Any errors occurring during the conversion process are /// passed through. fn try_convert_aggregate_if_better( - aggr_exprs: Vec>, + aggr_exprs: Vec>, prefix_requirement: &[PhysicalSortRequirement], eq_properties: &EquivalenceProperties, -) -> Result>> { +) -> Result>> { aggr_exprs .into_iter() .map(|aggr_expr| { diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 8d6c5089fa34..9501d3c6bbbb 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -58,8 +58,8 @@ use crate::physical_plan::unnest::UnnestExec; use crate::physical_plan::values::ValuesExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{ - displayable, windows, AggregateExpr, ExecutionPlan, ExecutionPlanProperties, - InputOrderMode, Partitioning, PhysicalExpr, WindowExpr, + displayable, windows, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, + Partitioning, PhysicalExpr, WindowExpr, }; use arrow::compute::SortOptions; @@ -81,9 +81,9 @@ use datafusion_expr::{ DescribeTable, DmlStatement, Extension, Filter, RecursiveQuery, StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, }; +use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::LexOrdering; -use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_sql::utils::window_expr_common_partition_keys; @@ -719,7 +719,7 @@ impl DefaultPhysicalPlanner { // optimization purposes. For example, a FIRST_VALUE may turn // into a LAST_VALUE with the reverse ordering requirement. // To reflect such changes to subsequent stages, use the updated - // `AggregateExpr`/`PhysicalSortExpr` objects. + // `AggregateFunctionExpr`/`PhysicalSortExpr` objects. let updated_aggregates = initial_aggr.aggr_expr().to_vec(); let next_partition_mode = if can_repartition { @@ -1541,7 +1541,7 @@ pub fn create_window_expr( } type AggregateExprWithOptionalArgs = ( - Arc, + Arc, // The filter clause, if any Option>, // Ordering requirements, if any diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index ca8376fdec0a..faa9378535fd 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -48,13 +48,11 @@ use datafusion_common::TableReference; use datafusion_expr::utils::COUNT_STAR_EXPANSION; use datafusion_expr::{CreateExternalTable, Expr, TableType}; use datafusion_functions_aggregate::count::count_udaf; -use datafusion_physical_expr::{ - expressions, AggregateExpr, EquivalenceProperties, PhysicalExpr, -}; +use datafusion_physical_expr::{expressions, EquivalenceProperties, PhysicalExpr}; use async_trait::async_trait; use datafusion_catalog::Session; -use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; +use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use futures::Stream; use tempfile::TempDir; // backwards compatibility @@ -429,7 +427,7 @@ impl TestAggregate { } /// Return appropriate expr depending if COUNT is for col or table (*) - pub fn count_expr(&self, schema: &Schema) -> Arc { + pub fn count_expr(&self, schema: &Schema) -> Arc { AggregateExprBuilder::new(count_udaf(), vec![self.column()]) .schema(Arc::new(schema.clone())) .alias(self.column_name()) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 138e5bda7f39..62e9be63983c 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -25,7 +25,7 @@ use arrow::util::pretty::pretty_format_batches; use arrow_array::types::Int64Type; use datafusion::common::Result; use datafusion::datasource::MemTable; -use datafusion::physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; +use datafusion::physical_expr::aggregate::AggregateExprBuilder; use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; diff --git a/datafusion/functions-aggregate-common/src/aggregate.rs b/datafusion/functions-aggregate-common/src/aggregate.rs index 698d1350cb61..c9cbaa8396fc 100644 --- a/datafusion/functions-aggregate-common/src/aggregate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate.rs @@ -15,172 +15,5 @@ // specific language governing permissions and limitations // under the License. -//! [`AggregateExpr`] which defines the interface all aggregate expressions -//! (built-in and custom) need to satisfy. - -use crate::order::AggregateOrderSensitivity; -use arrow::datatypes::{DataType, Field}; -use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; -use datafusion_expr_common::accumulator::Accumulator; -use datafusion_expr_common::groups_accumulator::GroupsAccumulator; -use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; -use std::fmt::Debug; -use std::{any::Any, sync::Arc}; - pub mod count_distinct; pub mod groups_accumulator; - -/// An aggregate expression that: -/// * knows its resulting field -/// * knows how to create its accumulator -/// * knows its accumulator's state's field -/// * knows the expressions from whose its accumulator will receive values -/// -/// Any implementation of this trait also needs to implement the -/// `PartialEq` to allows comparing equality between the -/// trait objects. -pub trait AggregateExpr: Send + Sync + Debug + PartialEq { - /// Returns the aggregate expression as [`Any`] so that it can be - /// downcast to a specific implementation. - fn as_any(&self) -> &dyn Any; - - /// the field of the final result of this aggregation. - fn field(&self) -> Result; - - /// the accumulator used to accumulate values from the expressions. - /// the accumulator expects the same number of arguments as `expressions` and must - /// return states with the same description as `state_fields` - fn create_accumulator(&self) -> Result>; - - /// the fields that encapsulate the Accumulator's state - /// the number of fields here equals the number of states that the accumulator contains - fn state_fields(&self) -> Result>; - - /// expressions that are passed to the Accumulator. - /// Single-column aggregations such as `sum` return a single value, others (e.g. `cov`) return many. - fn expressions(&self) -> Vec>; - - /// Order by requirements for the aggregate function - /// By default it is `None` (there is no requirement) - /// Order-sensitive aggregators, such as `FIRST_VALUE(x ORDER BY y)` should implement this - fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { - None - } - - /// Indicates whether aggregator can produce the correct result with any - /// arbitrary input ordering. By default, we assume that aggregate expressions - /// are order insensitive. - fn order_sensitivity(&self) -> AggregateOrderSensitivity { - AggregateOrderSensitivity::Insensitive - } - - /// Sets the indicator whether ordering requirements of the aggregator is - /// satisfied by its input. If this is not the case, aggregators with order - /// sensitivity `AggregateOrderSensitivity::Beneficial` can still produce - /// the correct result with possibly more work internally. - /// - /// # Returns - /// - /// Returns `Ok(Some(updated_expr))` if the process completes successfully. - /// If the expression can benefit from existing input ordering, but does - /// not implement the method, returns an error. Order insensitive and hard - /// requirement aggregators return `Ok(None)`. - fn with_beneficial_ordering( - self: Arc, - _requirement_satisfied: bool, - ) -> Result>> { - if self.order_bys().is_some() && self.order_sensitivity().is_beneficial() { - return exec_err!( - "Should implement with satisfied for aggregator :{:?}", - self.name() - ); - } - Ok(None) - } - - /// Human readable name such as `"MIN(c2)"`. The default - /// implementation returns placeholder text. - fn name(&self) -> &str { - "AggregateExpr: default name" - } - - /// If the aggregate expression has a specialized - /// [`GroupsAccumulator`] implementation. If this returns true, - /// `[Self::create_groups_accumulator`] will be called. - fn groups_accumulator_supported(&self) -> bool { - false - } - - /// Return a specialized [`GroupsAccumulator`] that manages state - /// for all groups. - /// - /// For maximum performance, a [`GroupsAccumulator`] should be - /// implemented in addition to [`Accumulator`]. - fn create_groups_accumulator(&self) -> Result> { - not_impl_err!("GroupsAccumulator hasn't been implemented for {self:?} yet") - } - - /// Construct an expression that calculates the aggregate in reverse. - /// Typically the "reverse" expression is itself (e.g. SUM, COUNT). - /// For aggregates that do not support calculation in reverse, - /// returns None (which is the default value). - fn reverse_expr(&self) -> Option> { - None - } - - /// Creates accumulator implementation that supports retract - fn create_sliding_accumulator(&self) -> Result> { - not_impl_err!("Retractable Accumulator hasn't been implemented for {self:?} yet") - } - - /// Returns all expressions used in the [`AggregateExpr`]. - /// These expressions are (1)function arguments, (2) order by expressions. - fn all_expressions(&self) -> AggregatePhysicalExpressions { - let args = self.expressions(); - let order_bys = self.order_bys().unwrap_or(&[]); - let order_by_exprs = order_bys - .iter() - .map(|sort_expr| Arc::clone(&sort_expr.expr)) - .collect::>(); - AggregatePhysicalExpressions { - args, - order_by_exprs, - } - } - - /// Rewrites [`AggregateExpr`], with new expressions given. The argument should be consistent - /// with the return value of the [`AggregateExpr::all_expressions`] method. - /// Returns `Some(Arc)` if re-write is supported, otherwise returns `None`. - fn with_new_expressions( - &self, - _args: Vec>, - _order_by_exprs: Vec>, - ) -> Option> { - None - } - - /// If this function is max, return (output_field, true) - /// if the function is min, return (output_field, false) - /// otherwise return None (the default) - /// - /// output_field is the name of the column produced by this aggregate - /// - /// Note: this is used to use special aggregate implementations in certain conditions - fn get_minmax_desc(&self) -> Option<(Field, bool)> { - None - } - - /// Returns default value of the function given the input is Null - /// Most of the aggregate function return Null if input is Null, - /// while `count` returns 0 if input is Null - fn default_value(&self, data_type: &DataType) -> Result; -} - -/// Stores the physical expressions used inside the `AggregateExpr`. -pub struct AggregatePhysicalExpressions { - /// Aggregate function arguments - pub args: Vec>, - /// Order by expressions - pub order_by_exprs: Vec>, -} diff --git a/datafusion/functions-aggregate-common/src/utils.rs b/datafusion/functions-aggregate-common/src/utils.rs index 7b8ce0397af8..4fba772d8ddc 100644 --- a/datafusion/functions-aggregate-common/src/utils.rs +++ b/datafusion/functions-aggregate-common/src/utils.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use std::{any::Any, sync::Arc}; +use std::sync::Arc; use arrow::array::{ArrayRef, AsArray}; use arrow::datatypes::ArrowNativeType; @@ -32,25 +32,6 @@ use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_expr_common::accumulator::Accumulator; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; -use crate::aggregate::AggregateExpr; - -/// Downcast a `Box` or `Arc` -/// and return the inner trait object as [`Any`] so -/// that it can be downcast to a specific implementation. -/// -/// This method is used when implementing the `PartialEq` -/// for [`AggregateExpr`] aggregation expressions and allows comparing the equality -/// between the trait objects. -pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { - if let Some(obj) = any.downcast_ref::>() { - obj.as_any() - } else if let Some(obj) = any.downcast_ref::>() { - obj.as_any() - } else { - any - } -} - /// Convert scalar values from an accumulator into arrays. pub fn get_accum_scalar_values_as_arrays( accum: &mut dyn Accumulator, diff --git a/datafusion/physical-expr-functions-aggregate/Cargo.toml b/datafusion/physical-expr-functions-aggregate/Cargo.toml deleted file mode 100644 index 6eed89614c53..000000000000 --- a/datafusion/physical-expr-functions-aggregate/Cargo.toml +++ /dev/null @@ -1,48 +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. - -[package] -name = "datafusion-physical-expr-functions-aggregate" -description = "Logical plan and expression representation for DataFusion query engine" -keywords = ["datafusion", "logical", "plan", "expressions"] -readme = "README.md" -version = { workspace = true } -edition = { workspace = true } -homepage = { workspace = true } -repository = { workspace = true } -license = { workspace = true } -authors = { workspace = true } -rust-version = { workspace = true } - -[lints] -workspace = true - -[lib] -name = "datafusion_physical_expr_functions_aggregate" -path = "src/lib.rs" - -[features] - -[dependencies] -ahash = { workspace = true } -arrow = { workspace = true } -datafusion-common = { workspace = true } -datafusion-expr = { workspace = true } -datafusion-expr-common = { workspace = true } -datafusion-functions-aggregate-common = { workspace = true } -datafusion-physical-expr-common = { workspace = true } -rand = { workspace = true } diff --git a/datafusion/physical-expr-functions-aggregate/src/lib.rs b/datafusion/physical-expr-functions-aggregate/src/lib.rs deleted file mode 100644 index 2ff7ff5777ec..000000000000 --- a/datafusion/physical-expr-functions-aggregate/src/lib.rs +++ /dev/null @@ -1,20 +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. - -//! Technically, all aggregate functions that depend on `expr` crate should be included here. - -pub mod aggregate; diff --git a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs similarity index 69% rename from datafusion/physical-expr-functions-aggregate/src/aggregate.rs rename to datafusion/physical-expr/src/aggregate.rs index fd986e00a7ef..5c1216f2a386 100644 --- a/datafusion/physical-expr-functions-aggregate/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -15,29 +15,46 @@ // specific language governing permissions and limitations // under the License. +pub(crate) mod groups_accumulator { + #[allow(unused_imports)] + pub(crate) mod accumulate { + pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; + } + pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ + accumulate::NullState, GroupsAccumulatorAdapter, + }; +} +pub(crate) mod stats { + pub use datafusion_functions_aggregate_common::stats::StatsType; +} +pub mod utils { + pub use datafusion_functions_aggregate_common::utils::{ + adjust_output_array, get_accum_scalar_values_as_arrays, get_sort_options, + ordering_fields, DecimalAverager, Hashable, + }; +} + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::ScalarValue; use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_expr::AggregateUDF; use datafusion_expr::ReversedUDAF; use datafusion_expr_common::accumulator::Accumulator; -use datafusion_expr_common::groups_accumulator::GroupsAccumulator; use datafusion_expr_common::type_coercion::aggregates::check_arg_count; use datafusion_functions_aggregate_common::accumulator::AccumulatorArgs; use datafusion_functions_aggregate_common::accumulator::StateFieldsArgs; -use datafusion_functions_aggregate_common::aggregate::AggregateExpr; use datafusion_functions_aggregate_common::order::AggregateOrderSensitivity; -use datafusion_functions_aggregate_common::utils::{self, down_cast_any_ref}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_expr_common::utils::reverse_order_bys; +use datafusion_expr_common::groups_accumulator::GroupsAccumulator; use std::fmt::Debug; -use std::{any::Any, sync::Arc}; +use std::sync::Arc; -/// Builder for physical [`AggregateExpr`] +/// Builder for physical [`AggregateFunctionExpr`] /// -/// `AggregateExpr` contains the information necessary to call +/// `AggregateFunctionExpr` contains the information necessary to call /// an aggregate expression. #[derive(Debug, Clone)] pub struct AggregateExprBuilder { @@ -71,7 +88,7 @@ impl AggregateExprBuilder { } } - pub fn build(self) -> Result> { + pub fn build(self) -> Result> { let Self { fun, args, @@ -204,6 +221,17 @@ impl AggregateFunctionExpr { &self.fun } + /// expressions that are passed to the Accumulator. + /// Single-column aggregations such as `sum` return a single value, others (e.g. `cov`) return many. + pub fn expressions(&self) -> Vec> { + self.args.clone() + } + + /// Human readable name such as `"MIN(c2)"`. + pub fn name(&self) -> &str { + &self.name + } + /// Return if the aggregation is distinct pub fn is_distinct(&self) -> bool { self.is_distinct @@ -219,34 +247,13 @@ impl AggregateFunctionExpr { self.is_reversed } + /// Return if the aggregation is nullable pub fn is_nullable(&self) -> bool { self.is_nullable } -} -impl AggregateExpr for AggregateFunctionExpr { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } - - fn expressions(&self) -> Vec> { - self.args.clone() - } - - fn state_fields(&self) -> Result> { - let args = StateFieldsArgs { - name: &self.name, - input_types: &self.input_types, - return_type: &self.data_type, - ordering_fields: &self.ordering_fields, - is_distinct: self.is_distinct, - }; - - self.fun.state_fields(args) - } - - fn field(&self) -> Result { + /// the field of the final result of this aggregation. + pub fn field(&self) -> Result { Ok(Field::new( &self.name, self.data_type.clone(), @@ -254,7 +261,10 @@ impl AggregateExpr for AggregateFunctionExpr { )) } - fn create_accumulator(&self) -> Result> { + /// the accumulator used to accumulate values from the expressions. + /// the accumulator expects the same number of arguments as `expressions` and must + /// return states with the same description as `state_fields` + pub fn create_accumulator(&self) -> Result> { let acc_args = AccumulatorArgs { return_type: &self.data_type, schema: &self.schema, @@ -269,7 +279,83 @@ impl AggregateExpr for AggregateFunctionExpr { self.fun.accumulator(acc_args) } - fn create_sliding_accumulator(&self) -> Result> { + /// the field of the final result of this aggregation. + pub fn state_fields(&self) -> Result> { + let args = StateFieldsArgs { + name: &self.name, + input_types: &self.input_types, + return_type: &self.data_type, + ordering_fields: &self.ordering_fields, + is_distinct: self.is_distinct, + }; + + self.fun.state_fields(args) + } + + /// Order by requirements for the aggregate function + /// By default it is `None` (there is no requirement) + /// Order-sensitive aggregators, such as `FIRST_VALUE(x ORDER BY y)` should implement this + pub fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { + if self.ordering_req.is_empty() { + return None; + } + + if !self.order_sensitivity().is_insensitive() { + return Some(&self.ordering_req); + } + + None + } + + /// Indicates whether aggregator can produce the correct result with any + /// arbitrary input ordering. By default, we assume that aggregate expressions + /// are order insensitive. + pub fn order_sensitivity(&self) -> AggregateOrderSensitivity { + if !self.ordering_req.is_empty() { + // If there is requirement, use the sensitivity of the implementation + self.fun.order_sensitivity() + } else { + // If no requirement, aggregator is order insensitive + AggregateOrderSensitivity::Insensitive + } + } + + /// Sets the indicator whether ordering requirements of the aggregator is + /// satisfied by its input. If this is not the case, aggregators with order + /// sensitivity `AggregateOrderSensitivity::Beneficial` can still produce + /// the correct result with possibly more work internally. + /// + /// # Returns + /// + /// Returns `Ok(Some(updated_expr))` if the process completes successfully. + /// If the expression can benefit from existing input ordering, but does + /// not implement the method, returns an error. Order insensitive and hard + /// requirement aggregators return `Ok(None)`. + pub fn with_beneficial_ordering( + self: Arc, + beneficial_ordering: bool, + ) -> Result>> { + let Some(updated_fn) = self + .fun + .clone() + .with_beneficial_ordering(beneficial_ordering)? + else { + return Ok(None); + }; + + AggregateExprBuilder::new(Arc::new(updated_fn), self.args.to_vec()) + .order_by(self.ordering_req.to_vec()) + .schema(Arc::new(self.schema.clone())) + .alias(self.name().to_string()) + .with_ignore_nulls(self.ignore_nulls) + .with_distinct(self.is_distinct) + .with_reversed(self.is_reversed) + .build() + .map(Some) + } + + /// Creates accumulator implementation that supports retract + pub fn create_sliding_accumulator(&self) -> Result> { let args = AccumulatorArgs { return_type: &self.data_type, schema: &self.schema, @@ -335,11 +421,10 @@ impl AggregateExpr for AggregateFunctionExpr { Ok(accumulator) } - fn name(&self) -> &str { - &self.name - } - - fn groups_accumulator_supported(&self) -> bool { + /// If the aggregate expression has a specialized + /// [`GroupsAccumulator`] implementation. If this returns true, + /// `[Self::create_groups_accumulator`] will be called. + pub fn groups_accumulator_supported(&self) -> bool { let args = AccumulatorArgs { return_type: &self.data_type, schema: &self.schema, @@ -353,7 +438,12 @@ impl AggregateExpr for AggregateFunctionExpr { self.fun.groups_accumulator_supported(args) } - fn create_groups_accumulator(&self) -> Result> { + /// Return a specialized [`GroupsAccumulator`] that manages state + /// for all groups. + /// + /// For maximum performance, a [`GroupsAccumulator`] should be + /// implemented in addition to [`Accumulator`]. + pub fn create_groups_accumulator(&self) -> Result> { let args = AccumulatorArgs { return_type: &self.data_type, schema: &self.schema, @@ -367,52 +457,11 @@ impl AggregateExpr for AggregateFunctionExpr { self.fun.create_groups_accumulator(args) } - fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { - if self.ordering_req.is_empty() { - return None; - } - - if !self.order_sensitivity().is_insensitive() { - return Some(&self.ordering_req); - } - - None - } - - fn order_sensitivity(&self) -> AggregateOrderSensitivity { - if !self.ordering_req.is_empty() { - // If there is requirement, use the sensitivity of the implementation - self.fun.order_sensitivity() - } else { - // If no requirement, aggregator is order insensitive - AggregateOrderSensitivity::Insensitive - } - } - - fn with_beneficial_ordering( - self: Arc, - beneficial_ordering: bool, - ) -> Result>> { - let Some(updated_fn) = self - .fun - .clone() - .with_beneficial_ordering(beneficial_ordering)? - else { - return Ok(None); - }; - - AggregateExprBuilder::new(Arc::new(updated_fn), self.args.to_vec()) - .order_by(self.ordering_req.to_vec()) - .schema(Arc::new(self.schema.clone())) - .alias(self.name().to_string()) - .with_ignore_nulls(self.ignore_nulls) - .with_distinct(self.is_distinct) - .with_reversed(self.is_reversed) - .build() - .map(Some) - } - - fn reverse_expr(&self) -> Option> { + /// Construct an expression that calculates the aggregate in reverse. + /// Typically the "reverse" expression is itself (e.g. SUM, COUNT). + /// For aggregates that do not support calculation in reverse, + /// returns None (which is the default value). + pub fn reverse_expr(&self) -> Option> { match self.fun.reverse_udf() { ReversedUDAF::NotSupported => None, ReversedUDAF::Identical => Some(Arc::new(self.clone())), @@ -440,33 +489,72 @@ impl AggregateExpr for AggregateFunctionExpr { } } - fn get_minmax_desc(&self) -> Option<(Field, bool)> { + /// Returns all expressions used in the [`AggregateFunctionExpr`]. + /// These expressions are (1)function arguments, (2) order by expressions. + pub fn all_expressions(&self) -> AggregatePhysicalExpressions { + let args = self.expressions(); + let order_bys = self.order_bys().unwrap_or(&[]); + let order_by_exprs = order_bys + .iter() + .map(|sort_expr| Arc::clone(&sort_expr.expr)) + .collect::>(); + AggregatePhysicalExpressions { + args, + order_by_exprs, + } + } + + /// Rewrites [`AggregateFunctionExpr`], with new expressions given. The argument should be consistent + /// with the return value of the [`AggregateFunctionExpr::all_expressions`] method. + /// Returns `Some(Arc)` if re-write is supported, otherwise returns `None`. + pub fn with_new_expressions( + &self, + _args: Vec>, + _order_by_exprs: Vec>, + ) -> Option> { + None + } + + /// If this function is max, return (output_field, true) + /// if the function is min, return (output_field, false) + /// otherwise return None (the default) + /// + /// output_field is the name of the column produced by this aggregate + /// + /// Note: this is used to use special aggregate implementations in certain conditions + pub fn get_minmax_desc(&self) -> Option<(Field, bool)> { self.fun .is_descending() .and_then(|flag| self.field().ok().map(|f| (f, flag))) } - fn default_value(&self, data_type: &DataType) -> Result { + /// Returns default value of the function given the input is Null + /// Most of the aggregate function return Null if input is Null, + /// while `count` returns 0 if input is Null + pub fn default_value(&self, data_type: &DataType) -> Result { self.fun.default_value(data_type) } } -impl PartialEq for AggregateFunctionExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.data_type == x.data_type - && self.fun == x.fun - && self.args.len() == x.args.len() - && self - .args - .iter() - .zip(x.args.iter()) - .all(|(this_arg, other_arg)| this_arg.eq(other_arg)) - }) - .unwrap_or(false) +/// Stores the physical expressions used inside the `AggregateExpr`. +pub struct AggregatePhysicalExpressions { + /// Aggregate function arguments + pub args: Vec>, + /// Order by expressions + pub order_by_exprs: Vec>, +} + +impl PartialEq for AggregateFunctionExpr { + fn eq(&self, other: &Self) -> bool { + self.name == other.name + && self.data_type == other.data_type + && self.fun == other.fun + && self.args.len() == other.args.len() + && self + .args + .iter() + .zip(other.args.iter()) + .all(|(this_arg, other_arg)| this_arg.eq(other_arg)) } } diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index c4255172d680..7db7188b85d3 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -19,27 +19,7 @@ #![deny(clippy::clone_on_ref_ptr)] // Backward compatibility -pub mod aggregate { - pub(crate) mod groups_accumulator { - #[allow(unused_imports)] - pub(crate) mod accumulate { - pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; - } - pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ - accumulate::NullState, GroupsAccumulatorAdapter, - }; - } - pub(crate) mod stats { - pub use datafusion_functions_aggregate_common::stats::StatsType; - } - pub mod utils { - pub use datafusion_functions_aggregate_common::utils::{ - adjust_output_array, down_cast_any_ref, get_accum_scalar_values_as_arrays, - get_sort_options, ordering_fields, DecimalAverager, Hashable, - }; - } - pub use datafusion_functions_aggregate_common::aggregate::AggregateExpr; -} +pub mod aggregate; pub mod analysis; pub mod binary_map { pub use datafusion_physical_expr_common::binary_map::{ArrowBytesSet, OutputType}; @@ -67,9 +47,6 @@ pub mod execution_props { pub use aggregate::groups_accumulator::{GroupsAccumulatorAdapter, NullState}; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; -pub use datafusion_functions_aggregate_common::aggregate::{ - AggregateExpr, AggregatePhysicalExpressions, -}; pub use equivalence::{calculate_union, ConstExpr, EquivalenceProperties}; pub use partitioning::{Distribution, Partitioning}; pub use physical_expr::{ diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 52015f425217..5439e140502a 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -29,20 +29,19 @@ use datafusion_common::ScalarValue; use datafusion_common::{DataFusionError, Result}; use datafusion_expr::{Accumulator, WindowFrame}; +use crate::aggregate::AggregateFunctionExpr; use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, SlidingAggregateWindowExpr, WindowExpr, }; -use crate::{ - expressions::PhysicalSortExpr, reverse_order_bys, AggregateExpr, PhysicalExpr, -}; +use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; /// A window expr that takes the form of an aggregate function. /// /// See comments on [`WindowExpr`] for more details. #[derive(Debug)] pub struct PlainAggregateWindowExpr { - aggregate: Arc, + aggregate: Arc, partition_by: Vec>, order_by: Vec, window_frame: Arc, @@ -51,7 +50,7 @@ pub struct PlainAggregateWindowExpr { impl PlainAggregateWindowExpr { /// Create a new aggregate window function expression pub fn new( - aggregate: Arc, + aggregate: Arc, partition_by: &[Arc], order_by: &[PhysicalSortExpr], window_frame: Arc, @@ -65,7 +64,7 @@ impl PlainAggregateWindowExpr { } /// Get aggregate expr of AggregateWindowExpr - pub fn get_aggregate_expr(&self) -> &Arc { + pub fn get_aggregate_expr(&self) -> &Arc { &self.aggregate } } diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index afa799e86953..ac3a4f4c09ec 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -28,13 +28,12 @@ use arrow::record_batch::RecordBatch; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::{Accumulator, WindowFrame}; +use crate::aggregate::AggregateFunctionExpr; use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, PlainAggregateWindowExpr, WindowExpr, }; -use crate::{ - expressions::PhysicalSortExpr, reverse_order_bys, AggregateExpr, PhysicalExpr, -}; +use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; /// A window expr that takes the form of an aggregate function that /// can be incrementally computed over sliding windows. @@ -42,7 +41,7 @@ use crate::{ /// See comments on [`WindowExpr`] for more details. #[derive(Debug)] pub struct SlidingAggregateWindowExpr { - aggregate: Arc, + aggregate: Arc, partition_by: Vec>, order_by: Vec, window_frame: Arc, @@ -51,7 +50,7 @@ pub struct SlidingAggregateWindowExpr { impl SlidingAggregateWindowExpr { /// Create a new (sliding) aggregate window function expression. pub fn new( - aggregate: Arc, + aggregate: Arc, partition_by: &[Arc], order_by: &[PhysicalSortExpr], window_frame: Arc, @@ -64,8 +63,8 @@ impl SlidingAggregateWindowExpr { } } - /// Get the [AggregateExpr] of this object. - pub fn get_aggregate_expr(&self) -> &Arc { + /// Get the [AggregateFunctionExpr] of this object. + pub fn get_aggregate_expr(&self) -> &Arc { &self.aggregate } } diff --git a/datafusion/physical-optimizer/src/aggregate_statistics.rs b/datafusion/physical-optimizer/src/aggregate_statistics.rs index 66b250c5063b..2b8725b5bac7 100644 --- a/datafusion/physical-optimizer/src/aggregate_statistics.rs +++ b/datafusion/physical-optimizer/src/aggregate_statistics.rs @@ -23,7 +23,7 @@ use datafusion_common::scalar::ScalarValue; use datafusion_common::Result; use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::projection::ProjectionExec; -use datafusion_physical_plan::{expressions, AggregateExpr, ExecutionPlan, Statistics}; +use datafusion_physical_plan::{expressions, ExecutionPlan, Statistics}; use crate::PhysicalOptimizerRule; use datafusion_common::stats::Precision; @@ -58,12 +58,12 @@ impl PhysicalOptimizerRule for AggregateStatistics { let mut projections = vec![]; for expr in partial_agg_exec.aggr_expr() { if let Some((non_null_rows, name)) = - take_optimizable_column_and_table_count(&**expr, &stats) + take_optimizable_column_and_table_count(expr, &stats) { projections.push((expressions::lit(non_null_rows), name.to_owned())); - } else if let Some((min, name)) = take_optimizable_min(&**expr, &stats) { + } else if let Some((min, name)) = take_optimizable_min(expr, &stats) { projections.push((expressions::lit(min), name.to_owned())); - } else if let Some((max, name)) = take_optimizable_max(&**expr, &stats) { + } else if let Some((max, name)) = take_optimizable_max(expr, &stats) { projections.push((expressions::lit(max), name.to_owned())); } else { // TODO: we need all aggr_expr to be resolved (cf TODO fullres) @@ -137,7 +137,7 @@ fn take_optimizable(node: &dyn ExecutionPlan) -> Option> /// If this agg_expr is a count that can be exactly derived from the statistics, return it. fn take_optimizable_column_and_table_count( - agg_expr: &dyn AggregateExpr, + agg_expr: &AggregateFunctionExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { let col_stats = &stats.column_statistics; @@ -174,7 +174,7 @@ fn take_optimizable_column_and_table_count( /// If this agg_expr is a min that is exactly defined in the statistics, return it. fn take_optimizable_min( - agg_expr: &dyn AggregateExpr, + agg_expr: &AggregateFunctionExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { if let Precision::Exact(num_rows) = &stats.num_rows { @@ -220,7 +220,7 @@ fn take_optimizable_min( /// If this agg_expr is a max that is exactly defined in the statistics, return it. fn take_optimizable_max( - agg_expr: &dyn AggregateExpr, + agg_expr: &AggregateFunctionExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { if let Precision::Exact(num_rows) = &stats.num_rows { @@ -266,33 +266,27 @@ fn take_optimizable_max( // TODO: Move this check into AggregateUDFImpl // https://github.com/apache/datafusion/issues/11153 -fn is_non_distinct_count(agg_expr: &dyn AggregateExpr) -> bool { - if let Some(agg_expr) = agg_expr.as_any().downcast_ref::() { - if agg_expr.fun().name() == "count" && !agg_expr.is_distinct() { - return true; - } +fn is_non_distinct_count(agg_expr: &AggregateFunctionExpr) -> bool { + if agg_expr.fun().name() == "count" && !agg_expr.is_distinct() { + return true; } false } // TODO: Move this check into AggregateUDFImpl // https://github.com/apache/datafusion/issues/11153 -fn is_min(agg_expr: &dyn AggregateExpr) -> bool { - if let Some(agg_expr) = agg_expr.as_any().downcast_ref::() { - if agg_expr.fun().name().to_lowercase() == "min" { - return true; - } +fn is_min(agg_expr: &AggregateFunctionExpr) -> bool { + if agg_expr.fun().name().to_lowercase() == "min" { + return true; } false } // TODO: Move this check into AggregateUDFImpl // https://github.com/apache/datafusion/issues/11153 -fn is_max(agg_expr: &dyn AggregateExpr) -> bool { - if let Some(agg_expr) = agg_expr.as_any().downcast_ref::() { - if agg_expr.fun().name().to_lowercase() == "max" { - return true; - } +fn is_max(agg_expr: &AggregateFunctionExpr) -> bool { + if agg_expr.fun().name().to_lowercase() == "max" { + return true; } false } diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 78da4dc9c53f..24387c5f15ee 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -55,7 +55,6 @@ datafusion-functions-aggregate = { workspace = true } datafusion-functions-aggregate-common = { workspace = true } datafusion-physical-expr = { workspace = true, default-features = true } datafusion-physical-expr-common = { workspace = true } -datafusion-physical-expr-functions-aggregate = { workspace = true } futures = { workspace = true } half = { workspace = true } hashbrown = { workspace = true } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 5aa255e7c341..0aeaa15b5f1d 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -42,10 +42,11 @@ use datafusion_expr::Accumulator; use datafusion_physical_expr::{ equivalence::{collapse_lex_req, ProjectionMapping}, expressions::{Column, UnKnownColumn}, - physical_exprs_contains, AggregateExpr, EquivalenceProperties, LexOrdering, - LexRequirement, PhysicalExpr, PhysicalSortRequirement, + physical_exprs_contains, EquivalenceProperties, LexOrdering, LexRequirement, + PhysicalExpr, PhysicalSortRequirement, }; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use itertools::Itertools; pub mod group_values; @@ -253,7 +254,7 @@ pub struct AggregateExec { /// Group by expressions group_by: PhysicalGroupBy, /// Aggregate expressions - aggr_expr: Vec>, + aggr_expr: Vec>, /// FILTER (WHERE clause) expression for each aggregate expression filter_expr: Vec>>, /// Set if the output of this aggregation is truncated by a upstream sort/limit clause @@ -280,7 +281,10 @@ impl AggregateExec { /// Function used in `ConvertFirstLast` optimizer rule, /// where we need parts of the new value, others cloned from the old one /// Rewrites aggregate exec with new aggregate expressions. - pub fn with_new_aggr_exprs(&self, aggr_expr: Vec>) -> Self { + pub fn with_new_aggr_exprs( + &self, + aggr_expr: Vec>, + ) -> Self { Self { aggr_expr, // clone the rest of the fields @@ -306,7 +310,7 @@ impl AggregateExec { pub fn try_new( mode: AggregateMode, group_by: PhysicalGroupBy, - aggr_expr: Vec>, + aggr_expr: Vec>, filter_expr: Vec>>, input: Arc, input_schema: SchemaRef, @@ -343,7 +347,7 @@ impl AggregateExec { fn try_new_with_schema( mode: AggregateMode, group_by: PhysicalGroupBy, - mut aggr_expr: Vec>, + mut aggr_expr: Vec>, filter_expr: Vec>>, input: Arc, input_schema: SchemaRef, @@ -451,7 +455,7 @@ impl AggregateExec { } /// Aggregate expressions - pub fn aggr_expr(&self) -> &[Arc] { + pub fn aggr_expr(&self) -> &[Arc] { &self.aggr_expr } @@ -788,7 +792,7 @@ impl ExecutionPlan for AggregateExec { fn create_schema( input_schema: &Schema, group_expr: &[(Arc, String)], - aggr_expr: &[Arc], + aggr_expr: &[Arc], contains_null_expr: bool, mode: AggregateMode, ) -> Result { @@ -834,7 +838,7 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { /// /// # Parameters /// -/// - `aggr_expr`: A reference to an `Arc` representing the +/// - `aggr_expr`: A reference to an `Arc` representing the /// aggregate expression. /// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the /// physical GROUP BY expression. @@ -846,7 +850,7 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { /// A `LexOrdering` instance indicating the lexical ordering requirement for /// the aggregate expression. fn get_aggregate_expr_req( - aggr_expr: &Arc, + aggr_expr: &Arc, group_by: &PhysicalGroupBy, agg_mode: &AggregateMode, ) -> LexOrdering { @@ -894,7 +898,7 @@ fn get_aggregate_expr_req( /// the aggregator requirement is incompatible. fn finer_ordering( existing_req: &LexOrdering, - aggr_expr: &Arc, + aggr_expr: &Arc, group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, @@ -912,7 +916,7 @@ pub fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { /// /// # Parameters /// -/// - `aggr_exprs`: A slice of `Arc` containing all the +/// - `aggr_exprs`: A slice of `Arc` containing all the /// aggregate expressions. /// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the /// physical GROUP BY expression. @@ -926,7 +930,7 @@ pub fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { /// A `LexRequirement` instance, which is the requirement that satisfies all the /// aggregate requirements. Returns an error in case of conflicting requirements. pub fn get_finer_aggregate_exprs_requirement( - aggr_exprs: &mut [Arc], + aggr_exprs: &mut [Arc], group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, @@ -996,10 +1000,10 @@ pub fn get_finer_aggregate_exprs_requirement( /// returns physical expressions for arguments to evaluate against a batch /// The expressions are different depending on `mode`: -/// * Partial: AggregateExpr::expressions -/// * Final: columns of `AggregateExpr::state_fields()` +/// * Partial: AggregateFunctionExpr::expressions +/// * Final: columns of `AggregateFunctionExpr::state_fields()` pub fn aggregate_expressions( - aggr_expr: &[Arc], + aggr_expr: &[Arc], mode: &AggregateMode, col_idx_base: usize, ) -> Result>>> { @@ -1035,12 +1039,12 @@ pub fn aggregate_expressions( } /// uses `state_fields` to build a vec of physical column expressions required to merge the -/// AggregateExpr' accumulator's state. +/// AggregateFunctionExpr' accumulator's state. /// /// `index_base` is the starting physical column index for the next expanded state field. fn merge_expressions( index_base: usize, - expr: &Arc, + expr: &Arc, ) -> Result>> { expr.state_fields().map(|fields| { fields @@ -1054,7 +1058,7 @@ fn merge_expressions( pub type AccumulatorItem = Box; pub fn create_accumulators( - aggr_expr: &[Arc], + aggr_expr: &[Arc], ) -> Result> { aggr_expr .iter() @@ -1218,8 +1222,8 @@ mod tests { use datafusion_physical_expr::PhysicalSortExpr; use crate::common::collect; + use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::Literal; - use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; use futures::{FutureExt, Stream}; // Generate a schema which consists of 5 columns (a, b, c, d, e) @@ -1496,13 +1500,12 @@ mod tests { groups: vec![vec![false]], }; - let aggregates: Vec> = - vec![ - AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) - .schema(Arc::clone(&input_schema)) - .alias("AVG(b)") - .build()?, - ]; + let aggregates: Vec> = vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) + .schema(Arc::clone(&input_schema)) + .alias("AVG(b)") + .build()?, + ]; let task_ctx = if spill { // set to an appropriate value to trigger spill @@ -1793,7 +1796,7 @@ mod tests { } // Median(a) - fn test_median_agg_expr(schema: SchemaRef) -> Result> { + fn test_median_agg_expr(schema: SchemaRef) -> Result> { AggregateExprBuilder::new(median_udaf(), vec![col("a", &schema)?]) .schema(schema) .alias("MEDIAN(a)") @@ -1819,17 +1822,16 @@ mod tests { }; // something that allocates within the aggregator - let aggregates_v0: Vec> = + let aggregates_v0: Vec> = vec![test_median_agg_expr(Arc::clone(&input_schema))?]; // use fast-path in `row_hash.rs`. - let aggregates_v2: Vec> = - vec![ - AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) - .schema(Arc::clone(&input_schema)) - .alias("AVG(b)") - .build()?, - ]; + let aggregates_v2: Vec> = vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) + .schema(Arc::clone(&input_schema)) + .alias("AVG(b)") + .build()?, + ]; for (version, groups, aggregates) in [ (0, groups_none, aggregates_v0), @@ -1883,13 +1885,12 @@ mod tests { let groups = PhysicalGroupBy::default(); - let aggregates: Vec> = - vec![ - AggregateExprBuilder::new(avg_udaf(), vec![col("a", &schema)?]) - .schema(Arc::clone(&schema)) - .alias("AVG(a)") - .build()?, - ]; + let aggregates: Vec> = vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("a", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("AVG(a)") + .build()?, + ]; let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let refs = blocking_exec.refs(); @@ -1923,13 +1924,12 @@ mod tests { let groups = PhysicalGroupBy::new_single(vec![(col("a", &schema)?, "a".to_string())]); - let aggregates: Vec> = - vec![ - AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) - .schema(Arc::clone(&schema)) - .alias("AVG(b)") - .build()?, - ]; + let aggregates: Vec> = vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("AVG(b)") + .build()?, + ]; let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let refs = blocking_exec.refs(); @@ -1974,7 +1974,7 @@ mod tests { fn test_first_value_agg_expr( schema: &Schema, sort_options: SortOptions, - ) -> Result> { + ) -> Result> { let ordering_req = [PhysicalSortExpr { expr: col("b", schema)?, options: sort_options, @@ -1992,7 +1992,7 @@ mod tests { fn test_last_value_agg_expr( schema: &Schema, sort_options: SortOptions, - ) -> Result> { + ) -> Result> { let ordering_req = [PhysicalSortExpr { expr: col("b", schema)?, options: sort_options, @@ -2047,7 +2047,7 @@ mod tests { descending: false, nulls_first: false, }; - let aggregates: Vec> = if is_first_acc { + let aggregates: Vec> = if is_first_acc { vec![test_first_value_agg_expr(&schema, sort_options)?] } else { vec![test_last_value_agg_expr(&schema, sort_options)?] @@ -2212,7 +2212,7 @@ mod tests { }; let groups = PhysicalGroupBy::new_single(vec![(col_a, "a".to_string())]); - let aggregates: Vec> = vec![ + let aggregates: Vec> = vec![ test_first_value_agg_expr(&schema, option_desc)?, test_last_value_agg_expr(&schema, option_desc)?, ]; @@ -2270,7 +2270,7 @@ mod tests { ], ); - let aggregates: Vec> = + let aggregates: Vec> = vec![AggregateExprBuilder::new(count_udaf(), vec![lit(1)]) .schema(Arc::clone(&schema)) .alias("1") diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 05f4ec621813..d022bb007d9b 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -47,10 +47,9 @@ use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::{ - AggregateExpr, GroupsAccumulatorAdapter, PhysicalSortExpr, -}; +use datafusion_physical_expr::{GroupsAccumulatorAdapter, PhysicalSortExpr}; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use futures::ready; use futures::stream::{Stream, StreamExt}; use log::debug; @@ -396,7 +395,7 @@ pub(crate) struct GroupedHashAggregateStream { /// processed. Reused across batches here to avoid reallocations current_group_indices: Vec, - /// Accumulators, one for each `AggregateExpr` in the query + /// Accumulators, one for each `AggregateFunctionExpr` in the query /// /// For example, if the query has aggregates, `SUM(x)`, /// `COUNT(y)`, there will be two accumulators, each one @@ -579,7 +578,7 @@ impl GroupedHashAggregateStream { /// that is supported by the aggregate, or a /// [`GroupsAccumulatorAdapter`] if not. pub(crate) fn create_group_accumulator( - agg_expr: &Arc, + agg_expr: &Arc, ) -> Result> { if agg_expr.groups_accumulator_supported() { agg_expr.create_groups_accumulator() diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index e1182719293d..c1c66f6d3923 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -34,7 +34,7 @@ pub use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; pub use datafusion_expr::{Accumulator, ColumnarValue}; pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{ - expressions, functions, udf, AggregateExpr, Distribution, Partitioning, PhysicalExpr, + expressions, functions, udf, Distribution, Partitioning, PhysicalExpr, }; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexRequirement; diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index fb86a008e2cd..026798c5798b 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -31,7 +31,7 @@ pub use datafusion_expr::{Accumulator, ColumnarValue}; pub use datafusion_physical_expr::window::WindowExpr; use datafusion_physical_expr::PhysicalSortExpr; pub use datafusion_physical_expr::{ - expressions, functions, udf, AggregateExpr, Distribution, Partitioning, PhysicalExpr, + expressions, functions, udf, Distribution, Partitioning, PhysicalExpr, }; pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; @@ -82,7 +82,7 @@ pub mod windows; pub mod work_table; pub mod udaf { - pub use datafusion_physical_expr_functions_aggregate::aggregate::AggregateFunctionExpr; + pub use datafusion_physical_expr::aggregate::AggregateFunctionExpr; } pub mod coalesce; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index d607bb79b44e..56823e6dec2d 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -37,14 +37,13 @@ use datafusion_expr::{ BuiltInWindowFunction, PartitionEvaluator, WindowFrame, WindowFunctionDefinition, WindowUDF, }; +use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ reverse_order_bys, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, - AggregateExpr, ConstExpr, EquivalenceProperties, LexOrdering, - PhysicalSortRequirement, + ConstExpr, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, }; -use datafusion_physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; use itertools::Itertools; mod bounded_window_agg_exec; @@ -142,7 +141,7 @@ fn window_expr_from_aggregate_expr( partition_by: &[Arc], order_by: &[PhysicalSortExpr], window_frame: Arc, - aggregate: Arc, + aggregate: Arc, ) -> Arc { // Is there a potentially unlimited sized window frame? let unbounded_window = window_frame.start_bound.is_unbounded(); diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 96fb45eafe62..78f370c714cc 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -18,7 +18,7 @@ use std::fmt::Debug; use std::sync::Arc; -use datafusion::physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; +use datafusion::physical_expr::aggregate::AggregateExprBuilder; use prost::bytes::BufMut; use prost::Message; @@ -34,6 +34,7 @@ use datafusion::datasource::physical_plan::ParquetExec; use datafusion::datasource::physical_plan::{AvroExec, CsvExec}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; +use datafusion::physical_expr::aggregate::AggregateFunctionExpr; use datafusion::physical_expr::{PhysicalExprRef, PhysicalSortRequirement}; use datafusion::physical_plan::aggregates::AggregateMode; use datafusion::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; @@ -59,7 +60,7 @@ use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMerge use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; use datafusion::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use datafusion::physical_plan::{ - AggregateExpr, ExecutionPlan, InputOrderMode, PhysicalExpr, WindowExpr, + ExecutionPlan, InputOrderMode, PhysicalExpr, WindowExpr, }; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::{AggregateUDF, ScalarUDF}; @@ -467,7 +468,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { }) .collect::, _>>()?; - let physical_aggr_expr: Vec> = hash_agg + let physical_aggr_expr: Vec> = hash_agg .aggr_expr .iter() .zip(hash_agg.aggr_expr_name.iter()) diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 7949a457f40f..555ad22a9bc1 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -29,7 +29,7 @@ use datafusion::physical_plan::expressions::{ }; use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; -use datafusion::physical_plan::{AggregateExpr, Partitioning, PhysicalExpr, WindowExpr}; +use datafusion::physical_plan::{Partitioning, PhysicalExpr, WindowExpr}; use datafusion::{ datasource::{ file_format::{csv::CsvSink, json::JsonSink}, @@ -49,58 +49,50 @@ use crate::protobuf::{ use super::PhysicalExtensionCodec; pub fn serialize_physical_aggr_expr( - aggr_expr: Arc, + aggr_expr: Arc, codec: &dyn PhysicalExtensionCodec, ) -> Result { let expressions = serialize_physical_exprs(aggr_expr.expressions(), codec)?; let ordering_req = aggr_expr.order_bys().unwrap_or(&[]).to_vec(); let ordering_req = serialize_physical_sort_exprs(ordering_req, codec)?; - if let Some(a) = aggr_expr.as_any().downcast_ref::() { - let name = a.fun().name().to_string(); - let mut buf = Vec::new(); - codec.try_encode_udaf(a.fun(), &mut buf)?; - Ok(protobuf::PhysicalExprNode { - expr_type: Some(protobuf::physical_expr_node::ExprType::AggregateExpr( - protobuf::PhysicalAggregateExprNode { - aggregate_function: Some(physical_aggregate_expr_node::AggregateFunction::UserDefinedAggrFunction(name)), - expr: expressions, - ordering_req, - distinct: a.is_distinct(), - ignore_nulls: a.ignore_nulls(), - fun_definition: (!buf.is_empty()).then_some(buf) - }, - )), - }) - } else { - unreachable!("No other types exists besides AggergationFunctionExpr"); - } + let name = aggr_expr.fun().name().to_string(); + let mut buf = Vec::new(); + codec.try_encode_udaf(aggr_expr.fun(), &mut buf)?; + Ok(protobuf::PhysicalExprNode { + expr_type: Some(protobuf::physical_expr_node::ExprType::AggregateExpr( + protobuf::PhysicalAggregateExprNode { + aggregate_function: Some(physical_aggregate_expr_node::AggregateFunction::UserDefinedAggrFunction(name)), + expr: expressions, + ordering_req, + distinct: aggr_expr.is_distinct(), + ignore_nulls: aggr_expr.ignore_nulls(), + fun_definition: (!buf.is_empty()).then_some(buf) + }, + )), + }) } fn serialize_physical_window_aggr_expr( - aggr_expr: &dyn AggregateExpr, + aggr_expr: &AggregateFunctionExpr, _window_frame: &WindowFrame, codec: &dyn PhysicalExtensionCodec, ) -> Result<(physical_window_expr_node::WindowFunction, Option>)> { - if let Some(a) = aggr_expr.as_any().downcast_ref::() { - if a.is_distinct() || a.ignore_nulls() { - // TODO - return not_impl_err!( - "Distinct aggregate functions not supported in window expressions" - ); - } - - let mut buf = Vec::new(); - codec.try_encode_udaf(a.fun(), &mut buf)?; - Ok(( - physical_window_expr_node::WindowFunction::UserDefinedAggrFunction( - a.fun().name().to_string(), - ), - (!buf.is_empty()).then_some(buf), - )) - } else { - unreachable!("No other types exists besides AggergationFunctionExpr"); + if aggr_expr.is_distinct() || aggr_expr.ignore_nulls() { + // TODO + return not_impl_err!( + "Distinct aggregate functions not supported in window expressions" + ); } + + let mut buf = Vec::new(); + codec.try_encode_udaf(aggr_expr.fun(), &mut buf)?; + Ok(( + physical_window_expr_node::WindowFunction::UserDefinedAggrFunction( + aggr_expr.fun().name().to_string(), + ), + (!buf.is_empty()).then_some(buf), + )) } pub fn serialize_physical_window_expr( diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 0ffc494321fb..60f5565bdeee 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -24,7 +24,7 @@ use std::vec; use arrow::array::RecordBatch; use arrow::csv::WriterBuilder; -use datafusion::physical_expr_functions_aggregate::aggregate::AggregateExprBuilder; +use datafusion::physical_expr::aggregate::AggregateExprBuilder; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_functions_aggregate::approx_percentile_cont::approx_percentile_cont_udaf; use datafusion_functions_aggregate::array_agg::array_agg_udaf; @@ -47,7 +47,6 @@ use datafusion::datasource::physical_plan::{ use datafusion::execution::FunctionRegistry; use datafusion::functions_aggregate::sum::sum_udaf; use datafusion::logical_expr::{create_udf, JoinType, Operator, Volatility}; -use datafusion::physical_expr::aggregate::utils::down_cast_any_ref; use datafusion::physical_expr::expressions::Literal; use datafusion::physical_expr::window::SlidingAggregateWindowExpr; use datafusion::physical_expr::{PhysicalSortRequirement, ScalarFunctionExpr}; @@ -70,13 +69,12 @@ use datafusion::physical_plan::placeholder_row::PlaceholderRowExec; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; +use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; use datafusion::physical_plan::windows::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowAggExec, }; -use datafusion::physical_plan::{ - AggregateExpr, ExecutionPlan, Partitioning, PhysicalExpr, Statistics, -}; +use datafusion::physical_plan::{ExecutionPlan, Partitioning, PhysicalExpr, Statistics}; use datafusion::prelude::SessionContext; use datafusion::scalar::ScalarValue; use datafusion_common::config::TableParquetOptions; @@ -362,7 +360,7 @@ fn rountrip_aggregate() -> Result<()> { .alias("NTH_VALUE(b, 1)") .build()?; - let test_cases: Vec>> = vec![ + let test_cases: Vec>> = vec![ // AVG vec![avg_expr], // NTH_VALUE @@ -395,7 +393,7 @@ fn rountrip_aggregate_with_limit() -> Result<()> { let groups: Vec<(Arc, String)> = vec![(col("a", &schema)?, "unused".to_string())]; - let aggregates: Vec> = + let aggregates: Vec> = vec![ AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) @@ -424,7 +422,7 @@ fn rountrip_aggregate_with_approx_pencentile_cont() -> Result<()> { let groups: Vec<(Arc, String)> = vec![(col("a", &schema)?, "unused".to_string())]; - let aggregates: Vec> = vec![AggregateExprBuilder::new( + let aggregates: Vec> = vec![AggregateExprBuilder::new( approx_percentile_cont_udaf(), vec![col("b", &schema)?, lit(0.5)], ) @@ -459,7 +457,7 @@ fn rountrip_aggregate_with_sort() -> Result<()> { }, }]; - let aggregates: Vec> = + let aggregates: Vec> = vec![ AggregateExprBuilder::new(array_agg_udaf(), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) @@ -526,7 +524,7 @@ fn roundtrip_aggregate_udaf() -> Result<()> { let groups: Vec<(Arc, String)> = vec![(col("a", &schema)?, "unused".to_string())]; - let aggregates: Vec> = + let aggregates: Vec> = vec![ AggregateExprBuilder::new(Arc::new(udaf), vec![col("b", &schema)?]) .schema(Arc::clone(&schema)) @@ -748,7 +746,7 @@ fn roundtrip_parquet_exec_with_custom_predicate_expr() -> Result<()> { } impl PartialEq for CustomPredicateExpr { fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) + other .downcast_ref::() .map(|x| self.inner.eq(&x.inner)) .unwrap_or(false) From 58ed702c8f7f2536a8f71cf8b7486812a3afa42d Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 15:17:23 +0200 Subject: [PATCH 13/75] Deduplicate sort unparsing logic (#12175) * Deduplicate sort unparsing logic Reconstruction of `ast::OrderByExpr` was implemented twice, in `plan.rs` and `expr.rs` submodules of the unparser. * empty --- datafusion/sql/src/unparser/plan.rs | 22 ++++------------------ 1 file changed, 4 insertions(+), 18 deletions(-) diff --git a/datafusion/sql/src/unparser/plan.rs b/datafusion/sql/src/unparser/plan.rs index 8b5a5b0942b8..106705c322fc 100644 --- a/datafusion/sql/src/unparser/plan.rs +++ b/datafusion/sql/src/unparser/plan.rs @@ -528,24 +528,10 @@ impl Unparser<'_> { fn sort_to_sql(&self, sort_exprs: Vec) -> Result> { sort_exprs .iter() - .map(|expr: &Expr| match expr { - Expr::Sort(sort_expr) => { - let col = self.expr_to_sql(&sort_expr.expr)?; - - let nulls_first = if self.dialect.supports_nulls_first_in_sort() { - Some(sort_expr.nulls_first) - } else { - None - }; - - Ok(ast::OrderByExpr { - asc: Some(sort_expr.asc), - expr: col, - nulls_first, - with_fill: None, - }) - } - _ => plan_err!("Expecting Sort expr"), + .map(|expr: &Expr| { + self.expr_to_unparsed(expr)? + .into_order_by_expr() + .or(plan_err!("Expecting Sort expr")) }) .collect::>>() } From 4a943560e1a24108aeb915be260db7c4264c9b11 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 15:22:44 +0200 Subject: [PATCH 14/75] Require sort expressions to be of type Sort (#12171) * Require sort expressions to be of type Sort in compare_sort_expr * fixup! Require sort expressions to be of type Sort in compare_sort_expr * fixup! Require sort expressions to be of type Sort in compare_sort_expr --- datafusion/expr/src/utils.rs | 2 +- datafusion/sql/src/expr/function.rs | 4 +++- datafusion/sql/src/unparser/rewrite.rs | 2 ++ 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 8f04d3ec3067..a01d5ef8973a 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -569,7 +569,7 @@ pub fn compare_sort_expr( } Ordering::Equal } - _ => Ordering::Equal, + _ => panic!("Sort expressions must be of type Sort"), } } diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index b95414a8cafd..72e08e4b8fb5 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -295,8 +295,10 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { Some(false) }; } + Some(false) + } else { + panic!("order_by expression must be of type Sort"); } - Some(false) }); let window_frame = window diff --git a/datafusion/sql/src/unparser/rewrite.rs b/datafusion/sql/src/unparser/rewrite.rs index fed3f2c39f9d..9e1adcf4df31 100644 --- a/datafusion/sql/src/unparser/rewrite.rs +++ b/datafusion/sql/src/unparser/rewrite.rs @@ -161,6 +161,8 @@ pub(super) fn rewrite_plan_for_sort_on_non_projected_fields( for expr in &sort.expr { if let Expr::Sort(s) = expr { collects.push(s.expr.as_ref().clone()); + } else { + panic!("sort expression must be of type Sort"); } } From a616e8895bdde48153d70ba32281d5174d172cea Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 15:24:11 +0200 Subject: [PATCH 15/75] Remove `parse_vec_expr` helper (#12178) * Remove `parse_vec_expr` helper Behavior of `parse_vec_expr` and `parse_exprs` is almost similar -- both take a collection of expressions to parse. The only difference is that `parse_vec_expr` returns `Option::None` when collections is empty, but this difference in behavior does not correspond to difference in function names. Since the function is used once only, remove it instead of coming up with a fancy name. * empty --- datafusion/proto/src/logical_plan/from_proto.rs | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index b74237b5281b..acda1298dd80 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -586,7 +586,10 @@ pub fn parse_expr( parse_exprs(&pb.args, registry, codec)?, pb.distinct, parse_optional_expr(pb.filter.as_deref(), registry, codec)?.map(Box::new), - parse_vec_expr(&pb.order_by, registry, codec)?, + match pb.order_by.len() { + 0 => None, + _ => Some(parse_exprs(&pb.order_by, registry, codec)?), + }, None, ))) } @@ -676,16 +679,6 @@ pub fn from_proto_binary_op(op: &str) -> Result { } } -fn parse_vec_expr( - p: &[protobuf::LogicalExprNode], - registry: &dyn FunctionRegistry, - codec: &dyn LogicalExtensionCodec, -) -> Result>, Error> { - let res = parse_exprs(p, registry, codec)?; - // Convert empty vector to None. - Ok((!res.is_empty()).then_some(res)) -} - fn parse_optional_expr( p: Option<&protobuf::LogicalExprNode>, registry: &dyn FunctionRegistry, From bf46f33f0746598c69ad6651c11f36842c5e71a2 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 15:25:23 +0200 Subject: [PATCH 16/75] Reuse bulk serialization helpers for protobuf (#12179) Reuse `parse_exprs` and `serialize_exprs`. Reduce code duplication, increasing readability. --- datafusion/proto/src/logical_plan/mod.rs | 205 +++++++---------------- 1 file changed, 60 insertions(+), 145 deletions(-) diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index bc019725f36c..67977b1795a6 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -29,7 +29,7 @@ use crate::{ }, }; -use crate::protobuf::{proto_error, FromProtoError, ToProtoError}; +use crate::protobuf::{proto_error, ToProtoError}; use arrow::datatypes::{DataType, Schema, SchemaRef}; #[cfg(feature = "parquet")] use datafusion::datasource::file_format::parquet::ParquetFormat; @@ -66,11 +66,10 @@ use datafusion_expr::{ }; use datafusion_expr::{AggregateUDF, Unnest}; +use self::to_proto::{serialize_expr, serialize_exprs}; use prost::bytes::BufMut; use prost::Message; -use self::to_proto::serialize_expr; - pub mod file_formats; pub mod from_proto; pub mod to_proto; @@ -273,13 +272,7 @@ impl AsLogicalPlan for LogicalPlanNode { values .values_list .chunks_exact(n_cols) - .map(|r| { - r.iter() - .map(|expr| { - from_proto::parse_expr(expr, ctx, extension_codec) - }) - .collect::, FromProtoError>>() - }) + .map(|r| from_proto::parse_exprs(r, ctx, extension_codec)) .collect::, _>>() .map_err(|e| e.into()) }?; @@ -288,11 +281,8 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanType::Projection(projection) => { let input: LogicalPlan = into_logical_plan!(projection.input, ctx, extension_codec)?; - let expr: Vec = projection - .expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let expr: Vec = + from_proto::parse_exprs(&projection.expr, ctx, extension_codec)?; let new_proj = project(input, expr)?; match projection.optional_alias.as_ref() { @@ -324,26 +314,17 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanType::Window(window) => { let input: LogicalPlan = into_logical_plan!(window.input, ctx, extension_codec)?; - let window_expr = window - .window_expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let window_expr = + from_proto::parse_exprs(&window.window_expr, ctx, extension_codec)?; LogicalPlanBuilder::from(input).window(window_expr)?.build() } LogicalPlanType::Aggregate(aggregate) => { let input: LogicalPlan = into_logical_plan!(aggregate.input, ctx, extension_codec)?; - let group_expr = aggregate - .group_expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; - let aggr_expr = aggregate - .aggr_expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let group_expr = + from_proto::parse_exprs(&aggregate.group_expr, ctx, extension_codec)?; + let aggr_expr = + from_proto::parse_exprs(&aggregate.aggr_expr, ctx, extension_codec)?; LogicalPlanBuilder::from(input) .aggregate(group_expr, aggr_expr)? .build() @@ -361,20 +342,16 @@ impl AsLogicalPlan for LogicalPlanNode { projection = Some(column_indices); } - let filters = scan - .filters - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let filters = + from_proto::parse_exprs(&scan.filters, ctx, extension_codec)?; let mut all_sort_orders = vec![]; for order in &scan.file_sort_order { - let file_sort_order = order - .logical_expr_nodes - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; - all_sort_orders.push(file_sort_order) + all_sort_orders.push(from_proto::parse_exprs( + &order.logical_expr_nodes, + ctx, + extension_codec, + )?) } let file_format: Arc = @@ -475,11 +452,8 @@ impl AsLogicalPlan for LogicalPlanNode { projection = Some(column_indices); } - let filters = scan - .filters - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let filters = + from_proto::parse_exprs(&scan.filters, ctx, extension_codec)?; let table_name = from_table_reference(scan.table_name.as_ref(), "CustomScan")?; @@ -502,11 +476,8 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanType::Sort(sort) => { let input: LogicalPlan = into_logical_plan!(sort.input, ctx, extension_codec)?; - let sort_expr: Vec = sort - .expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let sort_expr: Vec = + from_proto::parse_exprs(&sort.expr, ctx, extension_codec)?; LogicalPlanBuilder::from(input).sort(sort_expr)?.build() } LogicalPlanType::Repartition(repartition) => { @@ -525,12 +496,7 @@ impl AsLogicalPlan for LogicalPlanNode { hash_expr: pb_hash_expr, partition_count, }) => Partitioning::Hash( - pb_hash_expr - .iter() - .map(|expr| { - from_proto::parse_expr(expr, ctx, extension_codec) - }) - .collect::, _>>()?, + from_proto::parse_exprs(pb_hash_expr, ctx, extension_codec)?, *partition_count as usize, ), PartitionMethod::RoundRobin(partition_count) => { @@ -570,12 +536,11 @@ impl AsLogicalPlan for LogicalPlanNode { let mut order_exprs = vec![]; for expr in &create_extern_table.order_exprs { - let order_expr = expr - .logical_expr_nodes - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; - order_exprs.push(order_expr) + order_exprs.push(from_proto::parse_exprs( + &expr.logical_expr_nodes, + ctx, + extension_codec, + )?); } let mut column_defaults = @@ -693,16 +658,10 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanBuilder::from(input).limit(skip, fetch)?.build() } LogicalPlanType::Join(join) => { - let left_keys: Vec = join - .left_join_key - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; - let right_keys: Vec = join - .right_join_key - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let left_keys: Vec = + from_proto::parse_exprs(&join.left_join_key, ctx, extension_codec)?; + let right_keys: Vec = + from_proto::parse_exprs(&join.right_join_key, ctx, extension_codec)?; let join_type = protobuf::JoinType::try_from(join.join_type).map_err(|_| { proto_error(format!( @@ -804,27 +763,20 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanType::DistinctOn(distinct_on) => { let input: LogicalPlan = into_logical_plan!(distinct_on.input, ctx, extension_codec)?; - let on_expr = distinct_on - .on_expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; - let select_expr = distinct_on - .select_expr - .iter() - .map(|expr| from_proto::parse_expr(expr, ctx, extension_codec)) - .collect::, _>>()?; + let on_expr = + from_proto::parse_exprs(&distinct_on.on_expr, ctx, extension_codec)?; + let select_expr = from_proto::parse_exprs( + &distinct_on.select_expr, + ctx, + extension_codec, + )?; let sort_expr = match distinct_on.sort_expr.len() { 0 => None, - _ => Some( - distinct_on - .sort_expr - .iter() - .map(|expr| { - from_proto::parse_expr(expr, ctx, extension_codec) - }) - .collect::, _>>()?, - ), + _ => Some(from_proto::parse_exprs( + &distinct_on.sort_expr, + ctx, + extension_codec, + )?), }; LogicalPlanBuilder::from(input) .distinct_on(on_expr, select_expr, sort_expr)? @@ -943,11 +895,8 @@ impl AsLogicalPlan for LogicalPlanNode { } else { values[0].len() } as u64; - let values_list = values - .iter() - .flatten() - .map(|v| serialize_expr(v, extension_codec)) - .collect::, _>>()?; + let values_list = + serialize_exprs(values.iter().flatten(), extension_codec)?; Ok(protobuf::LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Values( protobuf::ValuesNode { @@ -982,10 +931,8 @@ impl AsLogicalPlan for LogicalPlanNode { }; let schema: protobuf::Schema = schema.as_ref().try_into()?; - let filters: Vec = filters - .iter() - .map(|filter| serialize_expr(filter, extension_codec)) - .collect::, _>>()?; + let filters: Vec = + serialize_exprs(filters, extension_codec)?; if let Some(listing_table) = source.downcast_ref::() { let any = listing_table.options().format.as_any(); @@ -1037,10 +984,7 @@ impl AsLogicalPlan for LogicalPlanNode { let mut exprs_vec: Vec = vec![]; for order in &options.file_sort_order { let expr_vec = LogicalExprNodeCollection { - logical_expr_nodes: order - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, ToProtoError>>()?, + logical_expr_nodes: serialize_exprs(order, extension_codec)?, }; exprs_vec.push(expr_vec); } @@ -1118,10 +1062,7 @@ impl AsLogicalPlan for LogicalPlanNode { extension_codec, )?, )), - expr: expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, ToProtoError>>()?, + expr: serialize_exprs(expr, extension_codec)?, optional_alias: None, }, ))), @@ -1173,22 +1114,13 @@ impl AsLogicalPlan for LogicalPlanNode { )?; let sort_expr = match sort_expr { None => vec![], - Some(sort_expr) => sort_expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, _>>()?, + Some(sort_expr) => serialize_exprs(sort_expr, extension_codec)?, }; Ok(protobuf::LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::DistinctOn(Box::new( protobuf::DistinctOnNode { - on_expr: on_expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, _>>()?, - select_expr: select_expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, _>>()?, + on_expr: serialize_exprs(on_expr, extension_codec)?, + select_expr: serialize_exprs(select_expr, extension_codec)?, sort_expr, input: Some(Box::new(input)), }, @@ -1207,10 +1139,7 @@ impl AsLogicalPlan for LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Window(Box::new( protobuf::WindowNode { input: Some(Box::new(input)), - window_expr: window_expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, _>>()?, + window_expr: serialize_exprs(window_expr, extension_codec)?, }, ))), }) @@ -1230,14 +1159,8 @@ impl AsLogicalPlan for LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Aggregate(Box::new( protobuf::AggregateNode { input: Some(Box::new(input)), - group_expr: group_expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, _>>()?, - aggr_expr: aggr_expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, _>>()?, + group_expr: serialize_exprs(group_expr, extension_codec)?, + aggr_expr: serialize_exprs(aggr_expr, extension_codec)?, }, ))), }) @@ -1335,10 +1258,8 @@ impl AsLogicalPlan for LogicalPlanNode { input.as_ref(), extension_codec, )?; - let selection_expr: Vec = expr - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, ToProtoError>>()?; + let selection_expr: Vec = + serialize_exprs(expr, extension_codec)?; Ok(protobuf::LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Sort(Box::new( protobuf::SortNode { @@ -1367,10 +1288,7 @@ impl AsLogicalPlan for LogicalPlanNode { let pb_partition_method = match partitioning_scheme { Partitioning::Hash(exprs, partition_count) => { PartitionMethod::Hash(protobuf::HashRepartition { - hash_expr: exprs - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, ToProtoError>>()?, + hash_expr: serialize_exprs(exprs, extension_codec)?, partition_count: *partition_count as u64, }) } @@ -1419,10 +1337,7 @@ impl AsLogicalPlan for LogicalPlanNode { let mut converted_order_exprs: Vec = vec![]; for order in order_exprs { let temp = LogicalExprNodeCollection { - logical_expr_nodes: order - .iter() - .map(|expr| serialize_expr(expr, extension_codec)) - .collect::, ToProtoError>>()?, + logical_expr_nodes: serialize_exprs(order, extension_codec)?, }; converted_order_exprs.push(temp); } From 66bc222c53e98e65f8b6811cffbaf5de8bdf1985 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 15:25:44 +0200 Subject: [PATCH 17/75] Make it easier to understand datafusion-cli exclusion (#12188) --- Cargo.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/Cargo.toml b/Cargo.toml index 479d2cadc65b..50d234c39576 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -16,6 +16,7 @@ # under the License. [workspace] +# datafusion-cli is excluded because of its Cargo.lock. See datafusion-cli/README.md. exclude = ["datafusion-cli", "dev/depcheck"] members = [ "datafusion/common", From 6ffb1f64a7b3f48c7693d2ecb34f72a2dca24acc Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 27 Aug 2024 17:52:01 +0200 Subject: [PATCH 18/75] Remove unnecessary clones from `.../logical_plan/builder.rs` (#12196) --- datafusion/expr/src/logical_plan/builder.rs | 29 +++++++++------------ 1 file changed, 13 insertions(+), 16 deletions(-) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 6abcfe4279bc..9894fe887de0 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -211,7 +211,7 @@ impl LogicalPlanBuilder { }; common_type = Some(new_type); } else { - common_type = Some(data_type.clone()); + common_type = Some(data_type); } } field_types.push(common_type.unwrap_or(DataType::Utf8)); @@ -220,7 +220,7 @@ impl LogicalPlanBuilder { for row in &mut values { for (j, field_type) in field_types.iter().enumerate() { if let Expr::Literal(ScalarValue::Null) = row[j] { - row[j] = Expr::Literal(ScalarValue::try_from(field_type.clone())?); + row[j] = Expr::Literal(ScalarValue::try_from(field_type)?); } else { row[j] = std::mem::take(&mut row[j]).cast_to(field_type, &empty_schema)?; @@ -552,20 +552,17 @@ impl LogicalPlanBuilder { // Collect sort columns that are missing in the input plan's schema let mut missing_cols: Vec = vec![]; - exprs - .clone() - .into_iter() - .try_for_each::<_, Result<()>>(|expr| { - let columns = expr.column_refs(); + exprs.iter().try_for_each::<_, Result<()>>(|expr| { + let columns = expr.column_refs(); - columns.into_iter().for_each(|c| { - if !schema.has_column(c) { - missing_cols.push(c.clone()); - } - }); + columns.into_iter().for_each(|c| { + if !schema.has_column(c) { + missing_cols.push(c.clone()); + } + }); - Ok(()) - })?; + Ok(()) + })?; if missing_cols.is_empty() { return Ok(Self::new(LogicalPlan::Sort(Sort { @@ -710,7 +707,7 @@ impl LogicalPlanBuilder { pub(crate) fn normalize( plan: &LogicalPlan, - column: impl Into + Clone, + column: impl Into, ) -> Result { let schema = plan.schema(); let fallback_schemas = plan.fallback_normalize_schemas(); @@ -1536,7 +1533,7 @@ pub fn get_unnested_columns( | DataType::FixedSizeList(field, _) | DataType::LargeList(field) => { let new_field = Arc::new(Field::new( - col_name.clone(), + col_name, field.data_type().clone(), // Unnesting may produce NULLs even if the list is not null. // For example: unnset([1], []) -> 1, null From 8ba6732af5f4f32cbe0a23ef6bc2f393c640898b Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 28 Aug 2024 09:47:41 +0200 Subject: [PATCH 19/75] Remove unnecessary clones with clippy (#12197) * Remove unnecessary clones with clippy This is automated change done with ``` cargo clippy --fix -- -Aclippy::all -Wclippy::redundant_clone cargo fmt # manually fix few new clippy errors introduced by clippy itself ``` Note: it doesn't remove all unnecessary clones because the command reported error and backed out for the common crate. * Prevent clippy from removing intentional clone clippy can be run with `--fix` and then it won't obey the code comment instructing not to delete the clone. * Remove unnecessary clones pointed out by clippy Change code as instructed by ``` cargo clippy --fix -- -Aclippy::all -Wclippy::redundant_clone ``` where clippy didn't apply the suggested changes by itself. --- .../examples/advanced_parquet_index.rs | 2 +- .../examples/file_stream_provider.rs | 2 +- datafusion/common/src/dfschema.rs | 3 +- .../common/src/file_options/parquet_writer.rs | 2 +- datafusion/common/src/scalar/mod.rs | 8 ++-- datafusion/common/src/stats.rs | 1 + .../core/src/datasource/listing/helpers.rs | 2 +- .../core/src/datasource/listing/table.rs | 2 +- .../datasource/physical_plan/file_groups.rs | 7 +--- .../physical_plan/file_scan_config.rs | 4 +- .../core/src/datasource/schema_adapter.rs | 2 +- datafusion/core/src/execution/context/mod.rs | 2 +- .../enforce_distribution.rs | 23 +++++------ .../src/physical_optimizer/join_selection.rs | 21 +++++----- .../physical_optimizer/projection_pushdown.rs | 18 +++------ .../core/src/physical_optimizer/pruning.rs | 1 - datafusion/core/src/physical_planner.rs | 4 +- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 10 ++--- .../physical_optimizer/limit_pushdown.rs | 16 ++++---- .../limited_distinct_aggregation.rs | 22 +++++----- .../user_defined/user_defined_aggregates.rs | 2 +- .../user_defined_scalar_functions.rs | 2 +- datafusion/execution/src/memory_pool/pool.rs | 4 +- .../expr-common/src/interval_arithmetic.rs | 10 ++--- .../expr-common/src/type_coercion/binary.rs | 2 +- datafusion/expr/src/expr_schema.rs | 4 +- datafusion/expr/src/logical_plan/builder.rs | 2 +- datafusion/expr/src/logical_plan/display.rs | 20 ++++------ .../aggregate/groups_accumulator/prim_op.rs | 2 +- .../src/approx_percentile_cont.rs | 4 +- .../functions-aggregate/src/array_agg.rs | 7 +--- .../functions-aggregate/src/nth_value.rs | 7 +--- datafusion/functions-nested/src/flatten.rs | 2 +- datafusion/functions-nested/src/utils.rs | 3 +- .../optimizer/src/common_subexpr_eliminate.rs | 12 +++--- .../optimizer/src/eliminate_nested_union.rs | 17 +++----- .../optimizer/src/eliminate_one_union.rs | 5 +-- .../optimizer/src/optimize_projections/mod.rs | 4 +- datafusion/optimizer/src/push_down_filter.rs | 2 +- .../simplify_expressions/expr_simplifier.rs | 26 ++++++------ .../src/simplify_expressions/guarantees.rs | 4 +- .../src/unwrap_cast_in_comparison.rs | 2 +- .../physical-expr/src/equivalence/ordering.rs | 2 +- .../physical-expr/src/expressions/binary.rs | 4 +- .../physical-expr/src/expressions/case.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/joins/utils.rs | 2 +- .../physical-plan/src/recursive_query.rs | 2 +- .../src/windows/bounded_window_agg_exec.rs | 4 +- datafusion/proto/gen/src/main.rs | 2 +- .../tests/cases/roundtrip_logical_plan.rs | 2 +- .../tests/cases/roundtrip_physical_plan.rs | 40 +++++++++---------- datafusion/sql/src/expr/function.rs | 3 +- datafusion/sql/src/relation/mod.rs | 2 +- datafusion/sql/src/unparser/expr.rs | 2 +- datafusion/sql/tests/cases/plan_to_sql.rs | 2 +- 56 files changed, 162 insertions(+), 204 deletions(-) diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index 903defafe3ab..f6860bb5b87a 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -300,7 +300,7 @@ impl IndexTableProvider { // analyze the predicate. In a real system, using // `PruningPredicate::prune` would likely be easier to do. let pruning_predicate = - PruningPredicate::try_new(Arc::clone(predicate), self.schema().clone())?; + PruningPredicate::try_new(Arc::clone(predicate), self.schema())?; // The PruningPredicate's guarantees must all be satisfied in order for // the predicate to possibly evaluate to true. diff --git a/datafusion-examples/examples/file_stream_provider.rs b/datafusion-examples/examples/file_stream_provider.rs index b8549bd6b6e6..4db7e0200f53 100644 --- a/datafusion-examples/examples/file_stream_provider.rs +++ b/datafusion-examples/examples/file_stream_provider.rs @@ -100,7 +100,7 @@ mod non_windows { ) { // Timeout for a long period of BrokenPipe error let broken_pipe_timeout = Duration::from_secs(10); - let sa = file_path.clone(); + let sa = file_path; // Spawn a new thread to write to the FIFO file #[allow(clippy::disallowed_methods)] // spawn allowed only in tests tasks.spawn_blocking(move || { diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index f0eecd2ffeb1..095f4c510194 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -1242,10 +1242,9 @@ mod tests { #[test] fn into() { // Demonstrate how to convert back and forth between Schema, SchemaRef, DFSchema, and DFSchemaRef - let metadata = test_metadata(); let arrow_schema = Schema::new_with_metadata( vec![Field::new("c0", DataType::Int64, true)], - metadata.clone(), + test_metadata(), ); let arrow_schema_ref = Arc::new(arrow_schema.clone()); diff --git a/datafusion/common/src/file_options/parquet_writer.rs b/datafusion/common/src/file_options/parquet_writer.rs index 4a229fe01b54..e42fb96ed6a5 100644 --- a/datafusion/common/src/file_options/parquet_writer.rs +++ b/datafusion/common/src/file_options/parquet_writer.rs @@ -565,7 +565,7 @@ mod tests { column_options_with_non_defaults(&parquet_options), )] .into(), - key_value_metadata: [(key.clone(), value.clone())].into(), + key_value_metadata: [(key, value)].into(), }; let writer_props = WriterPropertiesBuilder::try_from(&table_parquet_opts) diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 677685b2c65b..5acc2b6f188e 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -4905,7 +4905,7 @@ mod tests { let data_type = DataType::List(Arc::new(Field::new("item", DataType::Int32, true))); - assert_eq!(non_null_list_scalar.data_type(), data_type.clone()); + assert_eq!(non_null_list_scalar.data_type(), data_type); assert_eq!(null_list_scalar.data_type(), data_type); } @@ -5582,13 +5582,13 @@ mod tests { // Define list-of-structs scalars - let nl0_array = ScalarValue::iter_to_array(vec![s0.clone(), s1.clone()]).unwrap(); + let nl0_array = ScalarValue::iter_to_array(vec![s0, s1.clone()]).unwrap(); let nl0 = ScalarValue::List(Arc::new(array_into_list_array_nullable(nl0_array))); - let nl1_array = ScalarValue::iter_to_array(vec![s2.clone()]).unwrap(); + let nl1_array = ScalarValue::iter_to_array(vec![s2]).unwrap(); let nl1 = ScalarValue::List(Arc::new(array_into_list_array_nullable(nl1_array))); - let nl2_array = ScalarValue::iter_to_array(vec![s1.clone()]).unwrap(); + let nl2_array = ScalarValue::iter_to_array(vec![s1]).unwrap(); let nl2 = ScalarValue::List(Arc::new(array_into_list_array_nullable(nl2_array))); // iter_to_array for list-of-struct diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 50ae4e3ca71f..d8e62b3045f9 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -557,6 +557,7 @@ mod tests { let precision: Precision = Precision::Exact(ScalarValue::Int64(Some(42))); // Clippy would complain about this if it were Copy + #[allow(clippy::redundant_clone)] let p2 = precision.clone(); assert_eq!(precision, p2); } diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index b5dd2dd12e10..f6e938b72dab 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -282,7 +282,7 @@ async fn prune_partitions( Default::default(), )?; - let batch = RecordBatch::try_new(schema.clone(), arrays)?; + let batch = RecordBatch::try_new(schema, arrays)?; // TODO: Plumb this down let props = ExecutionProps::new(); diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 89066d8234ac..a0345a38e40c 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1016,7 +1016,7 @@ impl ListingTable { .collected_statistics .get_with_extra(&part_file.object_meta.location, &part_file.object_meta) { - Some(statistics) => Ok(statistics.clone()), + Some(statistics) => Ok(statistics), None => { let statistics = self .options diff --git a/datafusion/core/src/datasource/physical_plan/file_groups.rs b/datafusion/core/src/datasource/physical_plan/file_groups.rs index fb2cd4ad06ec..28f975ae193d 100644 --- a/datafusion/core/src/datasource/physical_plan/file_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/file_groups.rs @@ -394,7 +394,7 @@ mod test { #[test] fn repartition_empty_file_only() { let partitioned_file_empty = pfile("empty", 0); - let file_group = vec![vec![partitioned_file_empty.clone()]]; + let file_group = vec![vec![partitioned_file_empty]]; let partitioned_files = FileGroupPartitioner::new() .with_target_partitions(4) @@ -817,10 +817,7 @@ mod test { .with_preserve_order_within_groups(true) .repartition_file_groups(&file_groups); - assert_partitioned_files( - repartitioned.clone(), - repartitioned_preserving_sort.clone(), - ); + assert_partitioned_files(repartitioned.clone(), repartitioned_preserving_sort); repartitioned } } diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 34fb6226c1a2..bfa5488e5b5e 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -908,7 +908,7 @@ mod tests { schema.clone(), Some(vec![0, 3, 5, schema.fields().len()]), Statistics::new_unknown(&schema), - to_partition_cols(partition_cols.clone()), + to_partition_cols(partition_cols), ) .projected_file_schema(); @@ -941,7 +941,7 @@ mod tests { schema.clone(), None, Statistics::new_unknown(&schema), - to_partition_cols(partition_cols.clone()), + to_partition_cols(partition_cols), ) .projected_file_schema(); diff --git a/datafusion/core/src/datasource/schema_adapter.rs b/datafusion/core/src/datasource/schema_adapter.rs index 40cb40a83af2..5d2d0ff91b15 100644 --- a/datafusion/core/src/datasource/schema_adapter.rs +++ b/datafusion/core/src/datasource/schema_adapter.rs @@ -369,7 +369,7 @@ mod tests { let f1 = Field::new("id", DataType::Int32, true); let f2 = Field::new("extra_column", DataType::Utf8, true); - let schema = Arc::new(Schema::new(vec![f1.clone(), f2.clone()])); + let schema = Arc::new(Schema::new(vec![f1, f2])); let extra_column = Arc::new(StringArray::from(vec!["foo"])); let mut new_columns = batch.columns().to_vec(); diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index c2a072eef756..18943599b136 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -1165,7 +1165,7 @@ impl SessionContext { // check schema uniqueness let mut batches = batches.into_iter().peekable(); let schema = if let Some(batch) = batches.peek() { - batch.schema().clone() + batch.schema() } else { Arc::new(Schema::empty()) }; diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 77ddd098c7be..ba6f7d0439c2 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -3908,7 +3908,7 @@ pub(crate) mod tests { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); - let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias.clone()); + let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); let expected_parquet = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", @@ -3934,7 +3934,7 @@ pub(crate) mod tests { let alias = vec![("a".to_string(), "a".to_string())]; let plan_parquet = aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); - let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias.clone()); + let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); let expected_parquet = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", @@ -3964,7 +3964,7 @@ pub(crate) mod tests { options: SortOptions::default(), }]; let plan_parquet = limit_exec(sort_exec(sort_key.clone(), parquet_exec(), false)); - let plan_csv = limit_exec(sort_exec(sort_key.clone(), csv_exec(), false)); + let plan_csv = limit_exec(sort_exec(sort_key, csv_exec(), false)); let expected_parquet = &[ "GlobalLimitExec: skip=0, fetch=100", @@ -4000,8 +4000,7 @@ pub(crate) mod tests { parquet_exec(), false, ))); - let plan_csv = - limit_exec(filter_exec(sort_exec(sort_key.clone(), csv_exec(), false))); + let plan_csv = limit_exec(filter_exec(sort_exec(sort_key, csv_exec(), false))); let expected_parquet = &[ "GlobalLimitExec: skip=0, fetch=100", @@ -4042,7 +4041,7 @@ pub(crate) mod tests { ); let plan_csv = aggregate_exec_with_alias( limit_exec(filter_exec(limit_exec(csv_exec()))), - alias.clone(), + alias, ); let expected_parquet = &[ @@ -4126,7 +4125,7 @@ pub(crate) mod tests { ); let plan_csv = sort_preserving_merge_exec( sort_key.clone(), - csv_exec_with_sort(vec![sort_key.clone()]), + csv_exec_with_sort(vec![sort_key]), ); // parallelization is not beneficial for SortPreservingMerge @@ -4154,7 +4153,7 @@ pub(crate) mod tests { union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); let input_csv = union_exec(vec![csv_exec_with_sort(vec![sort_key.clone()]); 2]); let plan_parquet = sort_preserving_merge_exec(sort_key.clone(), input_parquet); - let plan_csv = sort_preserving_merge_exec(sort_key.clone(), input_csv); + let plan_csv = sort_preserving_merge_exec(sort_key, input_csv); // should not repartition (union doesn't benefit from increased parallelism) // should not sort (as the data was already sorted) @@ -4224,8 +4223,8 @@ pub(crate) mod tests { ("c".to_string(), "c2".to_string()), ]; let proj_parquet = projection_exec_with_alias( - parquet_exec_with_sort(vec![sort_key.clone()]), - alias_pairs.clone(), + parquet_exec_with_sort(vec![sort_key]), + alias_pairs, ); let sort_key_after_projection = vec![PhysicalSortExpr { expr: col("c2", &proj_parquet.schema()).unwrap(), @@ -4560,7 +4559,7 @@ pub(crate) mod tests { }]; let alias = vec![("a".to_string(), "a".to_string())]; let input = parquet_exec_with_sort(vec![sort_key]); - let physical_plan = aggregate_exec_with_alias(input, alias.clone()); + let physical_plan = aggregate_exec_with_alias(input, alias); let expected = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", @@ -4584,7 +4583,7 @@ pub(crate) mod tests { let alias = vec![("a".to_string(), "a".to_string())]; let input = parquet_exec_multiple_sorted(vec![sort_key]); let aggregate = aggregate_exec_with_alias(input, alias.clone()); - let physical_plan = aggregate_exec_with_alias(aggregate, alias.clone()); + let physical_plan = aggregate_exec_with_alias(aggregate, alias); let expected = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index b849df88e4aa..2643ade8f481 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -908,7 +908,7 @@ mod tests_statistical { ); let optimized_join = JoinSelection::new() - .optimize(join.clone(), &ConfigOptions::new()) + .optimize(join, &ConfigOptions::new()) .unwrap(); let swapping_projection = optimized_join @@ -964,7 +964,7 @@ mod tests_statistical { ); let optimized_join = JoinSelection::new() - .optimize(join.clone(), &ConfigOptions::new()) + .optimize(join, &ConfigOptions::new()) .unwrap(); let swapped_join = optimized_join @@ -1140,7 +1140,7 @@ mod tests_statistical { ); let optimized_join = JoinSelection::new() - .optimize(join.clone(), &ConfigOptions::new()) + .optimize(join, &ConfigOptions::new()) .unwrap(); let swapped_join = optimized_join @@ -1180,7 +1180,7 @@ mod tests_statistical { ); let optimized_join = JoinSelection::new() - .optimize(join.clone(), &ConfigOptions::new()) + .optimize(join, &ConfigOptions::new()) .unwrap(); let swapping_projection = optimized_join @@ -1356,7 +1356,7 @@ mod tests_statistical { Arc::new(Column::new_with_schema("small_col", &small.schema()).unwrap()) as _, )]; check_join_partition_mode( - big.clone(), + big, small.clone(), join_on, true, @@ -1380,8 +1380,8 @@ mod tests_statistical { Arc::new(Column::new_with_schema("small_col", &small.schema()).unwrap()) as _, )]; check_join_partition_mode( - empty.clone(), - small.clone(), + empty, + small, join_on, true, PartitionMode::CollectLeft, @@ -1424,7 +1424,7 @@ mod tests_statistical { Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()) as _, )]; check_join_partition_mode( - bigger.clone(), + bigger, big.clone(), join_on, true, @@ -1472,7 +1472,7 @@ mod tests_statistical { ); let optimized_join = JoinSelection::new() - .optimize(join.clone(), &ConfigOptions::new()) + .optimize(join, &ConfigOptions::new()) .unwrap(); if !is_swapped { @@ -1913,8 +1913,7 @@ mod hash_join_tests { false, )?); - let optimized_join_plan = - hash_join_swap_subrule(join.clone(), &ConfigOptions::new())?; + let optimized_join_plan = hash_join_swap_subrule(join, &ConfigOptions::new())?; // If swap did happen let projection_added = optimized_join_plan.as_any().is::(); diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 9c545c17da3c..b3f3f90154d0 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -1692,12 +1692,9 @@ mod tests { ])); Arc::new( CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_projection(Some(vec![0, 1, 2, 3, 4])), + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_projection(Some(vec![0, 1, 2, 3, 4])), ) .with_has_header(false) .with_delimeter(0) @@ -1719,12 +1716,9 @@ mod tests { ])); Arc::new( CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_projection(Some(vec![3, 2, 1])), + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_projection(Some(vec![3, 2, 1])), ) .with_has_header(false) .with_delimeter(0) diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index 0ef390fff45c..a16abc607ee6 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -1369,7 +1369,6 @@ fn build_predicate_expression( let change_expr = in_list .list() .iter() - .cloned() .map(|e| { Arc::new(phys_expr::BinaryExpr::new( in_list.expr().clone(), diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 9501d3c6bbbb..fe8d79846630 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -701,7 +701,7 @@ impl DefaultPhysicalPlanner { let initial_aggr = Arc::new(AggregateExec::try_new( AggregateMode::Partial, groups.clone(), - aggregates.clone(), + aggregates, filters.clone(), input_exec, physical_input_schema.clone(), @@ -2569,7 +2569,7 @@ mod tests { impl NoOpExecutionPlan { fn new(schema: SchemaRef) -> Self { - let cache = Self::compute_properties(schema.clone()); + let cache = Self::compute_properties(schema); Self { cache } } diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index f1cca66712d7..1c2d8ece2f36 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -350,12 +350,10 @@ impl JoinFuzzTestCase { fn left_right(&self) -> (Arc, Arc) { let schema1 = self.input1[0].schema(); let schema2 = self.input2[0].schema(); - let left = Arc::new( - MemoryExec::try_new(&[self.input1.clone()], schema1.clone(), None).unwrap(), - ); - let right = Arc::new( - MemoryExec::try_new(&[self.input2.clone()], schema2.clone(), None).unwrap(), - ); + let left = + Arc::new(MemoryExec::try_new(&[self.input1.clone()], schema1, None).unwrap()); + let right = + Arc::new(MemoryExec::try_new(&[self.input2.clone()], schema2, None).unwrap()); (left, right) } diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index b051feb5750e..750544ecdec1 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -54,7 +54,7 @@ impl PartitionStream for DummyStreamPartition { fn transforms_streaming_table_exec_into_fetching_version_when_skip_is_zero( ) -> datafusion_common::Result<()> { let schema = create_schema(); - let streaming_table = streaming_table_exec(schema.clone())?; + let streaming_table = streaming_table_exec(schema)?; let global_limit = global_limit_exec(streaming_table, 0, Some(5)); let initial = get_plan_string(&global_limit); @@ -79,7 +79,7 @@ fn transforms_streaming_table_exec_into_fetching_version_when_skip_is_zero( fn transforms_streaming_table_exec_into_fetching_version_and_keeps_the_global_limit_when_skip_is_nonzero( ) -> datafusion_common::Result<()> { let schema = create_schema(); - let streaming_table = streaming_table_exec(schema.clone())?; + let streaming_table = streaming_table_exec(schema)?; let global_limit = global_limit_exec(streaming_table, 2, Some(5)); let initial = get_plan_string(&global_limit); @@ -107,7 +107,7 @@ fn transforms_coalesce_batches_exec_into_fetching_version_and_removes_local_limi let schema = create_schema(); let streaming_table = streaming_table_exec(schema.clone())?; let repartition = repartition_exec(streaming_table)?; - let filter = filter_exec(schema.clone(), repartition)?; + let filter = filter_exec(schema, repartition)?; let coalesce_batches = coalesce_batches_exec(filter); let local_limit = local_limit_exec(coalesce_batches, 5); let coalesce_partitions = coalesce_partitions_exec(local_limit); @@ -146,7 +146,7 @@ fn pushes_global_limit_exec_through_projection_exec() -> datafusion_common::Resu let schema = create_schema(); let streaming_table = streaming_table_exec(schema.clone())?; let filter = filter_exec(schema.clone(), streaming_table)?; - let projection = projection_exec(schema.clone(), filter)?; + let projection = projection_exec(schema, filter)?; let global_limit = global_limit_exec(projection, 0, Some(5)); let initial = get_plan_string(&global_limit); @@ -178,7 +178,7 @@ fn pushes_global_limit_exec_through_projection_exec_and_transforms_coalesce_batc let schema = create_schema(); let streaming_table = streaming_table_exec(schema.clone()).unwrap(); let coalesce_batches = coalesce_batches_exec(streaming_table); - let projection = projection_exec(schema.clone(), coalesce_batches)?; + let projection = projection_exec(schema, coalesce_batches)?; let global_limit = global_limit_exec(projection, 0, Some(5)); let initial = get_plan_string(&global_limit); @@ -256,7 +256,7 @@ fn keeps_pushed_local_limit_exec_when_there_are_multiple_input_partitions( let schema = create_schema(); let streaming_table = streaming_table_exec(schema.clone())?; let repartition = repartition_exec(streaming_table)?; - let filter = filter_exec(schema.clone(), repartition)?; + let filter = filter_exec(schema, repartition)?; let coalesce_partitions = coalesce_partitions_exec(filter); let global_limit = global_limit_exec(coalesce_partitions, 0, Some(5)); @@ -398,9 +398,7 @@ fn streaming_table_exec( ) -> datafusion_common::Result> { Ok(Arc::new(StreamingTableExec::try_new( schema.clone(), - vec![Arc::new(DummyStreamPartition { - schema: schema.clone(), - }) as _], + vec![Arc::new(DummyStreamPartition { schema }) as _], None, None, true, diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index 48389b0304f6..042f6d622565 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -316,11 +316,11 @@ fn test_no_group_by() -> Result<()> { // `SELECT FROM MemoryExec LIMIT 10;`, Single AggregateExec let single_agg = AggregateExec::try_new( AggregateMode::Single, - build_group_by(&schema.clone(), vec![]), - vec![], /* aggr_expr */ - vec![], /* filter_expr */ - source, /* input */ - schema.clone(), /* input_schema */ + build_group_by(&schema, vec![]), + vec![], /* aggr_expr */ + vec![], /* filter_expr */ + source, /* input */ + schema, /* input_schema */ )?; let limit_exec = LocalLimitExec::new( Arc::new(single_agg), @@ -346,7 +346,7 @@ fn test_has_aggregate_expression() -> Result<()> { // `SELECT FROM MemoryExec LIMIT 10;`, Single AggregateExec let single_agg = AggregateExec::try_new( AggregateMode::Single, - build_group_by(&schema.clone(), vec!["a".to_string()]), + build_group_by(&schema, vec!["a".to_string()]), vec![agg.count_expr(&schema)], /* aggr_expr */ vec![None], /* filter_expr */ source, /* input */ @@ -418,11 +418,11 @@ fn test_has_order_by() -> Result<()> { // the `a > 1` filter is applied in the AggregateExec let single_agg = AggregateExec::try_new( AggregateMode::Single, - build_group_by(&schema.clone(), vec!["a".to_string()]), - vec![], /* aggr_expr */ - vec![], /* filter_expr */ - source, /* input */ - schema.clone(), /* input_schema */ + build_group_by(&schema, vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![], /* filter_expr */ + source, /* input */ + schema, /* input_schema */ )?; let limit_exec = LocalLimitExec::new( Arc::new(single_agg), diff --git a/datafusion/core/tests/user_defined/user_defined_aggregates.rs b/datafusion/core/tests/user_defined/user_defined_aggregates.rs index 93550d38021a..1e0d3d9d514e 100644 --- a/datafusion/core/tests/user_defined/user_defined_aggregates.rs +++ b/datafusion/core/tests/user_defined/user_defined_aggregates.rs @@ -272,7 +272,7 @@ async fn deregister_udaf() -> Result<()> { Arc::new(vec![DataType::UInt64, DataType::Float64]), ); - ctx.register_udaf(my_avg.clone()); + ctx.register_udaf(my_avg); assert!(ctx.state().aggregate_functions().contains_key("my_avg")); diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index 259cce74f2e5..0f1c3b8e53c4 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -594,7 +594,7 @@ async fn deregister_udf() -> Result<()> { let cast2i64 = ScalarUDF::from(CastToI64UDF::new()); let ctx = SessionContext::new(); - ctx.register_udf(cast2i64.clone()); + ctx.register_udf(cast2i64); assert!(ctx.udfs().contains("cast_to_i64")); diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index d3cd93979baf..e169c1f319cc 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -366,7 +366,7 @@ impl MemoryPool for TrackConsumersPool { // wrap OOM message in top consumers DataFusionError::ResourcesExhausted( provide_top_memory_consumers_to_error_msg( - e.to_owned(), + e, self.report_top(self.top.into()), ), ) @@ -540,7 +540,7 @@ mod tests { // Test: will be the same per Top Consumers reported. r0.grow(10); // make r0=10, pool available=90 let new_consumer_same_name = MemoryConsumer::new(same_name); - let mut r1 = new_consumer_same_name.clone().register(&pool); + let mut r1 = new_consumer_same_name.register(&pool); // TODO: the insufficient_capacity_err() message is per reservation, not per consumer. // a followup PR will clarify this message "0 bytes already allocated for this reservation" let expected = "Additional allocation failed with top memory consumers (across reservations) as: foo consumed 10 bytes. Error: Failed to allocate additional 150 bytes for foo with 0 bytes already allocated for this reservation - 90 bytes remain available for the total pool"; diff --git a/datafusion/expr-common/src/interval_arithmetic.rs b/datafusion/expr-common/src/interval_arithmetic.rs index e3ff412e785b..6424888c896a 100644 --- a/datafusion/expr-common/src/interval_arithmetic.rs +++ b/datafusion/expr-common/src/interval_arithmetic.rs @@ -1877,11 +1877,7 @@ mod tests { .sub(value.clone()) .unwrap() .lt(&eps)); - assert!(value - .clone() - .sub(prev_value(value.clone())) - .unwrap() - .lt(&eps)); + assert!(value.sub(prev_value(value.clone())).unwrap().lt(&eps)); assert_ne!(next_value(value.clone()), value); assert_ne!(prev_value(value.clone()), value); }); @@ -1913,11 +1909,11 @@ mod tests { min_max.into_iter().zip(inf).for_each(|((min, max), inf)| { assert_eq!(next_value(max.clone()), inf); assert_ne!(prev_value(max.clone()), max); - assert_ne!(prev_value(max.clone()), inf); + assert_ne!(prev_value(max), inf); assert_eq!(prev_value(min.clone()), inf); assert_ne!(next_value(min.clone()), min); - assert_ne!(next_value(min.clone()), inf); + assert_ne!(next_value(min), inf); assert_eq!(next_value(inf.clone()), inf); assert_eq!(prev_value(inf.clone()), inf); diff --git a/datafusion/expr-common/src/type_coercion/binary.rs b/datafusion/expr-common/src/type_coercion/binary.rs index 552bc89b38f4..3617f56905a9 100644 --- a/datafusion/expr-common/src/type_coercion/binary.rs +++ b/datafusion/expr-common/src/type_coercion/binary.rs @@ -1881,7 +1881,7 @@ mod tests { ); test_coercion_binary_rule!( DataType::Timestamp(TimeUnit::Second, Some("Europe/Brussels".into())), - DataType::Timestamp(TimeUnit::Second, utc.clone()), + DataType::Timestamp(TimeUnit::Second, utc), Operator::Eq, DataType::Timestamp(TimeUnit::Second, Some("Europe/Brussels".into())) ); diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 10ec10e61239..3920a1a3517c 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -184,7 +184,7 @@ impl ExprSchemable for Expr { err, utils::generate_signature_error_msg( fun.name(), - fun.signature().clone(), + fun.signature(), &data_types ) ) @@ -199,7 +199,7 @@ impl ExprSchemable for Expr { err, utils::generate_signature_error_msg( fun.name(), - fun.signature().clone(), + fun.signature(), &data_types ) ) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 9894fe887de0..2c2300b123c2 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -2072,7 +2072,7 @@ mod tests { let schema = Schema::new(vec![ Field::new("scalar", DataType::UInt32, false), Field::new_list("strings", string_field, false), - Field::new_list("structs", struct_field_in_list.clone(), false), + Field::new_list("structs", struct_field_in_list, false), Field::new( "struct_singular", DataType::Struct(Fields::from(vec![ diff --git a/datafusion/expr/src/logical_plan/display.rs b/datafusion/expr/src/logical_plan/display.rs index 343eda056ffe..5a881deb54e1 100644 --- a/datafusion/expr/src/logical_plan/display.rs +++ b/datafusion/expr/src/logical_plan/display.rs @@ -387,19 +387,16 @@ impl<'a, 'b> PgJsonVisitor<'a, 'b> { } if !full_filter.is_empty() { - object["Full Filters"] = serde_json::Value::String( - expr_vec_fmt!(full_filter).to_string(), - ); + object["Full Filters"] = + serde_json::Value::String(expr_vec_fmt!(full_filter)); }; if !partial_filter.is_empty() { - object["Partial Filters"] = serde_json::Value::String( - expr_vec_fmt!(partial_filter).to_string(), - ); + object["Partial Filters"] = + serde_json::Value::String(expr_vec_fmt!(partial_filter)); } if !unsupported_filters.is_empty() { - object["Unsupported Filters"] = serde_json::Value::String( - expr_vec_fmt!(unsupported_filters).to_string(), - ); + object["Unsupported Filters"] = + serde_json::Value::String(expr_vec_fmt!(unsupported_filters)); } } @@ -595,9 +592,8 @@ impl<'a, 'b> PgJsonVisitor<'a, 'b> { "Select": expr_vec_fmt!(select_expr), }); if let Some(sort_expr) = sort_expr { - object["Sort"] = serde_json::Value::String( - expr_vec_fmt!(sort_expr).to_string(), - ); + object["Sort"] = + serde_json::Value::String(expr_vec_fmt!(sort_expr)); } object diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs index b5c6171af37c..8bbcf756c37c 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs @@ -167,7 +167,7 @@ where // Rebuilding input values with a new nulls mask, which is equal to // the union of original nulls and filter mask - let (dt, values_buf, original_nulls) = values.clone().into_parts(); + let (dt, values_buf, original_nulls) = values.into_parts(); let nulls_buf = NullBuffer::union(original_nulls.as_ref(), Some(&filter_nulls)); PrimitiveArray::::new(values_buf, nulls_buf).with_data_type(dt) diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index 867dfed485c7..5578aebbf403 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -126,9 +126,9 @@ impl ApproxPercentileCont { | DataType::Float32 | DataType::Float64) => { if let Some(max_size) = tdigest_max_size { - ApproxPercentileAccumulator::new_with_max_size(percentile, t.clone(), max_size) + ApproxPercentileAccumulator::new_with_max_size(percentile, t, max_size) }else{ - ApproxPercentileAccumulator::new(percentile, t.clone()) + ApproxPercentileAccumulator::new(percentile, t) } } diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index b641d388a7c5..15146fc4a2d8 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -501,11 +501,8 @@ impl OrderSensitiveArrayAggAccumulator { column_wise_ordering_values.push(array); } - let ordering_array = StructArray::try_new( - struct_field.clone(), - column_wise_ordering_values, - None, - )?; + let ordering_array = + StructArray::try_new(struct_field, column_wise_ordering_values, None)?; Ok(ScalarValue::List(Arc::new(array_into_list_array_nullable( Arc::new(ordering_array), )))) diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index cb1ddd4738c4..7425bdfa18e7 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -374,11 +374,8 @@ impl NthValueAccumulator { column_wise_ordering_values.push(array); } - let ordering_array = StructArray::try_new( - struct_field.clone(), - column_wise_ordering_values, - None, - )?; + let ordering_array = + StructArray::try_new(struct_field, column_wise_ordering_values, None)?; Ok(ScalarValue::List(Arc::new(array_into_list_array_nullable( Arc::new(ordering_array), diff --git a/datafusion/functions-nested/src/flatten.rs b/datafusion/functions-nested/src/flatten.rs index 2b383af3d456..b04c35667226 100644 --- a/datafusion/functions-nested/src/flatten.rs +++ b/datafusion/functions-nested/src/flatten.rs @@ -147,7 +147,7 @@ fn flatten_internal( let list_arr = GenericListArray::::new(field, offsets, values, None); Ok(list_arr) } else { - Ok(list_arr.clone()) + Ok(list_arr) } } } diff --git a/datafusion/functions-nested/src/utils.rs b/datafusion/functions-nested/src/utils.rs index 688e1633e5cf..3d5b261618d5 100644 --- a/datafusion/functions-nested/src/utils.rs +++ b/datafusion/functions-nested/src/utils.rs @@ -296,8 +296,7 @@ mod tests { let array3d_1 = Arc::new(array_into_list_array_nullable(array2d_1)) as ArrayRef; let array3d_2 = array_into_list_array_nullable(array2d_2.to_owned()); let res = - align_array_dimensions::(vec![array1d_1, Arc::new(array3d_2.clone())]) - .unwrap(); + align_array_dimensions::(vec![array1d_1, Arc::new(array3d_2)]).unwrap(); let expected = as_list_array(&array3d_1).unwrap(); let expected_dim = datafusion_common::utils::list_ndims(array3d_1.data_type()); diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 3fcee4123b76..3a2b190359d4 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -1431,7 +1431,7 @@ mod test { fn nested_aliases() -> Result<()> { let table_scan = test_table_scan()?; - let plan = LogicalPlanBuilder::from(table_scan.clone()) + let plan = LogicalPlanBuilder::from(table_scan) .project(vec![ (col("a") + col("b") - col("c")).alias("alias1") * (col("a") + col("b")), col("a") + col("b"), @@ -1842,7 +1842,7 @@ mod test { let config = &OptimizerContext::new(); let _common_expr_1 = config.alias_generator().next(CSE_PREFIX); let common_expr_2 = config.alias_generator().next(CSE_PREFIX); - let plan = LogicalPlanBuilder::from(table_scan.clone()) + let plan = LogicalPlanBuilder::from(table_scan) .project(vec![ (col("a") + col("b")).alias(common_expr_2.clone()), col("c"), @@ -1886,7 +1886,7 @@ mod test { let extracted_short_circuit_leg_1 = (col("a") + col("b")).eq(lit(0)); let not_extracted_short_circuit_leg_2 = (col("a") - col("b")).eq(lit(0)); let extracted_short_circuit_leg_3 = (col("a") * col("b")).eq(lit(0)); - let plan = LogicalPlanBuilder::from(table_scan.clone()) + let plan = LogicalPlanBuilder::from(table_scan) .project(vec![ extracted_short_circuit.clone().alias("c1"), extracted_short_circuit.alias("c2"), @@ -1899,7 +1899,7 @@ mod test { .alias("c4"), extracted_short_circuit_leg_3 .clone() - .or(extracted_short_circuit_leg_3.clone()) + .or(extracted_short_circuit_leg_3) .alias("c5"), ])? .build()?; @@ -1920,7 +1920,7 @@ mod test { let extracted_child = col("a") + col("b"); let rand = rand_func().call(vec![]); let not_extracted_volatile = extracted_child + rand; - let plan = LogicalPlanBuilder::from(table_scan.clone()) + let plan = LogicalPlanBuilder::from(table_scan) .project(vec![ not_extracted_volatile.clone().alias("c1"), not_extracted_volatile.alias("c2"), @@ -1947,7 +1947,7 @@ mod test { let not_extracted_short_circuit_leg_2 = col("b").eq(lit(0)); let not_extracted_volatile_short_circuit_2 = rand.eq(lit(0)).or(not_extracted_short_circuit_leg_2); - let plan = LogicalPlanBuilder::from(table_scan.clone()) + let plan = LogicalPlanBuilder::from(table_scan) .project(vec![ not_extracted_volatile_short_circuit_1.clone().alias("c1"), not_extracted_volatile_short_circuit_1.alias("c2"), diff --git a/datafusion/optimizer/src/eliminate_nested_union.rs b/datafusion/optimizer/src/eliminate_nested_union.rs index e9b38567a982..965771326854 100644 --- a/datafusion/optimizer/src/eliminate_nested_union.rs +++ b/datafusion/optimizer/src/eliminate_nested_union.rs @@ -144,10 +144,7 @@ mod tests { fn eliminate_nothing() -> Result<()> { let plan_builder = table_scan(Some("table"), &schema(), None)?; - let plan = plan_builder - .clone() - .union(plan_builder.clone().build()?)? - .build()?; + let plan = plan_builder.clone().union(plan_builder.build()?)?.build()?; let expected = "\ Union\ @@ -162,7 +159,7 @@ mod tests { let plan = plan_builder .clone() - .union_distinct(plan_builder.clone().build()?)? + .union_distinct(plan_builder.build()?)? .build()?; let expected = "Distinct:\ @@ -180,7 +177,7 @@ mod tests { .clone() .union(plan_builder.clone().build()?)? .union(plan_builder.clone().build()?)? - .union(plan_builder.clone().build()?)? + .union(plan_builder.build()?)? .build()?; let expected = "\ @@ -200,7 +197,7 @@ mod tests { .clone() .union_distinct(plan_builder.clone().build()?)? .union(plan_builder.clone().build()?)? - .union(plan_builder.clone().build()?)? + .union(plan_builder.build()?)? .build()?; let expected = "Union\ @@ -222,7 +219,7 @@ mod tests { .union(plan_builder.clone().build()?)? .union_distinct(plan_builder.clone().build()?)? .union(plan_builder.clone().build()?)? - .union_distinct(plan_builder.clone().build()?)? + .union_distinct(plan_builder.build()?)? .build()?; let expected = "Distinct:\ @@ -243,7 +240,7 @@ mod tests { .clone() .union_distinct(plan_builder.clone().distinct()?.build()?)? .union(plan_builder.clone().distinct()?.build()?)? - .union_distinct(plan_builder.clone().build()?)? + .union_distinct(plan_builder.build()?)? .build()?; let expected = "Distinct:\ @@ -271,7 +268,6 @@ mod tests { )? .union( plan_builder - .clone() .project(vec![col("id").alias("_id"), col("key"), col("value")])? .build()?, )? @@ -300,7 +296,6 @@ mod tests { )? .union_distinct( plan_builder - .clone() .project(vec![col("id").alias("_id"), col("key"), col("value")])? .build()?, )? diff --git a/datafusion/optimizer/src/eliminate_one_union.rs b/datafusion/optimizer/src/eliminate_one_union.rs index 383b33637f6f..7a1c4e118e05 100644 --- a/datafusion/optimizer/src/eliminate_one_union.rs +++ b/datafusion/optimizer/src/eliminate_one_union.rs @@ -94,10 +94,7 @@ mod tests { fn eliminate_nothing() -> Result<()> { let plan_builder = table_scan(Some("table"), &schema(), None)?; - let plan = plan_builder - .clone() - .union(plan_builder.clone().build()?)? - .build()?; + let plan = plan_builder.clone().union(plan_builder.build()?)?.build()?; let expected = "\ Union\ diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index 5db5afd11062..35b0d07751ff 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -1336,8 +1336,8 @@ mod tests { let right_table = test_table_scan_with_name("r")?; let custom_plan = LogicalPlan::Extension(Extension { node: Arc::new(UserDefinedCrossJoin::new( - Arc::new(left_table.clone()), - Arc::new(right_table.clone()), + Arc::new(left_table), + Arc::new(right_table), )), }); let plan = LogicalPlanBuilder::from(custom_plan) diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 6e75f46c2d0b..82149a087e63 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -819,7 +819,7 @@ impl OptimizerRule for PushDownFilter { .map(|e| Ok(Column::from_qualified_name(e.schema_name().to_string()))) .collect::>>()?; - let predicates = split_conjunction_owned(filter.predicate.clone()); + let predicates = split_conjunction_owned(filter.predicate); let mut keep_predicates = vec![]; let mut push_predicates = vec![]; diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index c45df74a564d..7129ceb0fea1 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -3407,32 +3407,32 @@ mod tests { let expr = in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], false).and( in_list(col("c1"), vec![lit(5), lit(6), lit(7), lit(8)], false), ); - assert_eq!(simplify(expr.clone()), lit(false)); + assert_eq!(simplify(expr), lit(false)); // 2. c1 IN (1,2,3,4) AND c1 IN (4,5,6,7) -> c1 = 4 let expr = in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], false).and( in_list(col("c1"), vec![lit(4), lit(5), lit(6), lit(7)], false), ); - assert_eq!(simplify(expr.clone()), col("c1").eq(lit(4))); + assert_eq!(simplify(expr), col("c1").eq(lit(4))); // 3. c1 NOT IN (1, 2, 3, 4) OR c1 NOT IN (5, 6, 7, 8) -> true let expr = in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], true).or( in_list(col("c1"), vec![lit(5), lit(6), lit(7), lit(8)], true), ); - assert_eq!(simplify(expr.clone()), lit(true)); + assert_eq!(simplify(expr), lit(true)); // 3.5 c1 NOT IN (1, 2, 3, 4) OR c1 NOT IN (4, 5, 6, 7) -> c1 != 4 (4 overlaps) let expr = in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], true).or( in_list(col("c1"), vec![lit(4), lit(5), lit(6), lit(7)], true), ); - assert_eq!(simplify(expr.clone()), col("c1").not_eq(lit(4))); + assert_eq!(simplify(expr), col("c1").not_eq(lit(4))); // 4. c1 NOT IN (1,2,3,4) AND c1 NOT IN (4,5,6,7) -> c1 NOT IN (1,2,3,4,5,6,7) let expr = in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], true).and( in_list(col("c1"), vec![lit(4), lit(5), lit(6), lit(7)], true), ); assert_eq!( - simplify(expr.clone()), + simplify(expr), in_list( col("c1"), vec![lit(1), lit(2), lit(3), lit(4), lit(5), lit(6), lit(7)], @@ -3445,7 +3445,7 @@ mod tests { in_list(col("c1"), vec![lit(2), lit(3), lit(4), lit(5)], false), ); assert_eq!( - simplify(expr.clone()), + simplify(expr), in_list( col("c1"), vec![lit(1), lit(2), lit(3), lit(4), lit(5)], @@ -3459,7 +3459,7 @@ mod tests { vec![lit(1), lit(2), lit(3), lit(4), lit(5)], true, )); - assert_eq!(simplify(expr.clone()), lit(false)); + assert_eq!(simplify(expr), lit(false)); // 7. c1 NOT IN (1,2,3,4) AND c1 IN (1,2,3,4,5) -> c1 = 5 let expr = @@ -3468,14 +3468,14 @@ mod tests { vec![lit(1), lit(2), lit(3), lit(4), lit(5)], false, )); - assert_eq!(simplify(expr.clone()), col("c1").eq(lit(5))); + assert_eq!(simplify(expr), col("c1").eq(lit(5))); // 8. c1 IN (1,2,3,4) AND c1 NOT IN (5,6,7,8) -> c1 IN (1,2,3,4) let expr = in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], false).and( in_list(col("c1"), vec![lit(5), lit(6), lit(7), lit(8)], true), ); assert_eq!( - simplify(expr.clone()), + simplify(expr), in_list(col("c1"), vec![lit(1), lit(2), lit(3), lit(4)], false) ); @@ -3493,7 +3493,7 @@ mod tests { )) .and(in_list(col("c1"), vec![lit(3), lit(6)], false)); assert_eq!( - simplify(expr.clone()), + simplify(expr), col("c1").eq(lit(3)).or(col("c1").eq(lit(6))) ); @@ -3507,7 +3507,7 @@ mod tests { )) .and(in_list(col("c1"), vec![lit(8), lit(9), lit(10)], false)), ); - assert_eq!(simplify(expr.clone()), col("c1").eq(lit(8))); + assert_eq!(simplify(expr), col("c1").eq(lit(8))); // Contains non-InList expression // c1 NOT IN (1,2,3,4) OR c1 != 5 OR c1 NOT IN (6,7,8,9) -> c1 NOT IN (1,2,3,4) OR c1 != 5 OR c1 NOT IN (6,7,8,9) @@ -3622,7 +3622,7 @@ mod tests { let expr_x = col("c3").gt(lit(3_i64)); let expr_y = (col("c4") + lit(2_u32)).lt(lit(10_u32)); let expr_z = col("c1").in_list(vec![lit("a"), lit("b")], true); - let expr = expr_x.clone().and(expr_y.clone().or(expr_z)); + let expr = expr_x.clone().and(expr_y.or(expr_z)); // All guaranteed null let guarantees = vec![ @@ -3698,7 +3698,7 @@ mod tests { col("c4"), NullableInterval::from(ScalarValue::UInt32(Some(3))), )]; - let output = simplify_with_guarantee(expr.clone(), guarantees); + let output = simplify_with_guarantee(expr, guarantees); assert_eq!(&output, &expr_x); } diff --git a/datafusion/optimizer/src/simplify_expressions/guarantees.rs b/datafusion/optimizer/src/simplify_expressions/guarantees.rs index 09fdd7685a9c..afcbe528083b 100644 --- a/datafusion/optimizer/src/simplify_expressions/guarantees.rs +++ b/datafusion/optimizer/src/simplify_expressions/guarantees.rs @@ -225,12 +225,12 @@ mod tests { // x IS NULL => guaranteed false let expr = col("x").is_null(); - let output = expr.clone().rewrite(&mut rewriter).data().unwrap(); + let output = expr.rewrite(&mut rewriter).data().unwrap(); assert_eq!(output, lit(false)); // x IS NOT NULL => guaranteed true let expr = col("x").is_not_null(); - let output = expr.clone().rewrite(&mut rewriter).data().unwrap(); + let output = expr.rewrite(&mut rewriter).data().unwrap(); assert_eq!(output, lit(true)); } diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index e0f50a470d43..b17d69437cbe 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -627,7 +627,7 @@ mod tests { Box::new(DataType::Int32), Box::new(ScalarValue::LargeUtf8(Some("value".to_owned()))), ); - let expr_input = cast(col("largestr"), dict.data_type()).eq(lit(dict.clone())); + let expr_input = cast(col("largestr"), dict.data_type()).eq(lit(dict)); let expected = col("largestr").eq(lit(ScalarValue::LargeUtf8(Some("value".to_owned())))); assert_eq!(optimize_test(expr_input, &schema), expected); diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index c4b8a5c46563..49a0de7252ab 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -272,7 +272,7 @@ mod tests { // Crude ordering doesn't satisfy finer ordering. should return false let mut eq_properties_crude = EquivalenceProperties::new(Arc::clone(&input_schema)); - eq_properties_crude.oeq_class.push(crude.clone()); + eq_properties_crude.oeq_class.push(crude); assert!(!eq_properties_crude.ordering_satisfy(&finer)); Ok(()) } diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index b663d8614275..2680a7930ff1 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -2582,7 +2582,7 @@ mod tests { &a, &b, Operator::RegexIMatch, - regex_expected.clone(), + regex_expected, )?; apply_logic_op( &Arc::new(schema.clone()), @@ -2596,7 +2596,7 @@ mod tests { &a, &b, Operator::RegexNotIMatch, - regex_not_expected.clone(), + regex_not_expected, )?; Ok(()) diff --git a/datafusion/physical-expr/src/expressions/case.rs b/datafusion/physical-expr/src/expressions/case.rs index c6afb5c05985..712175c9afbe 100644 --- a/datafusion/physical-expr/src/expressions/case.rs +++ b/datafusion/physical-expr/src/expressions/case.rs @@ -380,7 +380,7 @@ impl CaseExpr { // keep `else_expr`'s data type and return type consistent let e = self.else_expr.as_ref().unwrap(); - let expr = try_cast(Arc::clone(e), &batch.schema(), return_type.clone()) + let expr = try_cast(Arc::clone(e), &batch.schema(), return_type) .unwrap_or_else(|_| Arc::clone(e)); let else_ = Scalar::new(expr.evaluate(batch)?.into_array(1)?); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 0aeaa15b5f1d..6ca01928c80f 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2220,7 +2220,7 @@ mod tests { let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, groups, - aggregates.clone(), + aggregates, vec![None, None], Arc::clone(&blocking_exec) as Arc, schema, diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 8fdbf7041e2f..89f3feaf07be 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -2443,7 +2443,7 @@ mod tests { Statistics { num_rows: Absent, total_byte_size: Absent, - column_statistics: dummy_column_stats.clone(), + column_statistics: dummy_column_stats, }, &join_on, ); diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index bd9303f97db0..e9ea9d4f5032 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -394,7 +394,7 @@ impl Stream for RecursiveQueryStream { self.recursive_stream = None; self.poll_next_iteration(cx) } - Some(Ok(batch)) => self.push_batch(batch.clone()), + Some(Ok(batch)) => self.push_batch(batch), _ => Poll::Ready(batch_result), } } else { diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 084436ee376d..c1bcd83a6fd2 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1312,7 +1312,7 @@ mod tests { &args, &partitionby_exprs, &orderby_exprs, - Arc::new(window_frame.clone()), + Arc::new(window_frame), &input.schema(), false, )?], @@ -1485,7 +1485,7 @@ mod tests { let partitions = vec![ Arc::new(TestStreamPartition { schema: Arc::clone(&schema), - batches: batches.clone(), + batches, idx: 0, state: PolingState::BatchReturn, sleep_duration: per_batch_wait_duration, diff --git a/datafusion/proto/gen/src/main.rs b/datafusion/proto/gen/src/main.rs index d3b3c92f6065..be61ff58fa8d 100644 --- a/datafusion/proto/gen/src/main.rs +++ b/datafusion/proto/gen/src/main.rs @@ -55,7 +55,7 @@ fn main() -> Result<(), String> { let common_path = proto_dir.join("src/datafusion_common.rs"); println!( "Copying {} to {}", - prost.clone().display(), + prost.display(), proto_dir.join("src/generated/prost.rs").display() ); std::fs::copy(prost, proto_dir.join("src/generated/prost.rs")).unwrap(); diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 4f58185798f7..94ac913e1968 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -2434,7 +2434,7 @@ fn roundtrip_window() { WindowFunctionDefinition::AggregateUDF(avg_udaf()), vec![col("col1")], )) - .window_frame(row_number_frame.clone()) + .window_frame(row_number_frame) .build() .unwrap(); diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 60f5565bdeee..3e49dc24fd5a 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -404,7 +404,7 @@ fn rountrip_aggregate_with_limit() -> Result<()> { let agg = AggregateExec::try_new( AggregateMode::Final, PhysicalGroupBy::new_single(groups.clone()), - aggregates.clone(), + aggregates, vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, @@ -433,7 +433,7 @@ fn rountrip_aggregate_with_approx_pencentile_cont() -> Result<()> { let agg = AggregateExec::try_new( AggregateMode::Final, PhysicalGroupBy::new_single(groups.clone()), - aggregates.clone(), + aggregates, vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, @@ -469,7 +469,7 @@ fn rountrip_aggregate_with_sort() -> Result<()> { let agg = AggregateExec::try_new( AggregateMode::Final, PhysicalGroupBy::new_single(groups.clone()), - aggregates.clone(), + aggregates, vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, @@ -536,7 +536,7 @@ fn roundtrip_aggregate_udaf() -> Result<()> { Arc::new(AggregateExec::try_new( AggregateMode::Final, PhysicalGroupBy::new_single(groups.clone()), - aggregates.clone(), + aggregates, vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, @@ -640,7 +640,7 @@ fn roundtrip_coalesce_with_fetch() -> Result<()> { )))?; roundtrip_test(Arc::new( - CoalesceBatchesExec::new(Arc::new(EmptyExec::new(schema.clone())), 8096) + CoalesceBatchesExec::new(Arc::new(EmptyExec::new(schema)), 8096) .with_fetch(Some(10)), )) } @@ -991,18 +991,16 @@ fn roundtrip_scalar_udf_extension_codec() -> Result<()> { )), input, )?); - let aggr_expr = AggregateExprBuilder::new( - max_udaf(), - vec![udf_expr.clone() as Arc], - ) - .schema(schema.clone()) - .alias("max") - .build()?; + let aggr_expr = + AggregateExprBuilder::new(max_udaf(), vec![udf_expr as Arc]) + .schema(schema.clone()) + .alias("max") + .build()?; let window = Arc::new(WindowAggExec::try_new( vec![Arc::new(PlainAggregateWindowExpr::new( aggr_expr.clone(), - &[col("author", &schema.clone())?], + &[col("author", &schema)?], &[], Arc::new(WindowFrame::new(None)), ))], @@ -1013,10 +1011,10 @@ fn roundtrip_scalar_udf_extension_codec() -> Result<()> { let aggregate = Arc::new(AggregateExec::try_new( AggregateMode::Final, PhysicalGroupBy::new(vec![], vec![], vec![]), - vec![aggr_expr.clone()], + vec![aggr_expr], vec![None], window, - schema.clone(), + schema, )?); let ctx = SessionContext::new(); @@ -1054,7 +1052,7 @@ fn roundtrip_aggregate_udf_extension_codec() -> Result<()> { Arc::new(BinaryExpr::new( col("published", &schema)?, Operator::And, - Arc::new(BinaryExpr::new(udf_expr.clone(), Operator::Gt, lit(0))), + Arc::new(BinaryExpr::new(udf_expr, Operator::Gt, lit(0))), )), input, )?); @@ -1083,7 +1081,7 @@ fn roundtrip_aggregate_udf_extension_codec() -> Result<()> { vec![aggr_expr], vec![None], window, - schema.clone(), + schema, )?); let ctx = SessionContext::new(); @@ -1158,7 +1156,7 @@ fn roundtrip_json_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - schema.clone(), + schema, Some(sort_order), ))) } @@ -1197,7 +1195,7 @@ fn roundtrip_csv_sink() -> Result<()> { Arc::new(DataSinkExec::new( input, data_sink, - schema.clone(), + schema, Some(sort_order), )), &ctx, @@ -1253,7 +1251,7 @@ fn roundtrip_parquet_sink() -> Result<()> { roundtrip_test(Arc::new(DataSinkExec::new( input, data_sink, - schema.clone(), + schema, Some(sort_order), ))) } @@ -1342,7 +1340,7 @@ fn roundtrip_interleave() -> Result<()> { )?; let right = RepartitionExec::try_new( Arc::new(EmptyExec::new(Arc::new(schema_right))), - partition.clone(), + partition, )?; let inputs: Vec> = vec![Arc::new(left), Arc::new(right)]; let interleave = InterleaveExec::try_new(inputs)?; diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 72e08e4b8fb5..71e40c20b80a 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -245,8 +245,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Build Unnest expression if name.eq("unnest") { - let mut exprs = - self.function_args_to_expr(args.clone(), schema, planner_context)?; + let mut exprs = self.function_args_to_expr(args, schema, planner_context)?; if exprs.len() != 1 { return plan_err!("unnest() requires exactly one argument"); } diff --git a/datafusion/sql/src/relation/mod.rs b/datafusion/sql/src/relation/mod.rs index 86e49780724b..f8ebb04f3810 100644 --- a/datafusion/sql/src/relation/mod.rs +++ b/datafusion/sql/src/relation/mod.rs @@ -214,7 +214,7 @@ fn optimize_subquery_sort(plan: LogicalPlan) -> Result> // 2. RANK / ROW_NUMBER ... => Handled by a `WindowAggr` and its requirements. // 3. LIMIT => Handled by a `Sort`, so we need to search for it. let mut has_limit = false; - let new_plan = plan.clone().transform_down(|c| { + let new_plan = plan.transform_down(|c| { if let LogicalPlan::Limit(_) = c { has_limit = true; return Ok(Transformed::no(c)); diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 9ce627aecc76..0dbcba162bc0 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -1864,7 +1864,7 @@ mod tests { r#"EXISTS (SELECT * FROM t WHERE (t.a = 1))"#, ), ( - not_exists(Arc::new(dummy_logical_plan.clone())), + not_exists(Arc::new(dummy_logical_plan)), r#"NOT EXISTS (SELECT * FROM t WHERE (t.a = 1))"#, ), ( diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index ed23fada0cfb..cdc7bef06afd 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -203,7 +203,7 @@ fn roundtrip_crossjoin() -> Result<()> { println!("plan {}", plan.display_indent()); let plan_roundtrip = sql_to_rel - .sql_statement_to_plan(roundtrip_statement.clone()) + .sql_statement_to_plan(roundtrip_statement) .unwrap(); let expected = "Projection: j1.j1_id, j2.j2_string\ From 7d8bb0b346643381fbe79abc480cc57a5522a5d6 Mon Sep 17 00:00:00 2001 From: WeblWabl Date: Wed, 28 Aug 2024 07:26:41 -0500 Subject: [PATCH 20/75] Add documentation on `EXPLAIN` and `EXPLAIN ANALYZE` (#12122) * feat(wip): working on adding explain docs Signed-off-by: Devan * working on it Signed-off-by: Devan * working on it -- adding plan descriptions Signed-off-by: Devan * adds descriptions for phys plan and note on parallel Signed-off-by: Devan * adds information about the logical plan Signed-off-by: Devan * Add page to index * Update example, add information on how to get the datafile * Add example of reading explain analyze * Add section on partitioning * prettier * adding aggregate plan explain docs Signed-off-by: Devan * field -> column Signed-off-by: Devan * repartition update Signed-off-by: Devan * prettier Signed-off-by: Devan * clarify some points --------- Signed-off-by: Devan Co-authored-by: Andrew Lamb --- docs/source/index.rst | 1 + docs/source/user-guide/explain-usage.md | 365 ++++++++++++++++++++++++ docs/source/user-guide/sql/explain.md | 2 + 3 files changed, 368 insertions(+) create mode 100644 docs/source/user-guide/explain-usage.md diff --git a/docs/source/index.rst b/docs/source/index.rst index 9c8c886d2502..bb5ea430a321 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -89,6 +89,7 @@ To get started, see user-guide/expressions user-guide/sql/index user-guide/configs + user-guide/explain-usage user-guide/faq .. _toc.library-user-guide: diff --git a/docs/source/user-guide/explain-usage.md b/docs/source/user-guide/explain-usage.md new file mode 100644 index 000000000000..a65fad92d104 --- /dev/null +++ b/docs/source/user-guide/explain-usage.md @@ -0,0 +1,365 @@ + + +# Reading Explain Plans + +## Introduction + +This section describes of how to read a DataFusion query plan. While fully +comprehending all details of these plans requires significant expertise in the +DataFusion engine, this guide will help you get started with the basics. + +Datafusion executes queries using a `query plan`. To see the plan without +running the query, add the keyword `EXPLAIN` to your SQL query or call the +[DataFrame::explain] method + +[dataframe::explain]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html#method.explain + +## Example: Select and filter + +In this section, we run example queries against the `hits.parquet` file. See +[below](#data-in-this-example)) for information on how to get this file. + +Let's see how DataFusion runs a query that selects the top 5 watch lists for the +site `http://domcheloveplanet.ru/`: + +```sql +EXPLAIN SELECT "WatchID" AS wid, "hits.parquet"."ClientIP" AS ip +FROM 'hits.parquet' +WHERE starts_with("URL", 'http://domcheloveplanet.ru/') +ORDER BY wid ASC, ip DESC +LIMIT 5; +``` + +The output will look like + +``` ++---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Sort: wid ASC NULLS LAST, ip DESC NULLS FIRST, fetch=5 | +| | Projection: hits.parquet.WatchID AS wid, hits.parquet.ClientIP AS ip | +| | Filter: starts_with(hits.parquet.URL, Utf8("http://domcheloveplanet.ru/")) | +| | TableScan: hits.parquet projection=[WatchID, ClientIP, URL], partial_filters=[starts_with(hits.parquet.URL, Utf8("http://domcheloveplanet.ru/"))] | +| physical_plan | SortPreservingMergeExec: [wid@0 ASC NULLS LAST,ip@1 DESC], fetch=5 | +| | SortExec: TopK(fetch=5), expr=[wid@0 ASC NULLS LAST,ip@1 DESC], preserve_partitioning=[true] | +| | ProjectionExec: expr=[WatchID@0 as wid, ClientIP@1 as ip] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | FilterExec: starts_with(URL@2, http://domcheloveplanet.ru/) | +| | ParquetExec: file_groups={16 groups: [[hits.parquet:0..923748528], ...]}, projection=[WatchID, ClientIP, URL], predicate=starts_with(URL@13, http://domcheloveplanet.ru/) | ++---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +2 row(s) fetched. +Elapsed 0.060 seconds. +``` + +There are two sections: logical plan and physical plan + +- **Logical Plan:** is a plan generated for a specific SQL query, DataFrame, or other language without the + knowledge of the underlying data organization. +- **Physical Plan:** is a plan generated from a logical plan along with consideration of the hardware + configuration (e.g number of CPUs) and the underlying data organization (e.g number of files). + This physical plan is specific to your hardware configuration and your data. If you load the same + data to different hardware with different configurations, the same query may generate different query plans. + +Understanding a query plan can help to you understand its performance. For example, when the plan shows your query reads +many files, it signals you to either add more filter in the query to read less data or to modify your file +design to make fewer but larger files. This document focuses on how to read a query plan. How to make a +query run faster depends on the reason it is slow and beyond the scope of this document. + +## Query plans are trees + +A query plan is an upside down tree, and we always read from bottom up. The +physical plan in Figure 1 in tree format will look like + +``` + ▲ + │ + │ +┌─────────────────────────────────────────────────┐ +│ SortPreservingMergeExec │ +│ [wid@0 ASC NULLS LAST,ip@1 DESC] │ +│ fetch=5 │ +└─────────────────────────────────────────────────┘ + ▲ + │ +┌─────────────────────────────────────────────────┐ +│ SortExec TopK(fetch=5), │ +│ expr=[wid@0 ASC NULLS LAST,ip@1 DESC], │ +│ preserve_partitioning=[true] │ +└─────────────────────────────────────────────────┘ + ▲ + │ +┌─────────────────────────────────────────────────┐ +│ ProjectionExec │ +│ expr=[WatchID@0 as wid, ClientIP@1 as ip] │ +└─────────────────────────────────────────────────┘ + ▲ + │ +┌─────────────────────────────────────────────────┐ +│ CoalesceBatchesExec │ +└─────────────────────────────────────────────────┘ + ▲ + │ +┌─────────────────────────────────────────────────┐ +│ FilterExec │ +│ starts_with(URL@2, http://domcheloveplanet.ru/) │ +└─────────────────────────────────────────────────┘ + ▲ + │ +┌────────────────────────────────────────────────┐ +│ ParquetExec │ +│ hits.parquet (filter = ...) │ +└────────────────────────────────────────────────┘ +``` + +Each node in the tree/plan ends with `Exec` and is sometimes also called an `operator` or `ExecutionPlan` where data is +processed, transformed and sent up. + +1. First, data in parquet the `hits.parquet` file us read in parallel using 16 cores in 16 "partitions" (more on this later) from `ParquetExec`, which applies a first pass at filtering during the scan. +2. Next, the output is filtered using `FilterExec` to ensure only rows where `starts_with(URL, 'http://domcheloveplanet.ru/')` evaluates to true are passed on +3. The `CoalesceBatchesExec` then ensures that the data is grouped into larger batches for processing +4. The `ProjectionExec` then projects the data to rename the `WatchID` and `ClientIP` columns to `wid` and `ip` respectively. +5. The `SortExec` then sorts the data by `wid ASC, ip DESC`. The `Topk(fetch=5)` indicates that a special implementation is used that only tracks and emits the top 5 values in each partition. +6. Finally the `SortPreservingMergeExec` merges the sorted data from all partitions and returns the top 5 rows overall. + +## Understanding large query plans + +A large query plan may look intimidating, but you can quickly understand what it does by following these steps + +1. As always, read from bottom up, one operator at a time. +2. Understand the job of this operator by reading + the [Physical Plan documentation](https://docs.rs/datafusion/latest/datafusion/physical_plan/index.html). +3. Understand the input data of the operator and how large/small it may be. +4. Understand how much data that operator produces and what it would look like. + +If you can answer those questions, you will be able to estimate how much work +that plan has to do and thus how long it will take. However, the `EXPLAIN` just +shows you the plan without executing it. + +If you want to know more about how much work each operator in query plan does, +you can use the `EXPLAIN ANALYZE` to get the explain with runtime added (see +next section) + +## More Debugging Information: `EXPLAIN VERBOSE` + +If the plan has to read too many files, not all of them will be shown in the +`EXPLAIN`. To see them, use `EXPLAIN VEBOSE`. Like `EXPLAIN`, `EXPLAIN VERBOSE` +does not run the query. Instead it shows the full explain plan, with information +that is omitted from the default explain, as well as all intermediate physical +plans DataFusion generates before returning. This mode can be very helpful for +debugging to see why and when DataFusion added and removed operators from a plan. + +## Execution Counters: `EXPLAIN ANALYZE` + +During execution, DataFusion operators collect detailed metrics. You can access +them programmatically via [`ExecutionPlan::metrics`] as well as with the +`EXPLAIN ANALYZE` command. For example here is the same query query as +above but with `EXPLAIN ANALYZE` (note the output is edited for clarity) + +[`executionplan::metrics`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#method.metrics + +``` +> EXPLAIN ANALYZE SELECT "WatchID" AS wid, "hits.parquet"."ClientIP" AS ip +FROM 'hits.parquet' +WHERE starts_with("URL", 'http://domcheloveplanet.ru/') +ORDER BY wid ASC, ip DESC +LIMIT 5; ++-------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++-------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| Plan with Metrics | SortPreservingMergeExec: [wid@0 ASC NULLS LAST,ip@1 DESC], fetch=5, metrics=[output_rows=5, elapsed_compute=2.375µs] | +| | SortExec: TopK(fetch=5), expr=[wid@0 ASC NULLS LAST,ip@1 DESC], preserve_partitioning=[true], metrics=[output_rows=75, elapsed_compute=7.243038ms, row_replacements=482] | +| | ProjectionExec: expr=[WatchID@0 as wid, ClientIP@1 as ip], metrics=[output_rows=811821, elapsed_compute=66.25µs] | +| | FilterExec: starts_with(URL@2, http://domcheloveplanet.ru/), metrics=[output_rows=811821, elapsed_compute=1.36923816s] | +| | ParquetExec: file_groups={16 groups: [[hits.parquet:0..923748528], ...]}, projection=[WatchID, ClientIP, URL], predicate=starts_with(URL@13, http://domcheloveplanet.ru/), metrics=[output_rows=99997497, elapsed_compute=16ns, ... bytes_scanned=3703192723, ... time_elapsed_opening=308.203002ms, time_elapsed_scanning_total=8.350342183s, ...] | ++-------------------+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +1 row(s) fetched. +Elapsed 0.720 seconds. +``` + +In this case, DataFusion actually ran the query, but discarded any results, and +instead returned an annotated plan with a new field, `metrics=[...]` + +Most operators have the common metrics `output_rows` and `elapsed_compute` and +some have operator specific metrics such as `ParquetExec` which has +`bytes_scanned=3703192723`. Note that times and counters are reported across all +cores, so if you have 16 cores, the time reported is the sum of the time taken +by all 16 cores. + +Again, reading from bottom up: + +- `ParquetExec` + - `output_rows=99997497`: A total 99.9M rows were produced + - `bytes_scanned=3703192723`: Of the 14GB file, 3.7GB were actually read (due to projection pushdown) + - `time_elapsed_opening=308.203002ms`: It took 300ms to open the file and prepare to read it + - `time_elapsed_scanning_total=8.350342183s`: It took 8.3 seconds of CPU time (across 16 cores) to actually decode the parquet data +- `FilterExec` + - `output_rows=811821`: Of the 99.9M rows at its input, only 811K rows passed the filter and were produced at the output + - `elapsed_compute=1.36923816s`: In total, 1.36s of CPU time (across 16 cores) was spend evaluating the filter +- `CoalesceBatchesExec` + - `output_rows=811821`, `elapsed_compute=12.873379ms`: Produced 811K rows in 13ms +- `ProjectionExec` + - `output_rows=811821, elapsed_compute=66.25µs`: Produced 811K rows in 66µs (microseconds). This projection is almost instantaneous as it does not manipulate any data +- `SortExec` + - `output_rows=75`: Produced 75 rows in total. Each of 16 cores could produce up to 5 rows, but in this case not all cores did. + - `elapsed_compute=7.243038ms`: 7ms was used to determine the top 5 rows + - `row_replacements=482`: Internally, the TopK operator updated its top list 482 times +- `SortPreservingMergeExec` + - `output_rows=5`, `elapsed_compute=2.375µs`: Produced the final 5 rows in 2.375µs (microseconds) + +## Partitions and Execution + +DataFusion determines the optimal number of cores to use as part of query +planning. Roughly speaking, each "partition" in the plan is run independently using +a separate core. Data crosses between cores only within certain operators such as +`RepartitionExec`, `CoalescePartitions` and `SortPreservingMergeExec` + +You can read more about this in the [Partitoning Docs]. + +[partitoning docs]: https://docs.rs/datafusion/latest/datafusion/physical_expr/enum.Partitioning.html + +## Example of an Aggregate Query + +Let us delve into an example query that aggregates data from the `hits.parquet` +file. For example, this query from ClickBench finds the top 10 users by their +number of hits: + +```sql +SELECT "UserID", COUNT(*) +FROM 'hits.parquet' +GROUP BY "UserID" +ORDER BY COUNT(*) DESC +LIMIT 10; +``` + +We can again see the query plan by using `EXPLAIN`: + +``` +> EXPLAIN SELECT "UserID", COUNT(*) FROM 'hits.parquet' GROUP BY "UserID" ORDER BY COUNT(*) DESC LIMIT 10; ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Limit: skip=0, fetch=10 | +| | Sort: count(*) DESC NULLS FIRST, fetch=10 | +| | Aggregate: groupBy=[[hits.parquet.UserID]], aggr=[[count(Int64(1)) AS count(*)]] | +| | TableScan: hits.parquet projection=[UserID] | +| physical_plan | GlobalLimitExec: skip=0, fetch=10 | +| | SortPreservingMergeExec: [count(*)@1 DESC], fetch=10 | +| | SortExec: TopK(fetch=10), expr=[count(*)@1 DESC], preserve_partitioning=[true] | +| | AggregateExec: mode=FinalPartitioned, gby=[UserID@0 as UserID], aggr=[count(*)] | +| | CoalesceBatchesExec: target_batch_size=8192 | +| | RepartitionExec: partitioning=Hash([UserID@0], 10), input_partitions=10 | +| | AggregateExec: mode=Partial, gby=[UserID@0 as UserID], aggr=[count(*)] | +| | ParquetExec: file_groups={10 groups: [[hits.parquet:0..1477997645], [hits.parquet:1477997645..2955995290], [hits.parquet:2955995290..4433992935], [hits.parquet:4433992935..5911990580], [hits.parquet:5911990580..7389988225], ...]}, projection=[UserID] | +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +``` + +For this query, let's again read the plan from the bottom to the top: + +**Logical plan operators** + +- `TableScan` + - `hits.parquet`: Scans data from the file `hits.parquet`. + - `projection=[UserID]`: Reads only the `UserID` column +- `Aggregate` + - `groupBy=[[hits.parquet.UserID]]`: Groups by `UserID` column. + - `aggr=[[count(Int64(1)) AS count(*)]]`: Applies the `COUNT` aggregate on each distinct group. +- `Sort` + - `count(*) DESC NULLS FIRST`: Sorts the data in descending count order. + - `fetch=10`: Returns only the first 10 rows. +- `Limit` + - `skip=0`: Does not skip any data for the results. + - `fetch=10`: Limits the results to 10 values. + +**Physical plan operators** + +- `ParquetExec` + - `file_groups={10 groups: [...]}`: Reads 10 groups in parallel from `hits.parquet`file. (The example above was run on a machine with 10 cores.) + - `projection=[UserID]`: Pushes down projection of the `UserID` column. The parquet format is columnar and the DataFusion reader only decodes the columns required. +- `AggregateExec` + - `mode=Partial` Runs a [partial aggregation] in parallel across each of the 10 partitions from the `ParquetExec` immediately after reading. + - `gby=[UserID@0 as UserID]`: Represents `GROUP BY` in the [physical plan] and groups together the same values of `UserID`. + - `aggr=[count(*)]`: Applies the `COUNT` aggregate on all rows for each group. +- `RepartitionExec` + - `partitioning=Hash([UserID@0], 10)`: Divides the input into into 10 (new) output partitions based on the value of `hash(UserID)`. You can read more about this in the [partitioning] documentation. + - `input_partitions=10`: Number of input partitions. +- `CoalesceBatchesExec` + - `target_batch_size=8192`: Combines smaller batches in to larger batches. In this case approximately 8192 rows in each batch. +- `AggregateExec` + - `mode=FinalPartitioned`: Performs the final aggregation on each group. See the [documentation on multi phase grouping] for more information. + - `gby=[UserID@0 as UserID]`: Groups by `UserID`. + - `aggr=[count(*)]`: Applies the `COUNT` aggregate on all rows for each group. +- `SortExec` + - `TopK(fetch=10)`: Use a special "TopK" sort that keeps only the largest 10 values in memory at a time. You can read more about this in the [TopK] documentation. + - `expr=[count(*)@1 DESC]`: Sorts all rows in descending order. Note this represents the `ORDER BY` in the physical plan. + - `preserve_partitioning=[true]`: The sort is done in parallel on each partition. In this case the top 10 values are found for each of the 10 partitions, in parallel. +- `SortPreservingMergeExec` + - `[count(*)@1 DESC]`: This operator merges the 10 distinct streams into a single stream using this expression. + - `fetch=10`: Returns only the first 10 rows +- `GlobalLimitExec` + - `skip=0`: Does not skip any rows + - `fetch=10`: Returns only the first 10 rows, denoted by `LIMIT 10` in the query. + +[partial aggregation]: https://docs.rs/datafusion/latest/datafusion/physical_plan/aggregates/enum.AggregateMode.html#variant.Partial +[physical plan]: https://docs.rs/datafusion/latest/datafusion/physical_plan/aggregates/struct.PhysicalGroupBy.html +[partitioning]: https://docs.rs/datafusion/latest/datafusion/physical_plan/repartition/struct.RepartitionExec.html +[topk]: https://docs.rs/datafusion/latest/datafusion/physical_plan/struct.TopK.html +[documentation on multi phase grouping]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.Accumulator.html#tymethod.state + +### Data in this Example + +The examples in this section use data from [ClickBench], a benchmark for data +analytics. The examples are in terms of the 14GB [`hits.parquet`] file and can be +downloaded from the website or using the following commands: + +```shell +cd benchmarks +./bench.sh data clickbench_1 +*************************** +DataFusion Benchmark Runner and Data Generator +COMMAND: data +BENCHMARK: clickbench_1 +DATA_DIR: /Users/andrewlamb/Software/datafusion2/benchmarks/data +CARGO_COMMAND: cargo run --release +PREFER_HASH_JOIN: true +*************************** +Checking hits.parquet...... found 14779976446 bytes ... Done +``` + +Then you can run `datafusion-cli` to get plans: + +```shell +cd datafusion/benchmarks/data +datafusion-cli + +DataFusion CLI v41.0.0 +> select count(*) from 'hits.parquet'; ++----------+ +| count(*) | ++----------+ +| 99997497 | ++----------+ +1 row(s) fetched. +Elapsed 0.062 seconds. +> +``` + +[clickbench]: https://benchmark.clickhouse.com/ +[`hits.parquet`]: https://datasets.clickhouse.com/hits_compatible/hits.parquet diff --git a/docs/source/user-guide/sql/explain.md b/docs/source/user-guide/sql/explain.md index 22f73e3d76d7..45bb3a57aa7c 100644 --- a/docs/source/user-guide/sql/explain.md +++ b/docs/source/user-guide/sql/explain.md @@ -21,6 +21,8 @@ The `EXPLAIN` command shows the logical and physical execution plan for the specified SQL statement. +See the [Reading Explain Plans](../explain-usage.md) page for more information on how to interpret these plans. +
 EXPLAIN [ANALYZE] [VERBOSE] statement
 
From 5163e1559f9d84d04bdd967adf34ad07e63b249d Mon Sep 17 00:00:00 2001 From: WeblWabl Date: Wed, 28 Aug 2024 07:29:16 -0500 Subject: [PATCH 21/75] Make RuntimeEnvBuilder rather than RuntimeConfig (#12157) * feat/12156: Make RuntimeEnvBuilder rather than RuntimeConfig Signed-off-by: Devan * feat/12156: Make RuntimeEnvBuilder rather than RuntimeConfig Signed-off-by: Devan * doc link Signed-off-by: Devan * update to use builder for rt env Signed-off-by: Devan * update to use builder Signed-off-by: Devan * clippy Signed-off-by: Devan * touch Signed-off-by: Devan * fmt Signed-off-by: Devan * revert some formatting that occurred Signed-off-by: Devan * revert some formatting that occurred Signed-off-by: Devan * use builder Signed-off-by: Devan * fmt Signed-off-by: Devan * Update datafusion/execution/src/runtime_env.rs Co-authored-by: Andrew Lamb --------- Signed-off-by: Devan Co-authored-by: Devan Co-authored-by: Andrew Lamb --- .../core/src/datasource/file_format/csv.rs | 4 +- datafusion/core/src/execution/context/mod.rs | 13 +++--- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 12 +++--- datafusion/core/tests/memory_limit/mod.rs | 12 +++--- .../core/tests/parquet/file_statistics.rs | 7 ++- datafusion/execution/src/runtime_env.rs | 27 +++++++++--- datafusion/execution/src/task.rs | 5 ++- .../physical-plan/src/aggregates/mod.rs | 15 ++++--- .../physical-plan/src/joins/cross_join.rs | 9 ++-- .../physical-plan/src/joins/hash_join.rs | 16 ++++--- .../src/joins/nested_loop_join.rs | 9 ++-- .../src/joins/sort_merge_join.rs | 43 +++++++++++-------- .../physical-plan/src/repartition/mod.rs | 6 ++- datafusion/physical-plan/src/sorts/sort.rs | 21 +++++---- datafusion/wasmtest/src/lib.rs | 13 +++--- 15 files changed, 129 insertions(+), 83 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index d1ce2afcccf3..e43f6ab29abc 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -680,7 +680,7 @@ mod tests { use datafusion_common::cast::as_string_array; use datafusion_common::internal_err; use datafusion_common::stats::Precision; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_expr::{col, lit}; use crate::execution::session_state::SessionStateBuilder; @@ -863,7 +863,7 @@ mod tests { async fn query_compress_data( file_compression_type: FileCompressionType, ) -> Result<()> { - let runtime = Arc::new(RuntimeEnv::new(RuntimeConfig::new()).unwrap()); + let runtime = Arc::new(RuntimeEnvBuilder::new().build()?); let mut cfg = SessionConfig::new(); cfg.options_mut().catalog.has_header = true; let session_state = SessionStateBuilder::new() diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 18943599b136..c67424c0fa53 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -212,15 +212,15 @@ where /// # use std::sync::Arc; /// # use datafusion::prelude::*; /// # use datafusion::execution::SessionStateBuilder; -/// # use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; +/// # use datafusion_execution::runtime_env::RuntimeEnvBuilder; /// // Configure a 4k batch size /// let config = SessionConfig::new() .with_batch_size(4 * 1024); /// /// // configure a memory limit of 1GB with 20% slop -/// let runtime_env = RuntimeEnv::new( -/// RuntimeConfig::new() +/// let runtime_env = RuntimeEnvBuilder::new() /// .with_memory_limit(1024 * 1024 * 1024, 0.80) -/// ).unwrap(); +/// .build() +/// .unwrap(); /// /// // Create a SessionState using the config and runtime_env /// let state = SessionStateBuilder::new() @@ -1623,7 +1623,7 @@ mod tests { use super::{super::options::CsvReadOptions, *}; use crate::assert_batches_eq; use crate::execution::memory_pool::MemoryConsumer; - use crate::execution::runtime_env::RuntimeConfig; + use crate::execution::runtime_env::RuntimeEnvBuilder; use crate::test; use crate::test_util::{plan_and_collect, populate_csv_partitions}; @@ -1758,8 +1758,7 @@ mod tests { let path = path.join("tests/tpch-csv"); let url = format!("file://{}", path.display()); - let rt_cfg = RuntimeConfig::new(); - let runtime = Arc::new(RuntimeEnv::new(rt_cfg).unwrap()); + let runtime = Arc::new(RuntimeEnvBuilder::new().build()?); let cfg = SessionConfig::new() .set_str("datafusion.catalog.location", url.as_str()) .set_str("datafusion.catalog.format", "CSV") diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index eda306dd3de5..1980589491a5 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -22,7 +22,7 @@ use arrow::{ compute::SortOptions, record_batch::RecordBatch, }; -use datafusion::execution::runtime_env::RuntimeConfig; +use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::sorts::sort::SortExec; @@ -136,10 +136,12 @@ impl SortTest { .sort_spill_reservation_bytes, ); - let runtime_env = RuntimeConfig::new() - .with_memory_pool(Arc::new(GreedyMemoryPool::new(pool_size))) - .build(); - let runtime = Arc::new(runtime_env.unwrap()); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_pool(Arc::new(GreedyMemoryPool::new(pool_size))) + .build() + .unwrap(), + ); SessionContext::new_with_config_rt(session_config, runtime) } else { SessionContext::new_with_config(session_config) diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index dbd5592e8020..592c25dedc50 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -40,7 +40,7 @@ use tokio::fs::File; use datafusion::datasource::streaming::StreamingTable; use datafusion::datasource::{MemTable, TableProvider}; use datafusion::execution::disk_manager::DiskManagerConfig; -use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; +use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::execution::session_state::SessionStateBuilder; use datafusion::physical_optimizer::join_selection::JoinSelection; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; @@ -509,17 +509,17 @@ impl TestCase { let table = scenario.table(); - let mut rt_config = RuntimeConfig::new() + let rt_config = RuntimeEnvBuilder::new() // disk manager setting controls the spilling .with_disk_manager(disk_manager_config) .with_memory_limit(memory_limit, MEMORY_FRACTION); - if let Some(pool) = memory_pool { - rt_config = rt_config.with_memory_pool(pool); + let runtime = if let Some(pool) = memory_pool { + rt_config.with_memory_pool(pool).build().unwrap() + } else { + rt_config.build().unwrap() }; - let runtime = RuntimeEnv::new(rt_config).unwrap(); - // Configure execution let builder = SessionStateBuilder::new() .with_config(config) diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index bf25b36f48e8..bd251f1a6669 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -33,7 +33,7 @@ use datafusion_execution::cache::cache_unit::{ DefaultFileStatisticsCache, DefaultListFilesCache, }; use datafusion_execution::config::SessionConfig; -use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; +use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion::execution::session_state::SessionStateBuilder; use tempfile::tempdir; @@ -198,7 +198,10 @@ fn get_cache_runtime_state() -> ( .with_list_files_cache(Some(list_file_cache.clone())); let rt = Arc::new( - RuntimeEnv::new(RuntimeConfig::new().with_cache_manager(cache_config)).unwrap(), + RuntimeEnvBuilder::new() + .with_cache_manager(cache_config) + .build() + .expect("could not build runtime environment"), ); let state = SessionContext::new_with_config_rt(SessionConfig::default(), rt).state(); diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index bddce81f537b..e7b48be95cff 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -41,7 +41,7 @@ use url::Url; /// Execution runtime environment that manages system resources such /// as memory, disk, cache and storage. /// -/// A [`RuntimeEnv`] is created from a [`RuntimeConfig`] and has the +/// A [`RuntimeEnv`] is created from a [`RuntimeEnvBuilder`] and has the /// following resource management functionality: /// /// * [`MemoryPool`]: Manage memory @@ -147,13 +147,17 @@ impl RuntimeEnv { impl Default for RuntimeEnv { fn default() -> Self { - RuntimeEnv::new(RuntimeConfig::new()).unwrap() + RuntimeEnvBuilder::new().build().unwrap() } } +/// Please see: +/// This a type alias for backwards compatibility. +pub type RuntimeConfig = RuntimeEnvBuilder; + #[derive(Clone)] /// Execution runtime configuration -pub struct RuntimeConfig { +pub struct RuntimeEnvBuilder { /// DiskManager to manage temporary disk file usage pub disk_manager: DiskManagerConfig, /// [`MemoryPool`] from which to allocate memory @@ -166,13 +170,13 @@ pub struct RuntimeConfig { pub object_store_registry: Arc, } -impl Default for RuntimeConfig { +impl Default for RuntimeEnvBuilder { fn default() -> Self { Self::new() } } -impl RuntimeConfig { +impl RuntimeEnvBuilder { /// New with default values pub fn new() -> Self { Self { @@ -229,8 +233,17 @@ impl RuntimeConfig { self.with_disk_manager(DiskManagerConfig::new_specified(vec![path.into()])) } - /// Build a `RuntimeEnv` object from the configuration + /// Build a RuntimeEnv pub fn build(self) -> Result { - RuntimeEnv::new(self) + let memory_pool = self + .memory_pool + .unwrap_or_else(|| Arc::new(UnboundedMemoryPool::default())); + + Ok(RuntimeEnv { + memory_pool, + disk_manager: DiskManager::try_new(self.disk_manager)?, + cache_manager: CacheManager::try_new(&self.cache_manager)?, + object_store_registry: self.object_store_registry, + }) } } diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index 21a644284c42..35689b8e08df 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -24,7 +24,7 @@ use crate::{ config::SessionConfig, memory_pool::MemoryPool, registry::FunctionRegistry, - runtime_env::{RuntimeConfig, RuntimeEnv}, + runtime_env::{RuntimeEnv, RuntimeEnvBuilder}, }; use datafusion_common::{plan_datafusion_err, DataFusionError, Result}; use datafusion_expr::planner::ExprPlanner; @@ -57,7 +57,8 @@ pub struct TaskContext { impl Default for TaskContext { fn default() -> Self { - let runtime = RuntimeEnv::new(RuntimeConfig::new()) + let runtime = RuntimeEnvBuilder::new() + .build() .expect("default runtime created successfully"); // Create a default task context, mostly useful for testing diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 6ca01928c80f..b5447a2d7317 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1212,7 +1212,7 @@ mod tests { }; use datafusion_execution::config::SessionConfig; use datafusion_execution::memory_pool::FairSpillPool; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_functions_aggregate::array_agg::array_agg_udaf; use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::count::count_udaf; @@ -1324,11 +1324,10 @@ mod tests { fn new_spill_ctx(batch_size: usize, max_memory: usize) -> Arc { let session_config = SessionConfig::new().with_batch_size(batch_size); let runtime = Arc::new( - RuntimeEnv::new( - RuntimeConfig::default() - .with_memory_pool(Arc::new(FairSpillPool::new(max_memory))), - ) - .unwrap(), + RuntimeEnvBuilder::default() + .with_memory_pool(Arc::new(FairSpillPool::new(max_memory))) + .build() + .unwrap(), ); let task_ctx = TaskContext::default() .with_session_config(session_config) @@ -1809,7 +1808,9 @@ mod tests { let input_schema = input.schema(); let runtime = Arc::new( - RuntimeEnv::new(RuntimeConfig::default().with_memory_limit(1, 1.0)).unwrap(), + RuntimeEnvBuilder::default() + .with_memory_limit(1, 1.0) + .build()?, ); let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 0868ee721665..b99d4f17c42a 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -488,7 +488,7 @@ mod tests { use crate::test::build_table_scan_i32; use datafusion_common::{assert_batches_sorted_eq, assert_contains}; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; async fn join_collect( left: Arc, @@ -673,8 +673,11 @@ mod tests { #[tokio::test] async fn test_overallocation() -> Result<()> { - let runtime_config = RuntimeConfig::new().with_memory_limit(100, 1.0); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build()?, + ); let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 7fac23ad5557..f20d00e1a298 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -1572,7 +1572,7 @@ mod tests { ScalarValue, }; use datafusion_execution::config::SessionConfig; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; @@ -3798,8 +3798,11 @@ mod tests { ]; for join_type in join_types { - let runtime_config = RuntimeConfig::new().with_memory_limit(100, 1.0); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build()?, + ); let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); @@ -3871,8 +3874,11 @@ mod tests { ]; for join_type in join_types { - let runtime_config = RuntimeConfig::new().with_memory_limit(100, 1.0); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build()?, + ); let session_config = SessionConfig::default().with_batch_size(50); let task_ctx = TaskContext::default() .with_session_config(session_config) diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 18de2de03192..3cd373544157 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -644,7 +644,7 @@ mod tests { use arrow::datatypes::{DataType, Field}; use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; use datafusion_physical_expr::{Partitioning, PhysicalExpr}; @@ -1019,8 +1019,11 @@ mod tests { ]; for join_type in join_types { - let runtime_config = RuntimeConfig::new().with_memory_limit(100, 1.0); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build()?, + ); let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 511cb4c55fcd..09fe5d9ebc54 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -1978,7 +1978,7 @@ mod tests { }; use datafusion_execution::config::SessionConfig; use datafusion_execution::disk_manager::DiskManagerConfig; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_execution::TaskContext; use crate::expressions::Column; @@ -2900,10 +2900,12 @@ mod tests { ]; // Disable DiskManager to prevent spilling - let runtime_config = RuntimeConfig::new() - .with_memory_limit(100, 1.0) - .with_disk_manager(DiskManagerConfig::Disabled); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .with_disk_manager(DiskManagerConfig::Disabled) + .build()?, + ); let session_config = SessionConfig::default().with_batch_size(50); for join_type in join_types { @@ -2985,10 +2987,12 @@ mod tests { ]; // Disable DiskManager to prevent spilling - let runtime_config = RuntimeConfig::new() - .with_memory_limit(100, 1.0) - .with_disk_manager(DiskManagerConfig::Disabled); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .with_disk_manager(DiskManagerConfig::Disabled) + .build()?, + ); let session_config = SessionConfig::default().with_batch_size(50); for join_type in join_types { @@ -3048,10 +3052,12 @@ mod tests { ]; // Enable DiskManager to allow spilling - let runtime_config = RuntimeConfig::new() - .with_memory_limit(100, 1.0) - .with_disk_manager(DiskManagerConfig::NewOs); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .with_disk_manager(DiskManagerConfig::NewOs) + .build()?, + ); for batch_size in [1, 50] { let session_config = SessionConfig::default().with_batch_size(batch_size); @@ -3156,10 +3162,13 @@ mod tests { ]; // Enable DiskManager to allow spilling - let runtime_config = RuntimeConfig::new() - .with_memory_limit(500, 1.0) - .with_disk_manager(DiskManagerConfig::NewOs); - let runtime = Arc::new(RuntimeEnv::new(runtime_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(500, 1.0) + .with_disk_manager(DiskManagerConfig::NewOs) + .build()?, + ); + for batch_size in [1, 50] { let session_config = SessionConfig::default().with_batch_size(batch_size); diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 5a3fcb5029e1..650006a9d02d 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1025,7 +1025,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::cast::as_string_array; use datafusion_common::{assert_batches_sorted_eq, exec_err}; - use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use tokio::task::JoinSet; @@ -1507,7 +1507,9 @@ mod tests { // setup up context let runtime = Arc::new( - RuntimeEnv::new(RuntimeConfig::default().with_memory_limit(1, 1.0)).unwrap(), + RuntimeEnvBuilder::default() + .with_memory_limit(1, 1.0) + .build()?, ); let task_ctx = TaskContext::default().with_runtime(runtime); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index a81b09948cca..e92a57493141 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -966,7 +966,7 @@ mod tests { use arrow::datatypes::*; use datafusion_common::cast::as_primitive_array; use datafusion_execution::config::SessionConfig; - use datafusion_execution::runtime_env::RuntimeConfig; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_common::ScalarValue; use datafusion_physical_expr::expressions::Literal; @@ -1009,9 +1009,11 @@ mod tests { .options() .execution .sort_spill_reservation_bytes; - let rt_config = RuntimeConfig::new() - .with_memory_limit(sort_spill_reservation_bytes + 12288, 1.0); - let runtime = Arc::new(RuntimeEnv::new(rt_config)?); + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit(sort_spill_reservation_bytes + 12288, 1.0) + .build()?, + ); let task_ctx = Arc::new( TaskContext::default() .with_session_config(session_config) @@ -1085,11 +1087,14 @@ mod tests { .execution .sort_spill_reservation_bytes; - let rt_config = RuntimeConfig::new().with_memory_limit( - sort_spill_reservation_bytes + avg_batch_size * (partitions - 1), - 1.0, + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_limit( + sort_spill_reservation_bytes + avg_batch_size * (partitions - 1), + 1.0, + ) + .build()?, ); - let runtime = Arc::new(RuntimeEnv::new(rt_config)?); let task_ctx = Arc::new( TaskContext::default() .with_runtime(runtime) diff --git a/datafusion/wasmtest/src/lib.rs b/datafusion/wasmtest/src/lib.rs index a74cce72ac64..50325d262d1d 100644 --- a/datafusion/wasmtest/src/lib.rs +++ b/datafusion/wasmtest/src/lib.rs @@ -78,9 +78,8 @@ mod test { use super::*; use datafusion::execution::context::SessionContext; use datafusion_execution::{ - config::SessionConfig, - disk_manager::DiskManagerConfig, - runtime_env::{RuntimeConfig, RuntimeEnv}, + config::SessionConfig, disk_manager::DiskManagerConfig, + runtime_env::RuntimeEnvBuilder, }; use datafusion_physical_plan::collect; use datafusion_sql::parser::DFParser; @@ -100,10 +99,10 @@ mod test { // Execute SQL (using datafusion) let rt = Arc::new( - RuntimeEnv::new( - RuntimeConfig::new().with_disk_manager(DiskManagerConfig::Disabled), - ) - .unwrap(), + RuntimeEnvBuilder::new() + .with_disk_manager(DiskManagerConfig::Disabled) + .build() + .unwrap(), ); let session_config = SessionConfig::new().with_target_partitions(1); let session_context = From 1fce2a98ef9c7f8dbd7f3dedcaf4aa069ab92154 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Wed, 28 Aug 2024 23:21:24 +0800 Subject: [PATCH 22/75] Minor: Fix grouping set typo (#12216) * Fix grouping set typo * More --- .../physical-optimizer/src/limited_distinct_aggregation.rs | 4 ++-- datafusion/physical-plan/src/aggregates/mod.rs | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-optimizer/src/limited_distinct_aggregation.rs b/datafusion/physical-optimizer/src/limited_distinct_aggregation.rs index e18e530072db..8653ad19da77 100644 --- a/datafusion/physical-optimizer/src/limited_distinct_aggregation.rs +++ b/datafusion/physical-optimizer/src/limited_distinct_aggregation.rs @@ -33,8 +33,8 @@ use itertools::Itertools; /// An optimizer rule that passes a `limit` hint into grouped aggregations which don't require all /// rows in the group to be processed for correctness. Example queries fitting this description are: -/// `SELECT distinct l_orderkey FROM lineitem LIMIT 10;` -/// `SELECT l_orderkey FROM lineitem GROUP BY l_orderkey LIMIT 10;` +/// - `SELECT distinct l_orderkey FROM lineitem LIMIT 10;` +/// - `SELECT l_orderkey FROM lineitem GROUP BY l_orderkey LIMIT 10;` pub struct LimitedDistinctAggregation {} impl LimitedDistinctAggregation { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index b5447a2d7317..0f33a9d7b992 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -111,9 +111,9 @@ impl AggregateMode { /// Represents `GROUP BY` clause in the plan (including the more general GROUPING SET) /// In the case of a simple `GROUP BY a, b` clause, this will contain the expression [a, b] /// and a single group [false, false]. -/// In the case of `GROUP BY GROUPING SET/CUBE/ROLLUP` the planner will expand the expression +/// In the case of `GROUP BY GROUPING SETS/CUBE/ROLLUP` the planner will expand the expression /// into multiple groups, using null expressions to align each group. -/// For example, with a group by clause `GROUP BY GROUPING SET ((a,b),(a),(b))` the planner should +/// For example, with a group by clause `GROUP BY GROUPING SETS ((a,b),(a),(b))` the planner should /// create a `PhysicalGroupBy` like /// ```text /// PhysicalGroupBy { @@ -134,7 +134,7 @@ pub struct PhysicalGroupBy { null_expr: Vec<(Arc, String)>, /// Null mask for each group in this grouping set. Each group is /// composed of either one of the group expressions in expr or a null - /// expression in null_expr. If `groups[i][j]` is true, then the the + /// expression in null_expr. If `groups[i][j]` is true, then the /// j-th expression in the i-th group is NULL, otherwise it is `expr[j]`. groups: Vec>, } From bd506980bd04c109d9fa979be5b627580e59d267 Mon Sep 17 00:00:00 2001 From: Austin Liu Date: Thu, 29 Aug 2024 12:49:04 +0800 Subject: [PATCH 23/75] Add `array_distance` function (#12211) * Add `distance` aggregation function Signed-off-by: Austin Liu Add `distance` aggregation function Signed-off-by: Austin Liu * Add sql logic test for `distance` Signed-off-by: Austin Liu * Simplify diff calculation Signed-off-by: Austin Liu * Add `array_distance`/`list_distance` as list function in functions-nested Signed-off-by: Austin Liu * Remove aggregate function `distance` Signed-off-by: Austin Liu * format Signed-off-by: Austin Liu * clean up error handling Signed-off-by: Austin Liu * Add `array_distance` in scalar array functions docs Signed-off-by: Austin Liu * Update bulletin Signed-off-by: Austin Liu * Prettify example Signed-off-by: Austin Liu --------- Signed-off-by: Austin Liu --- datafusion/functions-nested/src/distance.rs | 215 ++++++++++++++++++ datafusion/functions-nested/src/lib.rs | 3 + datafusion/sqllogictest/test_files/array.slt | 54 +++++ .../source/user-guide/sql/scalar_functions.md | 36 +++ 4 files changed, 308 insertions(+) create mode 100644 datafusion/functions-nested/src/distance.rs diff --git a/datafusion/functions-nested/src/distance.rs b/datafusion/functions-nested/src/distance.rs new file mode 100644 index 000000000000..4d779dc1fabf --- /dev/null +++ b/datafusion/functions-nested/src/distance.rs @@ -0,0 +1,215 @@ +// 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. + +//! [ScalarUDFImpl] definitions for array_distance function. + +use crate::utils::{downcast_arg, make_scalar_function}; +use arrow_array::{ + Array, ArrayRef, Float64Array, LargeListArray, ListArray, OffsetSizeTrait, +}; +use arrow_schema::DataType; +use arrow_schema::DataType::{FixedSizeList, Float64, LargeList, List}; +use core::any::type_name; +use datafusion_common::cast::{ + as_float32_array, as_float64_array, as_generic_list_array, as_int32_array, + as_int64_array, +}; +use datafusion_common::DataFusionError; +use datafusion_common::{exec_err, Result}; +use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; +use std::any::Any; +use std::sync::Arc; + +make_udf_expr_and_func!( + ArrayDistance, + array_distance, + array, + "returns the Euclidean distance between two numeric arrays.", + array_distance_udf +); + +#[derive(Debug)] +pub(super) struct ArrayDistance { + signature: Signature, + aliases: Vec, +} + +impl ArrayDistance { + pub fn new() -> Self { + Self { + signature: Signature::variadic_any(Volatility::Immutable), + aliases: vec!["list_distance".to_string()], + } + } +} + +impl ScalarUDFImpl for ArrayDistance { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "array_distance" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> Result { + match arg_types[0] { + List(_) | LargeList(_) | FixedSizeList(_, _) => Ok(Float64), + _ => exec_err!("The array_distance function can only accept List/LargeList/FixedSizeList."), + } + } + + fn invoke(&self, args: &[ColumnarValue]) -> Result { + make_scalar_function(array_distance_inner)(args) + } + + fn aliases(&self) -> &[String] { + &self.aliases + } +} + +pub fn array_distance_inner(args: &[ArrayRef]) -> Result { + if args.len() != 2 { + return exec_err!("array_distance expects exactly two arguments"); + } + + match (&args[0].data_type(), &args[1].data_type()) { + (List(_), List(_)) => general_array_distance::(args), + (LargeList(_), LargeList(_)) => general_array_distance::(args), + (array_type1, array_type2) => { + exec_err!("array_distance does not support types '{array_type1:?}' and '{array_type2:?}'") + } + } +} + +fn general_array_distance(arrays: &[ArrayRef]) -> Result { + let list_array1 = as_generic_list_array::(&arrays[0])?; + let list_array2 = as_generic_list_array::(&arrays[1])?; + + let result = list_array1 + .iter() + .zip(list_array2.iter()) + .map(|(arr1, arr2)| compute_array_distance(arr1, arr2)) + .collect::>()?; + + Ok(Arc::new(result) as ArrayRef) +} + +/// Computes the Euclidean distance between two arrays +fn compute_array_distance( + arr1: Option, + arr2: Option, +) -> Result> { + let value1 = match arr1 { + Some(arr) => arr, + None => return Ok(None), + }; + let value2 = match arr2 { + Some(arr) => arr, + None => return Ok(None), + }; + + let mut value1 = value1; + let mut value2 = value2; + + loop { + match value1.data_type() { + List(_) => { + if downcast_arg!(value1, ListArray).null_count() > 0 { + return Ok(None); + } + value1 = downcast_arg!(value1, ListArray).value(0); + } + LargeList(_) => { + if downcast_arg!(value1, LargeListArray).null_count() > 0 { + return Ok(None); + } + value1 = downcast_arg!(value1, LargeListArray).value(0); + } + _ => break, + } + + match value2.data_type() { + List(_) => { + if downcast_arg!(value2, ListArray).null_count() > 0 { + return Ok(None); + } + value2 = downcast_arg!(value2, ListArray).value(0); + } + LargeList(_) => { + if downcast_arg!(value2, LargeListArray).null_count() > 0 { + return Ok(None); + } + value2 = downcast_arg!(value2, LargeListArray).value(0); + } + _ => break, + } + } + + // Check for NULL values inside the arrays + if value1.null_count() != 0 || value2.null_count() != 0 { + return Ok(None); + } + + let values1 = convert_to_f64_array(&value1)?; + let values2 = convert_to_f64_array(&value2)?; + + if values1.len() != values2.len() { + return exec_err!("Both arrays must have the same length"); + } + + let sum_squares: f64 = values1 + .iter() + .zip(values2.iter()) + .map(|(v1, v2)| { + let diff = v1.unwrap_or(0.0) - v2.unwrap_or(0.0); + diff * diff + }) + .sum(); + + Ok(Some(sum_squares.sqrt())) +} + +/// Converts an array of any numeric type to a Float64Array. +fn convert_to_f64_array(array: &ArrayRef) -> Result { + match array.data_type() { + DataType::Float64 => Ok(as_float64_array(array)?.clone()), + DataType::Float32 => { + let array = as_float32_array(array)?; + let converted: Float64Array = + array.iter().map(|v| v.map(|v| v as f64)).collect(); + Ok(converted) + } + DataType::Int64 => { + let array = as_int64_array(array)?; + let converted: Float64Array = + array.iter().map(|v| v.map(|v| v as f64)).collect(); + Ok(converted) + } + DataType::Int32 => { + let array = as_int32_array(array)?; + let converted: Float64Array = + array.iter().map(|v| v.map(|v| v as f64)).collect(); + Ok(converted) + } + _ => exec_err!("Unsupported array type for conversion to Float64Array"), + } +} diff --git a/datafusion/functions-nested/src/lib.rs b/datafusion/functions-nested/src/lib.rs index cc0a7b55cf86..863b5a876adc 100644 --- a/datafusion/functions-nested/src/lib.rs +++ b/datafusion/functions-nested/src/lib.rs @@ -34,6 +34,7 @@ pub mod array_has; pub mod cardinality; pub mod concat; pub mod dimension; +pub mod distance; pub mod empty; pub mod except; pub mod expr_ext; @@ -73,6 +74,7 @@ pub mod expr_fn { pub use super::concat::array_prepend; pub use super::dimension::array_dims; pub use super::dimension::array_ndims; + pub use super::distance::array_distance; pub use super::empty::array_empty; pub use super::except::array_except; pub use super::extract::array_element; @@ -128,6 +130,7 @@ pub fn all_default_nested_functions() -> Vec> { array_has::array_has_any_udf(), empty::array_empty_udf(), length::array_length_udf(), + distance::array_distance_udf(), flatten::flatten_udf(), sort::array_sort_udf(), repeat::array_repeat_udf(), diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index c80fd7e92417..edc0cd7577e1 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -4715,6 +4715,60 @@ NULL 10 NULL 10 NULL 10 +query RRR +select array_distance([2], [3]), list_distance([1], [2]), list_distance([1], [-2]); +---- +1 1 3 + +query error +select list_distance([1], [1, 2]); + +query R +select array_distance([[1, 1]], [1, 2]); +---- +1 + +query R +select array_distance([[1, 1]], [[1, 2]]); +---- +1 + +query R +select array_distance([[1, 1]], [[1, 2]]); +---- +1 + +query RR +select array_distance([1, 1, 0, 0], [2, 2, 1, 1]), list_distance([1, 2, 3], [1, 2, 3]); +---- +2 0 + +query RR +select array_distance([1.0, 1, 0, 0], [2, 2.0, 1, 1]), list_distance([1, 2.0, 3], [1, 2, 3]); +---- +2 0 + +query R +select list_distance([1, 1, NULL, 0], [2, 2, NULL, NULL]); +---- +NULL + +query R +select list_distance([NULL, NULL], [NULL, NULL]); +---- +NULL + +query R +select list_distance([1.0, 2.0, 3.0], [1.0, 2.0, 3.5]) AS distance; +---- +0.5 + +query R +select list_distance([1, 2, 3], [1, 2, 3]) AS distance; +---- +0 + + ## array_dims (aliases: `list_dims`) # array dims error diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index c7b3409ba7cd..9569f4e65ff3 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -2093,6 +2093,7 @@ to_unixtime(expression[, ..., format_n]) - [array_concat](#array_concat) - [array_contains](#array_contains) - [array_dims](#array_dims) +- [array_distance](#array_distance) - [array_distinct](#array_distinct) - [array_has](#array_has) - [array_has_all](#array_has_all) @@ -2135,6 +2136,7 @@ to_unixtime(expression[, ..., format_n]) - [list_cat](#list_cat) - [list_concat](#list_concat) - [list_dims](#list_dims) +- [list_distance](#list_distance) - [list_distinct](#list_distinct) - [list_element](#list_element) - [list_except](#list_except) @@ -2388,6 +2390,36 @@ array_dims(array) - list_dims +### `array_distance` + +Returns the Euclidean distance between two input arrays of equal length. + +``` +array_distance(array1, array2) +``` + +#### Arguments + +- **array1**: Array expression. + Can be a constant, column, or function, and any combination of array operators. +- **array2**: Array expression. + Can be a constant, column, or function, and any combination of array operators. + +#### Example + +``` +> select array_distance([1, 2], [1, 4]); ++------------------------------------+ +| array_distance(List([1,2], [1,4])) | ++------------------------------------+ +| 2.0 | ++------------------------------------+ +``` + +#### Aliases + +- list_distance + ### `array_distinct` Returns distinct values from the array after removing duplicates. @@ -3224,6 +3256,10 @@ _Alias of [array_concat](#array_concat)._ _Alias of [array_dims](#array_dims)._ +### `list_distance` + +_Alias of [array_distance](#array_distance)._ + ### `list_distinct` _Alias of [array_dims](#array_distinct)._ From f5dcdf00318e00f6e8e3200d81c1239c330a94c0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Thu, 29 Aug 2024 14:25:36 +0300 Subject: [PATCH 24/75] Unbounded SortExec (and Top-K) Implementation When Req's Are Satisfied (#12174) * Sort fetch updates execution mode * Update sort.rs * Update sort.rs * Update sort.rs * Update sort.rs * Update sort.rs * Apply suggestions from code review * Update sort.rs * Update datafusion/physical-plan/src/sorts/sort.rs * Update datafusion/physical-plan/src/sorts/sort.rs * Reuse LimitStream --------- Co-authored-by: Mehmet Ozan Kabak --- .../physical-plan/src/metrics/baseline.rs | 2 +- datafusion/physical-plan/src/sorts/sort.rs | 311 ++++++++++++++---- .../src/windows/bounded_window_agg_exec.rs | 41 ++- 3 files changed, 265 insertions(+), 89 deletions(-) diff --git a/datafusion/physical-plan/src/metrics/baseline.rs b/datafusion/physical-plan/src/metrics/baseline.rs index dc345cd8cdcd..b26a08dd0fad 100644 --- a/datafusion/physical-plan/src/metrics/baseline.rs +++ b/datafusion/physical-plan/src/metrics/baseline.rs @@ -56,7 +56,7 @@ pub struct BaselineMetrics { } impl BaselineMetrics { - /// Create a new BaselineMetric structure, and set `start_time` to now + /// Create a new BaselineMetric structure, and set `start_time` to now pub fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self { let start_time = MetricBuilder::new(metrics).start_timestamp(partition); start_time.record(); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index e92a57493141..e0041194016c 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -26,6 +26,7 @@ use std::sync::Arc; use crate::common::spawn_buffered; use crate::expressions::PhysicalSortExpr; +use crate::limit::LimitStream; use crate::metrics::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, }; @@ -51,6 +52,7 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_physical_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::PhysicalSortRequirement; use futures::{StreamExt, TryStreamExt}; use log::{debug, trace}; @@ -737,9 +739,22 @@ impl SortExec { /// This can reduce the memory pressure required by the sort /// operation since rows that are not going to be included /// can be dropped. - pub fn with_fetch(mut self, fetch: Option) -> Self { - self.fetch = fetch; - self + pub fn with_fetch(&self, fetch: Option) -> Self { + let mut cache = self.cache.clone(); + if fetch.is_some() && self.cache.execution_mode == ExecutionMode::Unbounded { + // When a theoretically unnecessary sort becomes a top-K (which + // sometimes arises as an intermediate state before full removal), + // its execution mode should become `Bounded`. + cache.execution_mode = ExecutionMode::Bounded; + } + SortExec { + input: Arc::clone(&self.input), + expr: self.expr.clone(), + metrics_set: self.metrics_set.clone(), + preserve_partitioning: self.preserve_partitioning, + fetch, + cache, + } } /// Input schema @@ -775,6 +790,16 @@ impl SortExec { sort_exprs: LexOrdering, preserve_partitioning: bool, ) -> PlanProperties { + // Determine execution mode: + let sort_satisfied = input.equivalence_properties().ordering_satisfy_requirement( + PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()).as_slice(), + ); + let mode = match input.execution_mode() { + ExecutionMode::Unbounded if sort_satisfied => ExecutionMode::Unbounded, + ExecutionMode::Bounded => ExecutionMode::Bounded, + _ => ExecutionMode::PipelineBreaking, + }; + // Calculate equivalence properties; i.e. reset the ordering equivalence // class with the new ordering: let eq_properties = input @@ -786,14 +811,6 @@ impl SortExec { let output_partitioning = Self::output_partitioning_helper(input, preserve_partitioning); - // Determine execution mode: - let mode = match input.execution_mode() { - ExecutionMode::Unbounded | ExecutionMode::PipelineBreaking => { - ExecutionMode::PipelineBreaking - } - ExecutionMode::Bounded => ExecutionMode::Bounded, - }; - PlanProperties::new(eq_properties, output_partitioning, mode) } } @@ -874,53 +891,68 @@ impl ExecutionPlan for SortExec { trace!("End SortExec's input.execute for partition: {}", partition); - if let Some(fetch) = self.fetch.as_ref() { - let mut topk = TopK::try_new( - partition, - input.schema(), - self.expr.clone(), - *fetch, - context.session_config().batch_size(), - context.runtime_env(), - &self.metrics_set, - partition, - )?; - - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - futures::stream::once(async move { - while let Some(batch) = input.next().await { - let batch = batch?; - topk.insert_batch(batch)?; - } - topk.emit() - }) - .try_flatten(), - ))) - } else { - let mut sorter = ExternalSorter::new( - partition, - input.schema(), - self.expr.clone(), - context.session_config().batch_size(), - self.fetch, - execution_options.sort_spill_reservation_bytes, - execution_options.sort_in_place_threshold_bytes, - &self.metrics_set, - context.runtime_env(), + let sort_satisfied = self + .input + .equivalence_properties() + .ordering_satisfy_requirement( + PhysicalSortRequirement::from_sort_exprs(self.expr.iter()).as_slice(), ); - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - futures::stream::once(async move { - while let Some(batch) = input.next().await { - let batch = batch?; - sorter.insert_batch(batch).await?; - } - sorter.sort() - }) - .try_flatten(), - ))) + match (sort_satisfied, self.fetch.as_ref()) { + (true, Some(fetch)) => Ok(Box::pin(LimitStream::new( + input, + 0, + Some(*fetch), + BaselineMetrics::new(&self.metrics_set, partition), + ))), + (true, None) => Ok(input), + (false, Some(fetch)) => { + let mut topk = TopK::try_new( + partition, + input.schema(), + self.expr.clone(), + *fetch, + context.session_config().batch_size(), + context.runtime_env(), + &self.metrics_set, + partition, + )?; + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + futures::stream::once(async move { + while let Some(batch) = input.next().await { + let batch = batch?; + topk.insert_batch(batch)?; + } + topk.emit() + }) + .try_flatten(), + ))) + } + (false, None) => { + let mut sorter = ExternalSorter::new( + partition, + input.schema(), + self.expr.clone(), + context.session_config().batch_size(), + self.fetch, + execution_options.sort_spill_reservation_bytes, + execution_options.sort_in_place_threshold_bytes, + &self.metrics_set, + context.runtime_env(), + ); + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + futures::stream::once(async move { + while let Some(batch) = input.next().await { + let batch = batch?; + sorter.insert_batch(batch).await?; + } + sorter.sort() + }) + .try_flatten(), + ))) + } } } @@ -933,14 +965,7 @@ impl ExecutionPlan for SortExec { } fn with_fetch(&self, limit: Option) -> Option> { - Some(Arc::new(SortExec { - input: Arc::clone(&self.input), - expr: self.expr.clone(), - metrics_set: self.metrics_set.clone(), - preserve_partitioning: self.preserve_partitioning, - fetch: limit, - cache: self.cache.clone(), - })) + Some(Arc::new(SortExec::with_fetch(self, limit))) } fn fetch(&self) -> Option { @@ -951,6 +976,8 @@ impl ExecutionPlan for SortExec { #[cfg(test)] mod tests { use std::collections::HashMap; + use std::pin::Pin; + use std::task::{Context, Poll}; use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; @@ -965,12 +992,124 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::*; use datafusion_common::cast::as_primitive_array; + use datafusion_common::{assert_batches_eq, Result, ScalarValue}; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnvBuilder; + use datafusion_execution::RecordBatchStream; + use datafusion_physical_expr::expressions::{Column, Literal}; + use datafusion_physical_expr::EquivalenceProperties; + + use futures::{FutureExt, Stream}; + + #[derive(Debug, Clone)] + pub struct SortedUnboundedExec { + schema: Schema, + batch_size: u64, + cache: PlanProperties, + } + + impl DisplayAs for SortedUnboundedExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "UnboundableExec",).unwrap() + } + } + Ok(()) + } + } + + impl SortedUnboundedExec { + fn compute_properties(schema: SchemaRef) -> PlanProperties { + let mut eq_properties = EquivalenceProperties::new(schema); + eq_properties.add_new_orderings(vec![vec![PhysicalSortExpr::new( + Arc::new(Column::new("c1", 0)), + SortOptions::default(), + )]]); + let mode = ExecutionMode::Unbounded; + PlanProperties::new(eq_properties, Partitioning::UnknownPartitioning(1), mode) + } + } + + impl ExecutionPlan for SortedUnboundedExec { + fn name(&self) -> &'static str { + Self::static_name() + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn children(&self) -> Vec<&Arc> { + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> Result { + Ok(Box::pin(SortedUnboundedStream { + schema: Arc::new(self.schema.clone()), + batch_size: self.batch_size, + offset: 0, + })) + } + } + + #[derive(Debug)] + pub struct SortedUnboundedStream { + schema: SchemaRef, + batch_size: u64, + offset: u64, + } - use datafusion_common::ScalarValue; - use datafusion_physical_expr::expressions::Literal; - use futures::FutureExt; + impl Stream for SortedUnboundedStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + _cx: &mut Context<'_>, + ) -> Poll> { + let batch = SortedUnboundedStream::create_record_batch( + Arc::clone(&self.schema), + self.offset, + self.batch_size, + ); + self.offset += self.batch_size; + Poll::Ready(Some(Ok(batch))) + } + } + + impl RecordBatchStream for SortedUnboundedStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } + } + + impl SortedUnboundedStream { + fn create_record_batch( + schema: SchemaRef, + offset: u64, + batch_size: u64, + ) -> RecordBatch { + let values = (0..batch_size).map(|i| offset + i).collect::>(); + let array = UInt64Array::from(values); + let array_ref: ArrayRef = Arc::new(array); + RecordBatch::try_new(schema, vec![array_ref]).unwrap() + } + } #[tokio::test] async fn test_in_mem_sort() -> Result<()> { @@ -1414,4 +1553,42 @@ mod tests { let result = sort_batch(&batch, &expressions, None).unwrap(); assert_eq!(result.num_rows(), 1); } + + #[tokio::test] + async fn topk_unbounded_source() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + let schema = Schema::new(vec![Field::new("c1", DataType::UInt64, false)]); + let source = SortedUnboundedExec { + schema: schema.clone(), + batch_size: 2, + cache: SortedUnboundedExec::compute_properties(Arc::new(schema.clone())), + }; + let mut plan = SortExec::new( + vec![PhysicalSortExpr::new( + Arc::new(Column::new("c1", 0)), + SortOptions::default(), + )], + Arc::new(source), + ); + plan = plan.with_fetch(Some(9)); + + let batches = collect(Arc::new(plan), task_ctx).await?; + #[rustfmt::skip] + let expected = [ + "+----+", + "| c1 |", + "+----+", + "| 0 |", + "| 1 |", + "| 2 |", + "| 3 |", + "| 4 |", + "| 5 |", + "| 6 |", + "| 7 |", + "| 8 |", + "+----+",]; + assert_batches_eq!(expected, &batches); + Ok(()) + } } diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index c1bcd83a6fd2..001e134581c0 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -551,7 +551,7 @@ impl PartitionSearcher for LinearSearch { window_expr: &[Arc], ) -> Result> { let partition_bys = - self.evaluate_partition_by_column_values(record_batch, window_expr)?; + evaluate_partition_by_column_values(record_batch, window_expr)?; // NOTE: In Linear or PartiallySorted modes, we are sure that // `partition_bys` are not empty. // Calculate indices for each partition and construct a new record @@ -618,25 +618,6 @@ impl LinearSearch { } } - /// Calculates partition by expression results for each window expression - /// on `record_batch`. - fn evaluate_partition_by_column_values( - &self, - record_batch: &RecordBatch, - window_expr: &[Arc], - ) -> Result> { - window_expr[0] - .partition_by() - .iter() - .map(|item| match item.evaluate(record_batch)? { - ColumnarValue::Array(array) => Ok(array), - ColumnarValue::Scalar(scalar) => { - scalar.to_array_of_size(record_batch.num_rows()) - } - }) - .collect() - } - /// Calculate indices of each partition (according to PARTITION BY expression) /// `columns` contain partition by expression results. fn get_per_partition_indices( @@ -683,7 +664,7 @@ impl LinearSearch { window_expr: &[Arc], ) -> Result)>> { let partition_by_columns = - self.evaluate_partition_by_column_values(input_buffer, window_expr)?; + evaluate_partition_by_column_values(input_buffer, window_expr)?; // Reset the row_map state: self.row_map_out.clear(); let mut partition_indices: Vec<(PartitionKey, Vec)> = vec![]; @@ -852,6 +833,24 @@ impl SortedSearch { } } +/// Calculates partition by expression results for each window expression +/// on `record_batch`. +fn evaluate_partition_by_column_values( + record_batch: &RecordBatch, + window_expr: &[Arc], +) -> Result> { + window_expr[0] + .partition_by() + .iter() + .map(|item| match item.evaluate(record_batch)? { + ColumnarValue::Array(array) => Ok(array), + ColumnarValue::Scalar(scalar) => { + scalar.to_array_of_size(record_batch.num_rows()) + } + }) + .collect() +} + /// Stream for the bounded window aggregation plan. pub struct BoundedWindowAggStream { schema: SchemaRef, From 85adb6c4e6c0b6009f9866118c318b078263e118 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 29 Aug 2024 17:38:28 +0200 Subject: [PATCH 25/75] Remove Sort expression (`Expr::Sort`) (#12177) * Take Sort (SortExpr) in file options Part of effort to remove `Expr::Sort`. * Return Sort from Expr.Sort Part of effort to remove `Expr::Sort`. * Accept Sort (SortExpr) in `LogicalPlanBuilder.sort` Take `expr::Sort` in `LogicalPlanBuilder.sort`. Accept any `Expr` in new function, `LogicalPlanBuilder.sort_by` which apply default sort ordering. Part of effort to remove `Expr::Sort`. * Operate on `Sort` in to_substrait_sort_field / from_substrait_sorts Part of effort to remove `Expr::Sort`. * Take Sort (SortExpr) in tests' TopKPlanNode Part of effort to remove `Expr::Sort`. * Remove Sort expression (`Expr::Sort`) Remove sort as an expression, i.e. remove `Expr::Sort` from `Expr` enum. Use `expr::Sort` directly when sorting. The sort expression was used in context of ordering (sort, topk, create table, file sorting). Those places require their sort expression to be of type Sort anyway and no other expression was allowed, so this change improves static typing. Sort as an expression was illegal in other contexts. * use assert_eq just like in LogicalPlan.with_new_exprs * avoid clone in replace_sort_expressions * reduce cloning in EliminateDuplicatedExpr * restore SortExprWrapper this commit is longer than advised in the review comment, but after squashing the diff will be smaller * shorthand SortExprWrapper struct definition --- .../examples/file_stream_provider.rs | 4 +- datafusion/core/src/dataframe/mod.rs | 15 +- .../src/datasource/file_format/options.rs | 14 +- .../core/src/datasource/listing/helpers.rs | 3 +- .../core/src/datasource/listing/table.rs | 14 +- datafusion/core/src/datasource/memory.rs | 5 +- datafusion/core/src/datasource/mod.rs | 41 ++-- .../physical_plan/file_scan_config.rs | 2 +- datafusion/core/src/datasource/stream.rs | 6 +- datafusion/core/src/physical_planner.rs | 32 ++- datafusion/core/src/test_util/mod.rs | 4 +- datafusion/core/tests/dataframe/mod.rs | 20 +- datafusion/core/tests/expr_api/mod.rs | 17 +- datafusion/core/tests/fifo/mod.rs | 4 +- .../core/tests/fuzz_cases/limit_fuzz.rs | 2 +- .../tests/user_defined/user_defined_plan.rs | 9 +- datafusion/expr/src/expr.rs | 113 +++++------ datafusion/expr/src/expr_fn.rs | 28 +-- datafusion/expr/src/expr_rewriter/mod.rs | 34 ++-- datafusion/expr/src/expr_rewriter/order_by.rs | 38 ++-- datafusion/expr/src/expr_schema.rs | 15 +- datafusion/expr/src/logical_plan/builder.rs | 38 ++-- datafusion/expr/src/logical_plan/ddl.rs | 7 +- datafusion/expr/src/logical_plan/plan.rs | 41 ++-- datafusion/expr/src/logical_plan/tree_node.rs | 18 +- datafusion/expr/src/tree_node.rs | 53 ++++- datafusion/expr/src/utils.rs | 192 +++++++----------- datafusion/expr/src/window_frame.rs | 4 +- .../functions-aggregate/src/first_last.rs | 4 +- .../src/analyzer/count_wildcard_rule.rs | 2 +- .../optimizer/src/analyzer/type_coercion.rs | 7 +- .../optimizer/src/common_subexpr_eliminate.rs | 15 +- .../src/eliminate_duplicated_expr.rs | 30 +-- datafusion/optimizer/src/eliminate_limit.rs | 12 +- datafusion/optimizer/src/push_down_filter.rs | 3 +- datafusion/optimizer/src/push_down_limit.rs | 8 +- .../simplify_expressions/expr_simplifier.rs | 1 - .../src/single_distinct_to_groupby.rs | 4 +- datafusion/proto/proto/datafusion.proto | 17 +- datafusion/proto/src/generated/pbjson.rs | 105 ++++++++-- datafusion/proto/src/generated/prost.rs | 26 ++- .../proto/src/logical_plan/from_proto.rs | 49 +++-- datafusion/proto/src/logical_plan/mod.rs | 40 ++-- datafusion/proto/src/logical_plan/to_proto.rs | 48 +++-- .../tests/cases/roundtrip_logical_plan.rs | 10 +- datafusion/sql/src/expr/function.rs | 25 +-- datafusion/sql/src/expr/order_by.rs | 8 +- datafusion/sql/src/query.rs | 3 +- datafusion/sql/src/select.rs | 4 +- datafusion/sql/src/statement.rs | 13 +- datafusion/sql/src/unparser/expr.rs | 120 +++-------- datafusion/sql/src/unparser/mod.rs | 2 - datafusion/sql/src/unparser/plan.rs | 17 +- datafusion/sql/src/unparser/rewrite.rs | 37 ++-- .../substrait/src/logical_plan/consumer.rs | 12 +- .../substrait/src/logical_plan/producer.rs | 77 ++++--- .../using-the-dataframe-api.md | 4 +- 57 files changed, 704 insertions(+), 772 deletions(-) diff --git a/datafusion-examples/examples/file_stream_provider.rs b/datafusion-examples/examples/file_stream_provider.rs index 4db7e0200f53..e4fd937fd373 100644 --- a/datafusion-examples/examples/file_stream_provider.rs +++ b/datafusion-examples/examples/file_stream_provider.rs @@ -39,7 +39,7 @@ mod non_windows { use datafusion::datasource::TableProvider; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::{exec_err, Result}; - use datafusion_expr::Expr; + use datafusion_expr::SortExpr; // Number of lines written to FIFO const TEST_BATCH_SIZE: usize = 5; @@ -49,7 +49,7 @@ mod non_windows { fn fifo_table( schema: SchemaRef, path: impl Into, - sort: Vec>, + sort: Vec>, ) -> Arc { let source = FileStreamProvider::new_file(schema, path.into()) .with_batch_size(TEST_BATCH_SIZE) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index c516c7985d54..5dbeb535a546 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -52,7 +52,7 @@ use datafusion_common::config::{CsvOptions, JsonOptions}; use datafusion_common::{ plan_err, Column, DFSchema, DataFusionError, ParamValues, SchemaError, UnnestOptions, }; -use datafusion_expr::{case, is_null, lit}; +use datafusion_expr::{case, is_null, lit, SortExpr}; use datafusion_expr::{ utils::COUNT_STAR_EXPANSION, TableProviderFilterPushDown, UNNAMED_TABLE, }; @@ -577,7 +577,7 @@ impl DataFrame { self, on_expr: Vec, select_expr: Vec, - sort_expr: Option>, + sort_expr: Option>, ) -> Result { let plan = LogicalPlanBuilder::from(self.plan) .distinct_on(on_expr, select_expr, sort_expr)? @@ -776,6 +776,15 @@ impl DataFrame { }) } + /// Apply a sort by provided expressions with default direction + pub fn sort_by(self, expr: Vec) -> Result { + self.sort( + expr.into_iter() + .map(|e| e.sort(true, false)) + .collect::>(), + ) + } + /// Sort the DataFrame by the specified sorting expressions. /// /// Note that any expression can be turned into @@ -797,7 +806,7 @@ impl DataFrame { /// # Ok(()) /// # } /// ``` - pub fn sort(self, expr: Vec) -> Result { + pub fn sort(self, expr: Vec) -> Result { let plan = LogicalPlanBuilder::from(self.plan).sort(expr)?.build()?; Ok(DataFrame { session_state: self.session_state, diff --git a/datafusion/core/src/datasource/file_format/options.rs b/datafusion/core/src/datasource/file_format/options.rs index 552977baba17..db90262edbf8 100644 --- a/datafusion/core/src/datasource/file_format/options.rs +++ b/datafusion/core/src/datasource/file_format/options.rs @@ -31,7 +31,6 @@ use crate::datasource::{ }; use crate::error::Result; use crate::execution::context::{SessionConfig, SessionState}; -use crate::logical_expr::Expr; use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion_common::config::TableOptions; @@ -41,6 +40,7 @@ use datafusion_common::{ }; use async_trait::async_trait; +use datafusion_expr::SortExpr; /// Options that control the reading of CSV files. /// @@ -84,7 +84,7 @@ pub struct CsvReadOptions<'a> { /// File compression type pub file_compression_type: FileCompressionType, /// Indicates how the file is sorted - pub file_sort_order: Vec>, + pub file_sort_order: Vec>, } impl<'a> Default for CsvReadOptions<'a> { @@ -199,7 +199,7 @@ impl<'a> CsvReadOptions<'a> { } /// Configure if file has known sort order - pub fn file_sort_order(mut self, file_sort_order: Vec>) -> Self { + pub fn file_sort_order(mut self, file_sort_order: Vec>) -> Self { self.file_sort_order = file_sort_order; self } @@ -231,7 +231,7 @@ pub struct ParquetReadOptions<'a> { /// based on data in file. pub schema: Option<&'a Schema>, /// Indicates how the file is sorted - pub file_sort_order: Vec>, + pub file_sort_order: Vec>, } impl<'a> Default for ParquetReadOptions<'a> { @@ -278,7 +278,7 @@ impl<'a> ParquetReadOptions<'a> { } /// Configure if file has known sort order - pub fn file_sort_order(mut self, file_sort_order: Vec>) -> Self { + pub fn file_sort_order(mut self, file_sort_order: Vec>) -> Self { self.file_sort_order = file_sort_order; self } @@ -397,7 +397,7 @@ pub struct NdJsonReadOptions<'a> { /// Flag indicating whether this file may be unbounded (as in a FIFO file). pub infinite: bool, /// Indicates how the file is sorted - pub file_sort_order: Vec>, + pub file_sort_order: Vec>, } impl<'a> Default for NdJsonReadOptions<'a> { @@ -452,7 +452,7 @@ impl<'a> NdJsonReadOptions<'a> { } /// Configure if file has known sort order - pub fn file_sort_order(mut self, file_sort_order: Vec>) -> Self { + pub fn file_sort_order(mut self, file_sort_order: Vec>) -> Self { self.file_sort_order = file_sort_order; self } diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index f6e938b72dab..dbeaf5dfcc36 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -102,11 +102,10 @@ pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { } // TODO other expressions are not handled yet: - // - AGGREGATE, WINDOW and SORT should not end up in filter conditions, except maybe in some edge cases + // - AGGREGATE and WINDOW should not end up in filter conditions, except maybe in some edge cases // - Can `Wildcard` be considered as a `Literal`? // - ScalarVariable could be `applicable`, but that would require access to the context Expr::AggregateFunction { .. } - | Expr::Sort { .. } | Expr::WindowFunction { .. } | Expr::Wildcard { .. } | Expr::Unnest { .. } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index a0345a38e40c..1f5fa738b253 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -33,8 +33,8 @@ use crate::datasource::{ use crate::execution::context::SessionState; use datafusion_catalog::TableProvider; use datafusion_common::{DataFusionError, Result}; -use datafusion_expr::TableType; use datafusion_expr::{utils::conjunction, Expr, TableProviderFilterPushDown}; +use datafusion_expr::{SortExpr, TableType}; use datafusion_physical_plan::{empty::EmptyExec, ExecutionPlan, Statistics}; use arrow::datatypes::{DataType, Field, SchemaBuilder, SchemaRef}; @@ -222,7 +222,7 @@ pub struct ListingOptions { /// ordering (encapsulated by a `Vec`). If there aren't /// multiple equivalent orderings, the outer `Vec` will have a /// single element. - pub file_sort_order: Vec>, + pub file_sort_order: Vec>, } impl ListingOptions { @@ -385,7 +385,7 @@ impl ListingOptions { /// /// assert_eq!(listing_options.file_sort_order, file_sort_order); /// ``` - pub fn with_file_sort_order(mut self, file_sort_order: Vec>) -> Self { + pub fn with_file_sort_order(mut self, file_sort_order: Vec>) -> Self { self.file_sort_order = file_sort_order; self } @@ -909,8 +909,7 @@ impl TableProvider for ListingTable { keep_partition_by_columns, }; - let unsorted: Vec> = vec![]; - let order_requirements = if self.options().file_sort_order != unsorted { + let order_requirements = if !self.options().file_sort_order.is_empty() { // Multiple sort orders in outer vec are equivalent, so we pass only the first one let ordering = self .try_create_output_ordering()? @@ -1160,11 +1159,6 @@ mod tests { // (file_sort_order, expected_result) let cases = vec![ (vec![], Ok(vec![])), - // not a sort expr - ( - vec![vec![col("string_col")]], - Err("Expected Expr::Sort in output_ordering, but got string_col"), - ), // sort expr, but non column ( vec![vec![ diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 44e01e71648a..cef7f210e118 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -43,6 +43,7 @@ use datafusion_physical_plan::metrics::MetricsSet; use async_trait::async_trait; use datafusion_catalog::Session; +use datafusion_expr::SortExpr; use futures::StreamExt; use log::debug; use parking_lot::Mutex; @@ -64,7 +65,7 @@ pub struct MemTable { column_defaults: HashMap, /// Optional pre-known sort order(s). Must be `SortExpr`s. /// inserting data into this table removes the order - pub sort_order: Arc>>>, + pub sort_order: Arc>>>, } impl MemTable { @@ -118,7 +119,7 @@ impl MemTable { /// /// Note that multiple sort orders are supported, if some are known to be /// equivalent, - pub fn with_sort_order(self, mut sort_order: Vec>) -> Self { + pub fn with_sort_order(self, mut sort_order: Vec>) -> Self { std::mem::swap(self.sort_order.lock().as_mut(), &mut sort_order); self } diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 1c9924735735..55e88e572be1 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -50,38 +50,39 @@ pub use statistics::get_statistics_with_limit; use arrow_schema::{Schema, SortOptions}; use datafusion_common::{plan_err, Result}; -use datafusion_expr::Expr; +use datafusion_expr::{Expr, SortExpr}; use datafusion_physical_expr::{expressions, LexOrdering, PhysicalSortExpr}; fn create_ordering( schema: &Schema, - sort_order: &[Vec], + sort_order: &[Vec], ) -> Result> { let mut all_sort_orders = vec![]; for exprs in sort_order { // Construct PhysicalSortExpr objects from Expr objects: let mut sort_exprs = vec![]; - for expr in exprs { - match expr { - Expr::Sort(sort) => match sort.expr.as_ref() { - Expr::Column(col) => match expressions::col(&col.name, schema) { - Ok(expr) => { - sort_exprs.push(PhysicalSortExpr { - expr, - options: SortOptions { - descending: !sort.asc, - nulls_first: sort.nulls_first, - }, - }); - } - // Cannot find expression in the projected_schema, stop iterating - // since rest of the orderings are violated - Err(_) => break, + for sort in exprs { + match sort.expr.as_ref() { + Expr::Column(col) => match expressions::col(&col.name, schema) { + Ok(expr) => { + sort_exprs.push(PhysicalSortExpr { + expr, + options: SortOptions { + descending: !sort.asc, + nulls_first: sort.nulls_first, + }, + }); } - expr => return plan_err!("Expected single column references in output_ordering, got {expr}"), + // Cannot find expression in the projected_schema, stop iterating + // since rest of the orderings are violated + Err(_) => break, + }, + expr => { + return plan_err!( + "Expected single column references in output_ordering, got {expr}" + ) } - expr => return plan_err!("Expected Expr::Sort in output_ordering, but got {expr}"), } } if !sort_exprs.is_empty() { diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index bfa5488e5b5e..3ea467539adc 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -979,7 +979,7 @@ mod tests { name: &'static str, file_schema: Schema, files: Vec, - sort: Vec, + sort: Vec, expected_result: Result>, &'static str>, } diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index b53fe8663178..ef6d195cdaff 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -33,7 +33,7 @@ use arrow_schema::SchemaRef; use datafusion_common::{config_err, plan_err, Constraints, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_expr::{CreateExternalTable, Expr, TableType}; +use datafusion_expr::{CreateExternalTable, Expr, SortExpr, TableType}; use datafusion_physical_plan::insert::{DataSink, DataSinkExec}; use datafusion_physical_plan::metrics::MetricsSet; use datafusion_physical_plan::stream::RecordBatchReceiverStreamBuilder; @@ -248,7 +248,7 @@ impl StreamProvider for FileStreamProvider { #[derive(Debug)] pub struct StreamConfig { source: Arc, - order: Vec>, + order: Vec>, constraints: Constraints, } @@ -263,7 +263,7 @@ impl StreamConfig { } /// Specify a sort order for the stream - pub fn with_order(mut self, order: Vec>) -> Self { + pub fn with_order(mut self, order: Vec>) -> Self { self.order = order; self } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index fe8d79846630..82405dd98e30 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -73,13 +73,13 @@ use datafusion_common::{ }; use datafusion_expr::dml::CopyTo; use datafusion_expr::expr::{ - self, physical_name, AggregateFunction, Alias, GroupingSet, WindowFunction, + physical_name, AggregateFunction, Alias, GroupingSet, WindowFunction, }; use datafusion_expr::expr_rewriter::unnormalize_cols; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; use datafusion_expr::{ - DescribeTable, DmlStatement, Extension, Filter, RecursiveQuery, StringifiedPlan, - WindowFrame, WindowFrameBound, WriteOp, + DescribeTable, DmlStatement, Extension, Filter, RecursiveQuery, SortExpr, + StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, }; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::Literal; @@ -1641,31 +1641,27 @@ pub fn create_aggregate_expr_and_maybe_filter( /// Create a physical sort expression from a logical expression pub fn create_physical_sort_expr( - e: &Expr, + e: &SortExpr, input_dfschema: &DFSchema, execution_props: &ExecutionProps, ) -> Result { - if let Expr::Sort(expr::Sort { + let SortExpr { expr, asc, nulls_first, - }) = e - { - Ok(PhysicalSortExpr { - expr: create_physical_expr(expr, input_dfschema, execution_props)?, - options: SortOptions { - descending: !asc, - nulls_first: *nulls_first, - }, - }) - } else { - internal_err!("Expects a sort expression") - } + } = e; + Ok(PhysicalSortExpr { + expr: create_physical_expr(expr, input_dfschema, execution_props)?, + options: SortOptions { + descending: !asc, + nulls_first: *nulls_first, + }, + }) } /// Create vector of physical sort expression from a vector of logical expression pub fn create_physical_sort_exprs( - exprs: &[Expr], + exprs: &[SortExpr], input_dfschema: &DFSchema, execution_props: &ExecutionProps, ) -> Result { diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index faa9378535fd..dd8b697666ee 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -46,7 +46,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::TableReference; use datafusion_expr::utils::COUNT_STAR_EXPANSION; -use datafusion_expr::{CreateExternalTable, Expr, TableType}; +use datafusion_expr::{CreateExternalTable, Expr, SortExpr, TableType}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::{expressions, EquivalenceProperties, PhysicalExpr}; @@ -360,7 +360,7 @@ pub fn register_unbounded_file_with_ordering( schema: SchemaRef, file_path: &Path, table_name: &str, - file_sort_order: Vec>, + file_sort_order: Vec>, ) -> Result<()> { let source = FileStreamProvider::new_file(schema, file_path.into()); let config = StreamConfig::new(Arc::new(source)).with_order(file_sort_order); diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 86cacbaa06d8..c5b9db7588e9 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -184,7 +184,7 @@ async fn test_count_wildcard_on_window() -> Result<()> { WindowFunctionDefinition::AggregateUDF(count_udaf()), vec![wildcard()], )) - .order_by(vec![Expr::Sort(Sort::new(Box::new(col("a")), false, true))]) + .order_by(vec![Sort::new(Box::new(col("a")), false, true)]) .window_frame(WindowFrame::new_bounds( WindowFrameUnits::Range, WindowFrameBound::Preceding(ScalarValue::UInt32(Some(6))), @@ -352,7 +352,7 @@ async fn sort_on_unprojected_columns() -> Result<()> { .unwrap() .select(vec![col("a")]) .unwrap() - .sort(vec![Expr::Sort(Sort::new(Box::new(col("b")), false, true))]) + .sort(vec![Sort::new(Box::new(col("b")), false, true)]) .unwrap(); let results = df.collect().await.unwrap(); @@ -396,7 +396,7 @@ async fn sort_on_distinct_columns() -> Result<()> { .unwrap() .distinct() .unwrap() - .sort(vec![Expr::Sort(Sort::new(Box::new(col("a")), false, true))]) + .sort(vec![Sort::new(Box::new(col("a")), false, true)]) .unwrap(); let results = df.collect().await.unwrap(); @@ -435,7 +435,7 @@ async fn sort_on_distinct_unprojected_columns() -> Result<()> { .await? .select(vec![col("a")])? .distinct()? - .sort(vec![Expr::Sort(Sort::new(Box::new(col("b")), false, true))]) + .sort(vec![Sort::new(Box::new(col("b")), false, true)]) .unwrap_err(); assert_eq!(err.strip_backtrace(), "Error during planning: For SELECT DISTINCT, ORDER BY expressions b must appear in select list"); Ok(()) @@ -599,8 +599,8 @@ async fn test_grouping_sets() -> Result<()> { .await? .aggregate(vec![grouping_set_expr], vec![count(col("a"))])? .sort(vec![ - Expr::Sort(Sort::new(Box::new(col("a")), false, true)), - Expr::Sort(Sort::new(Box::new(col("b")), false, true)), + Sort::new(Box::new(col("a")), false, true), + Sort::new(Box::new(col("b")), false, true), ])?; let results = df.collect().await?; @@ -640,8 +640,8 @@ async fn test_grouping_sets_count() -> Result<()> { .await? .aggregate(vec![grouping_set_expr], vec![count(lit(1))])? .sort(vec![ - Expr::Sort(Sort::new(Box::new(col("c1")), false, true)), - Expr::Sort(Sort::new(Box::new(col("c2")), false, true)), + Sort::new(Box::new(col("c1")), false, true), + Sort::new(Box::new(col("c2")), false, true), ])?; let results = df.collect().await?; @@ -687,8 +687,8 @@ async fn test_grouping_set_array_agg_with_overflow() -> Result<()> { ], )? .sort(vec![ - Expr::Sort(Sort::new(Box::new(col("c1")), false, true)), - Expr::Sort(Sort::new(Box::new(col("c2")), false, true)), + Sort::new(Box::new(col("c1")), false, true), + Sort::new(Box::new(col("c2")), false, true), ])?; let results = df.collect().await?; diff --git a/datafusion/core/tests/expr_api/mod.rs b/datafusion/core/tests/expr_api/mod.rs index 051d65652633..cbd892672152 100644 --- a/datafusion/core/tests/expr_api/mod.rs +++ b/datafusion/core/tests/expr_api/mod.rs @@ -20,7 +20,7 @@ use arrow_array::builder::{ListBuilder, StringBuilder}; use arrow_array::{ArrayRef, Int64Array, RecordBatch, StringArray, StructArray}; use arrow_schema::{DataType, Field}; use datafusion::prelude::*; -use datafusion_common::{assert_contains, DFSchema, ScalarValue}; +use datafusion_common::{DFSchema, ScalarValue}; use datafusion_expr::ExprFunctionExt; use datafusion_functions::core::expr_ext::FieldAccessor; use datafusion_functions_aggregate::first_last::first_value_udaf; @@ -167,21 +167,6 @@ fn test_list_range() { ); } -#[tokio::test] -async fn test_aggregate_error() { - let err = first_value_udaf() - .call(vec![col("props")]) - // not a sort column - .order_by(vec![col("id")]) - .build() - .unwrap_err() - .to_string(); - assert_contains!( - err, - "Error during planning: ORDER BY expressions must be Expr::Sort" - ); -} - #[tokio::test] async fn test_aggregate_ext_order_by() { let agg = first_value_udaf().call(vec![col("props")]); diff --git a/datafusion/core/tests/fifo/mod.rs b/datafusion/core/tests/fifo/mod.rs index 6efbb9b029de..cb587e3510c2 100644 --- a/datafusion/core/tests/fifo/mod.rs +++ b/datafusion/core/tests/fifo/mod.rs @@ -38,7 +38,7 @@ mod unix_test { }; use datafusion_common::instant::Instant; use datafusion_common::{exec_err, Result}; - use datafusion_expr::Expr; + use datafusion_expr::SortExpr; use futures::StreamExt; use nix::sys::stat; @@ -51,7 +51,7 @@ mod unix_test { fn fifo_table( schema: SchemaRef, path: impl Into, - sort: Vec>, + sort: Vec>, ) -> Arc { let source = FileStreamProvider::new_file(schema, path.into()) .with_batch_size(TEST_BATCH_SIZE) diff --git a/datafusion/core/tests/fuzz_cases/limit_fuzz.rs b/datafusion/core/tests/fuzz_cases/limit_fuzz.rs index 9889ce2ae562..95d97709f319 100644 --- a/datafusion/core/tests/fuzz_cases/limit_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/limit_fuzz.rs @@ -226,7 +226,7 @@ impl SortedData { } /// Return the sort expression to use for this data, depending on the type - fn sort_expr(&self) -> Vec { + fn sort_expr(&self) -> Vec { match self { Self::I32 { .. } | Self::F64 { .. } | Self::Str { .. } => { vec![datafusion_expr::col("x").sort(true, true)] diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 62ba113da0d3..da27cf8869d1 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -97,7 +97,8 @@ use datafusion::{ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::ScalarValue; -use datafusion_expr::Projection; +use datafusion_expr::tree_node::replace_sort_expression; +use datafusion_expr::{Projection, SortExpr}; use datafusion_optimizer::optimizer::ApplyOrder; use datafusion_optimizer::AnalyzerRule; @@ -392,7 +393,7 @@ struct TopKPlanNode { input: LogicalPlan, /// The sort expression (this example only supports a single sort /// expr) - expr: Expr, + expr: SortExpr, } impl Debug for TopKPlanNode { @@ -418,7 +419,7 @@ impl UserDefinedLogicalNodeCore for TopKPlanNode { } fn expressions(&self) -> Vec { - vec![self.expr.clone()] + vec![self.expr.expr.as_ref().clone()] } /// For example: `TopK: k=10` @@ -436,7 +437,7 @@ impl UserDefinedLogicalNodeCore for TopKPlanNode { Ok(Self { k: self.k, input: inputs.swap_remove(0), - expr: exprs.swap_remove(0), + expr: replace_sort_expression(self.expr.clone(), exprs.swap_remove(0)), }) } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 85ba80396c8e..b81c02ccd0b7 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -289,10 +289,6 @@ pub enum Expr { /// Casts the expression to a given type and will return a null value if the expression cannot be cast. /// This expression is guaranteed to have a fixed type. TryCast(TryCast), - /// A sort expression, that can be used to sort values. - /// - /// See [Expr::sort] for more details - Sort(Sort), /// Represents the call of a scalar function with a set of arguments. ScalarFunction(ScalarFunction), /// Calls an aggregate function with arguments, and optional @@ -633,6 +629,23 @@ impl Sort { } } +impl Display for Sort { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.expr)?; + if self.asc { + write!(f, " ASC")?; + } else { + write!(f, " DESC")?; + } + if self.nulls_first { + write!(f, " NULLS FIRST")?; + } else { + write!(f, " NULLS LAST")?; + } + Ok(()) + } +} + /// Aggregate function /// /// See also [`ExprFunctionExt`] to set these fields on `Expr` @@ -649,7 +662,7 @@ pub struct AggregateFunction { /// Optional filter pub filter: Option>, /// Optional ordering - pub order_by: Option>, + pub order_by: Option>, pub null_treatment: Option, } @@ -660,7 +673,7 @@ impl AggregateFunction { args: Vec, distinct: bool, filter: Option>, - order_by: Option>, + order_by: Option>, null_treatment: Option, ) -> Self { Self { @@ -785,7 +798,7 @@ pub struct WindowFunction { /// List of partition by expressions pub partition_by: Vec, /// List of order by expressions - pub order_by: Vec, + pub order_by: Vec, /// Window frame pub window_frame: window_frame::WindowFrame, /// Specifies how NULL value is treated: ignore or respect @@ -1141,7 +1154,6 @@ impl Expr { Expr::ScalarFunction(..) => "ScalarFunction", Expr::ScalarSubquery { .. } => "ScalarSubquery", Expr::ScalarVariable(..) => "ScalarVariable", - Expr::Sort { .. } => "Sort", Expr::TryCast { .. } => "TryCast", Expr::WindowFunction { .. } => "WindowFunction", Expr::Wildcard { .. } => "Wildcard", @@ -1227,14 +1239,9 @@ impl Expr { Expr::Like(Like::new(true, Box::new(self), Box::new(other), None, true)) } - /// Return the name to use for the specific Expr, recursing into - /// `Expr::Sort` as appropriate + /// Return the name to use for the specific Expr pub fn name_for_alias(&self) -> Result { - match self { - // call Expr::display_name() on a Expr::Sort will throw an error - Expr::Sort(Sort { expr, .. }) => expr.name_for_alias(), - expr => Ok(expr.schema_name().to_string()), - } + Ok(self.schema_name().to_string()) } /// Ensure `expr` has the name as `original_name` by adding an @@ -1250,14 +1257,7 @@ impl Expr { /// Return `self AS name` alias expression pub fn alias(self, name: impl Into) -> Expr { - match self { - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => Expr::Sort(Sort::new(Box::new(expr.alias(name)), asc, nulls_first)), - _ => Expr::Alias(Alias::new(self, None::<&str>, name.into())), - } + Expr::Alias(Alias::new(self, None::<&str>, name.into())) } /// Return `self AS name` alias expression with a specific qualifier @@ -1266,18 +1266,7 @@ impl Expr { relation: Option>, name: impl Into, ) -> Expr { - match self { - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => Expr::Sort(Sort::new( - Box::new(expr.alias_qualified(relation, name)), - asc, - nulls_first, - )), - _ => Expr::Alias(Alias::new(self, relation, name.into())), - } + Expr::Alias(Alias::new(self, relation, name.into())) } /// Remove an alias from an expression if one exists. @@ -1372,14 +1361,14 @@ impl Expr { Expr::IsNotNull(Box::new(self)) } - /// Create a sort expression from an existing expression. + /// Create a sort configuration from an existing expression. /// /// ``` /// # use datafusion_expr::col; /// let sort_expr = col("foo").sort(true, true); // SORT ASC NULLS_FIRST /// ``` - pub fn sort(self, asc: bool, nulls_first: bool) -> Expr { - Expr::Sort(Sort::new(Box::new(self), asc, nulls_first)) + pub fn sort(self, asc: bool, nulls_first: bool) -> Sort { + Sort::new(Box::new(self), asc, nulls_first) } /// Return `IsTrue(Box(self))` @@ -1655,7 +1644,6 @@ impl Expr { | Expr::Wildcard { .. } | Expr::WindowFunction(..) | Expr::Literal(..) - | Expr::Sort(..) | Expr::Placeholder(..) => false, } } @@ -1752,14 +1740,6 @@ impl Expr { }) => { data_type.hash(hasher); } - Expr::Sort(Sort { - expr: _expr, - asc, - nulls_first, - }) => { - asc.hash(hasher); - nulls_first.hash(hasher); - } Expr::ScalarFunction(ScalarFunction { func, args: _args }) => { func.hash(hasher); } @@ -1871,7 +1851,6 @@ impl<'a> Display for SchemaDisplay<'a> { Expr::Column(_) | Expr::Literal(_) | Expr::ScalarVariable(..) - | Expr::Sort(_) | Expr::OuterReferenceColumn(..) | Expr::Placeholder(_) | Expr::Wildcard { .. } => write!(f, "{}", self.0), @@ -1901,7 +1880,7 @@ impl<'a> Display for SchemaDisplay<'a> { }; if let Some(order_by) = order_by { - write!(f, " ORDER BY [{}]", schema_name_from_exprs(order_by)?)?; + write!(f, " ORDER BY [{}]", schema_name_from_sorts(order_by)?)?; }; Ok(()) @@ -2107,7 +2086,7 @@ impl<'a> Display for SchemaDisplay<'a> { } if !order_by.is_empty() { - write!(f, " ORDER BY [{}]", schema_name_from_exprs(order_by)?)?; + write!(f, " ORDER BY [{}]", schema_name_from_sorts(order_by)?)?; }; write!(f, " {window_frame}") @@ -2144,6 +2123,24 @@ fn schema_name_from_exprs_inner(exprs: &[Expr], sep: &str) -> Result Result { + let mut s = String::new(); + for (i, e) in sorts.iter().enumerate() { + if i > 0 { + write!(&mut s, ", ")?; + } + let ordering = if e.asc { "ASC" } else { "DESC" }; + let nulls_ordering = if e.nulls_first { + "NULLS FIRST" + } else { + "NULLS LAST" + }; + write!(&mut s, "{} {} {}", e.expr, ordering, nulls_ordering)?; + } + + Ok(s) +} + /// Format expressions for display as part of a logical plan. In many cases, this will produce /// similar output to `Expr.name()` except that column names will be prefixed with '#'. impl fmt::Display for Expr { @@ -2203,22 +2200,6 @@ impl fmt::Display for Expr { }) => write!(f, "{expr} IN ({subquery:?})"), Expr::ScalarSubquery(subquery) => write!(f, "({subquery:?})"), Expr::BinaryExpr(expr) => write!(f, "{expr}"), - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => { - if *asc { - write!(f, "{expr} ASC")?; - } else { - write!(f, "{expr} DESC")?; - } - if *nulls_first { - write!(f, " NULLS FIRST") - } else { - write!(f, " NULLS LAST") - } - } Expr::ScalarFunction(fun) => { fmt_function(f, fun.name(), false, &fun.args, true) } diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 1e0b601146dd..8d01712b95ad 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -26,9 +26,9 @@ use crate::function::{ StateFieldsArgs, }; use crate::{ - conditional_expressions::CaseBuilder, logical_plan::Subquery, AggregateUDF, Expr, - LogicalPlan, Operator, ScalarFunctionImplementation, ScalarUDF, Signature, - Volatility, + conditional_expressions::CaseBuilder, expr::Sort, logical_plan::Subquery, + AggregateUDF, Expr, LogicalPlan, Operator, ScalarFunctionImplementation, ScalarUDF, + Signature, Volatility, }; use crate::{ AggregateUDFImpl, ColumnarValue, ScalarUDFImpl, WindowFrame, WindowUDF, WindowUDFImpl, @@ -723,9 +723,7 @@ pub fn interval_month_day_nano_lit(value: &str) -> Expr { /// ``` pub trait ExprFunctionExt { /// Add `ORDER BY ` - /// - /// Note: `order_by` must be [`Expr::Sort`] - fn order_by(self, order_by: Vec) -> ExprFuncBuilder; + fn order_by(self, order_by: Vec) -> ExprFuncBuilder; /// Add `FILTER ` fn filter(self, filter: Expr) -> ExprFuncBuilder; /// Add `DISTINCT` @@ -753,7 +751,7 @@ pub enum ExprFuncKind { #[derive(Debug, Clone)] pub struct ExprFuncBuilder { fun: Option, - order_by: Option>, + order_by: Option>, filter: Option, distinct: bool, null_treatment: Option, @@ -798,16 +796,6 @@ impl ExprFuncBuilder { ); }; - if let Some(order_by) = &order_by { - for expr in order_by.iter() { - if !matches!(expr, Expr::Sort(_)) { - return plan_err!( - "ORDER BY expressions must be Expr::Sort, found {expr:?}" - ); - } - } - } - let fun_expr = match fun { ExprFuncKind::Aggregate(mut udaf) => { udaf.order_by = order_by; @@ -833,9 +821,7 @@ impl ExprFuncBuilder { impl ExprFunctionExt for ExprFuncBuilder { /// Add `ORDER BY ` - /// - /// Note: `order_by` must be [`Expr::Sort`] - fn order_by(mut self, order_by: Vec) -> ExprFuncBuilder { + fn order_by(mut self, order_by: Vec) -> ExprFuncBuilder { self.order_by = Some(order_by); self } @@ -873,7 +859,7 @@ impl ExprFunctionExt for ExprFuncBuilder { } impl ExprFunctionExt for Expr { - fn order_by(self, order_by: Vec) -> ExprFuncBuilder { + fn order_by(self, order_by: Vec) -> ExprFuncBuilder { let mut builder = match self { Expr::AggregateFunction(udaf) => { ExprFuncBuilder::new(Some(ExprFuncKind::Aggregate(udaf))) diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index 768c4aabc840..b809b015d929 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -21,7 +21,7 @@ use std::collections::HashMap; use std::collections::HashSet; use std::sync::Arc; -use crate::expr::{Alias, Unnest}; +use crate::expr::{Alias, Sort, Unnest}; use crate::logical_plan::Projection; use crate::{Expr, ExprSchemable, LogicalPlan, LogicalPlanBuilder}; @@ -117,6 +117,20 @@ pub fn normalize_cols( .collect() } +pub fn normalize_sorts( + sorts: impl IntoIterator>, + plan: &LogicalPlan, +) -> Result> { + sorts + .into_iter() + .map(|e| { + let sort = e.into(); + normalize_col(*sort.expr, plan) + .map(|expr| Sort::new(Box::new(expr), sort.asc, sort.nulls_first)) + }) + .collect() +} + /// Recursively replace all [`Column`] expressions in a given expression tree with /// `Column` expressions provided by the hash map argument. pub fn replace_col(expr: Expr, replace_map: &HashMap<&Column, &Column>) -> Result { @@ -335,7 +349,6 @@ mod test { use std::ops::Add; use super::*; - use crate::expr::Sort; use crate::{col, lit, Cast}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::ScalarValue; @@ -496,12 +509,6 @@ mod test { // change literal type from i32 to i64 test_rewrite(col("a").add(lit(1i32)), col("a").add(lit(1i64))); - - // SortExpr a+1 ==> b + 2 - test_rewrite( - Expr::Sort(Sort::new(Box::new(col("a").add(lit(1i32))), true, false)), - Expr::Sort(Sort::new(Box::new(col("b").add(lit(2i64))), true, false)), - ); } /// rewrites `expr_from` to `rewrite_to` using @@ -524,15 +531,8 @@ mod test { }; let expr = rewrite_preserving_name(expr_from.clone(), &mut rewriter).unwrap(); - let original_name = match &expr_from { - Expr::Sort(Sort { expr, .. }) => expr.schema_name().to_string(), - expr => expr.schema_name().to_string(), - }; - - let new_name = match &expr { - Expr::Sort(Sort { expr, .. }) => expr.schema_name().to_string(), - expr => expr.schema_name().to_string(), - }; + let original_name = expr_from.schema_name().to_string(); + let new_name = expr.schema_name().to_string(); assert_eq!( original_name, new_name, diff --git a/datafusion/expr/src/expr_rewriter/order_by.rs b/datafusion/expr/src/expr_rewriter/order_by.rs index bbb855801c3e..af5b8c4f9177 100644 --- a/datafusion/expr/src/expr_rewriter/order_by.rs +++ b/datafusion/expr/src/expr_rewriter/order_by.rs @@ -17,9 +17,9 @@ //! Rewrite for order by expressions -use crate::expr::{Alias, Sort}; +use crate::expr::Alias; use crate::expr_rewriter::normalize_col; -use crate::{Cast, Expr, ExprSchemable, LogicalPlan, TryCast}; +use crate::{expr::Sort, Cast, Expr, ExprSchemable, LogicalPlan, TryCast}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Column, Result}; @@ -27,28 +27,18 @@ use datafusion_common::{Column, Result}; /// Rewrite sort on aggregate expressions to sort on the column of aggregate output /// For example, `max(x)` is written to `col("max(x)")` pub fn rewrite_sort_cols_by_aggs( - exprs: impl IntoIterator>, + sorts: impl IntoIterator>, plan: &LogicalPlan, -) -> Result> { - exprs +) -> Result> { + sorts .into_iter() .map(|e| { - let expr = e.into(); - match expr { - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => { - let sort = Expr::Sort(Sort::new( - Box::new(rewrite_sort_col_by_aggs(*expr, plan)?), - asc, - nulls_first, - )); - Ok(sort) - } - expr => Ok(expr), - } + let sort = e.into(); + Ok(Sort::new( + Box::new(rewrite_sort_col_by_aggs(*sort.expr, plan)?), + sort.asc, + sort.nulls_first, + )) }) .collect() } @@ -289,8 +279,8 @@ mod test { struct TestCase { desc: &'static str, - input: Expr, - expected: Expr, + input: Sort, + expected: Sort, } impl TestCase { @@ -332,7 +322,7 @@ mod test { .unwrap() } - fn sort(expr: Expr) -> Expr { + fn sort(expr: Expr) -> Sort { let asc = true; let nulls_first = true; expr.sort(asc, nulls_first) diff --git a/datafusion/expr/src/expr_schema.rs b/datafusion/expr/src/expr_schema.rs index 3920a1a3517c..894b7e58d954 100644 --- a/datafusion/expr/src/expr_schema.rs +++ b/datafusion/expr/src/expr_schema.rs @@ -18,7 +18,7 @@ use super::{Between, Expr, Like}; use crate::expr::{ AggregateFunction, Alias, BinaryExpr, Cast, InList, InSubquery, Placeholder, - ScalarFunction, Sort, TryCast, Unnest, WindowFunction, + ScalarFunction, TryCast, Unnest, WindowFunction, }; use crate::type_coercion::binary::get_result_type; use crate::type_coercion::functions::{ @@ -107,7 +107,7 @@ impl ExprSchemable for Expr { }, _ => expr.get_type(schema), }, - Expr::Sort(Sort { expr, .. }) | Expr::Negative(expr) => expr.get_type(schema), + Expr::Negative(expr) => expr.get_type(schema), Expr::Column(c) => Ok(schema.data_type(c)?.clone()), Expr::OuterReferenceColumn(ty, _) => Ok(ty.clone()), Expr::ScalarVariable(ty, _) => Ok(ty.clone()), @@ -280,10 +280,9 @@ impl ExprSchemable for Expr { /// column that does not exist in the schema. fn nullable(&self, input_schema: &dyn ExprSchema) -> Result { match self { - Expr::Alias(Alias { expr, .. }) - | Expr::Not(expr) - | Expr::Negative(expr) - | Expr::Sort(Sort { expr, .. }) => expr.nullable(input_schema), + Expr::Alias(Alias { expr, .. }) | Expr::Not(expr) | Expr::Negative(expr) => { + expr.nullable(input_schema) + } Expr::InList(InList { expr, list, .. }) => { // Avoid inspecting too many expressions. @@ -422,9 +421,7 @@ impl ExprSchemable for Expr { }, _ => expr.data_type_and_nullable(schema), }, - Expr::Sort(Sort { expr, .. }) | Expr::Negative(expr) => { - expr.data_type_and_nullable(schema) - } + Expr::Negative(expr) => expr.data_type_and_nullable(schema), Expr::Column(c) => schema .data_type_and_nullable(c) .map(|(d, n)| (d.clone(), n)), diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index 2c2300b123c2..f5770167861b 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -23,10 +23,10 @@ use std::collections::{HashMap, HashSet}; use std::sync::Arc; use crate::dml::CopyTo; -use crate::expr::Alias; +use crate::expr::{Alias, Sort as SortExpr}; use crate::expr_rewriter::{ coerce_plan_expr_for_schema, normalize_col, - normalize_col_with_schemas_and_ambiguity_check, normalize_cols, + normalize_col_with_schemas_and_ambiguity_check, normalize_cols, normalize_sorts, rewrite_sort_cols_by_aggs, }; use crate::logical_plan::{ @@ -541,19 +541,31 @@ impl LogicalPlanBuilder { plan_err!("For SELECT DISTINCT, ORDER BY expressions {missing_col_names} must appear in select list") } + /// Apply a sort by provided expressions with default direction + pub fn sort_by( + self, + expr: impl IntoIterator> + Clone, + ) -> Result { + self.sort( + expr.into_iter() + .map(|e| e.into().sort(true, false)) + .collect::>(), + ) + } + /// Apply a sort pub fn sort( self, - exprs: impl IntoIterator> + Clone, + sorts: impl IntoIterator> + Clone, ) -> Result { - let exprs = rewrite_sort_cols_by_aggs(exprs, &self.plan)?; + let sorts = rewrite_sort_cols_by_aggs(sorts, &self.plan)?; let schema = self.plan.schema(); // Collect sort columns that are missing in the input plan's schema let mut missing_cols: Vec = vec![]; - exprs.iter().try_for_each::<_, Result<()>>(|expr| { - let columns = expr.column_refs(); + sorts.iter().try_for_each::<_, Result<()>>(|sort| { + let columns = sort.expr.column_refs(); columns.into_iter().for_each(|c| { if !schema.has_column(c) { @@ -566,7 +578,7 @@ impl LogicalPlanBuilder { if missing_cols.is_empty() { return Ok(Self::new(LogicalPlan::Sort(Sort { - expr: normalize_cols(exprs, &self.plan)?, + expr: normalize_sorts(sorts, &self.plan)?, input: self.plan, fetch: None, }))); @@ -582,7 +594,7 @@ impl LogicalPlanBuilder { is_distinct, )?; let sort_plan = LogicalPlan::Sort(Sort { - expr: normalize_cols(exprs, &plan)?, + expr: normalize_sorts(sorts, &plan)?, input: Arc::new(plan), fetch: None, }); @@ -618,7 +630,7 @@ impl LogicalPlanBuilder { self, on_expr: Vec, select_expr: Vec, - sort_expr: Option>, + sort_expr: Option>, ) -> Result { Ok(Self::new(LogicalPlan::Distinct(Distinct::On( DistinctOn::try_new(on_expr, select_expr, sort_expr, self.plan)?, @@ -1708,8 +1720,8 @@ mod tests { let plan = table_scan(Some("employee_csv"), &employee_schema(), Some(vec![3, 4]))? .sort(vec![ - Expr::Sort(expr::Sort::new(Box::new(col("state")), true, true)), - Expr::Sort(expr::Sort::new(Box::new(col("salary")), false, false)), + expr::Sort::new(Box::new(col("state")), true, true), + expr::Sort::new(Box::new(col("salary")), false, false), ])? .build()?; @@ -2135,8 +2147,8 @@ mod tests { let plan = table_scan(Some("employee_csv"), &employee_schema(), Some(vec![3, 4]))? .sort(vec![ - Expr::Sort(expr::Sort::new(Box::new(col("state")), true, true)), - Expr::Sort(expr::Sort::new(Box::new(col("salary")), false, false)), + expr::Sort::new(Box::new(col("state")), true, true), + expr::Sort::new(Box::new(col("salary")), false, false), ])? .build()?; diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index ad0fcd2d4771..3fc43200efe6 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -22,8 +22,9 @@ use std::{ hash::{Hash, Hasher}, }; -use crate::{Expr, LogicalPlan, Volatility}; +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; @@ -204,7 +205,7 @@ pub struct CreateExternalTable { /// SQL used to create the table, if available pub definition: Option, /// Order expressions supplied by user - pub order_exprs: Vec>, + pub order_exprs: Vec>, /// Whether the table is an infinite streams pub unbounded: bool, /// Table(provider) specific options @@ -365,7 +366,7 @@ pub struct CreateIndex { pub name: Option, pub table: TableReference, pub using: Option, - pub columns: Vec, + pub columns: Vec, pub unique: bool, pub if_not_exists: bool, pub schema: DFSchemaRef, diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 359de2d30a57..8e6ec762f549 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -26,7 +26,9 @@ use super::dml::CopyTo; use super::DdlStatement; use crate::builder::{change_redundant_column, unnest_with_options}; use crate::expr::{Placeholder, Sort as SortExpr, WindowFunction}; -use crate::expr_rewriter::{create_col_from_scalar_expr, normalize_cols, NamePreserver}; +use crate::expr_rewriter::{ + create_col_from_scalar_expr, normalize_cols, normalize_sorts, NamePreserver, +}; use crate::logical_plan::display::{GraphvizVisitor, IndentVisitor}; use crate::logical_plan::extension::UserDefinedLogicalNode; use crate::logical_plan::{DmlStatement, Statement}; @@ -51,6 +53,7 @@ use datafusion_common::{ // backwards compatibility use crate::display::PgJsonVisitor; +use crate::tree_node::replace_sort_expressions; pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; @@ -884,8 +887,12 @@ impl LogicalPlan { Aggregate::try_new(Arc::new(inputs.swap_remove(0)), expr, agg_expr) .map(LogicalPlan::Aggregate) } - LogicalPlan::Sort(Sort { fetch, .. }) => Ok(LogicalPlan::Sort(Sort { - expr, + LogicalPlan::Sort(Sort { + expr: sort_expr, + fetch, + .. + }) => Ok(LogicalPlan::Sort(Sort { + expr: replace_sort_expressions(sort_expr.clone(), expr), input: Arc::new(inputs.swap_remove(0)), fetch: *fetch, })), @@ -1014,14 +1021,11 @@ impl LogicalPlan { }) => { let sort_expr = expr.split_off(on_expr.len() + select_expr.len()); let select_expr = expr.split_off(on_expr.len()); + assert!(sort_expr.is_empty(), "with_new_exprs for Distinct does not support sort expressions"); Distinct::On(DistinctOn::try_new( expr, select_expr, - if !sort_expr.is_empty() { - Some(sort_expr) - } else { - None - }, + None, // no sort expressions accepted Arc::new(inputs.swap_remove(0)), )?) } @@ -2559,7 +2563,7 @@ pub struct DistinctOn { /// 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: Option>, + pub sort_expr: Option>, /// The logical plan that is being DISTINCT'd pub input: Arc, /// The schema description of the DISTINCT ON output @@ -2571,7 +2575,7 @@ impl DistinctOn { pub fn try_new( on_expr: Vec, select_expr: Vec, - sort_expr: Option>, + sort_expr: Option>, input: Arc, ) -> Result { if on_expr.is_empty() { @@ -2606,20 +2610,15 @@ impl DistinctOn { /// Try to update `self` with a new sort expressions. /// /// Validates that the sort expressions are a super-set of the `ON` expressions. - pub fn with_sort_expr(mut self, sort_expr: Vec) -> Result { - let sort_expr = normalize_cols(sort_expr, self.input.as_ref())?; + pub fn with_sort_expr(mut self, sort_expr: Vec) -> Result { + let sort_expr = normalize_sorts(sort_expr, self.input.as_ref())?; // Check that the left-most sort expressions are the same as the `ON` expressions. let mut matched = true; for (on, sort) in self.on_expr.iter().zip(sort_expr.iter()) { - match sort { - Expr::Sort(SortExpr { expr, .. }) => { - if on != &**expr { - matched = false; - break; - } - } - _ => return plan_err!("Not a sort expression: {sort}"), + if on != &*sort.expr { + matched = false; + break; } } @@ -2833,7 +2832,7 @@ fn calc_func_dependencies_for_project( #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct Sort { /// The sort expressions - pub expr: Vec, + pub expr: Vec, /// The incoming logical plan pub input: Arc, /// Optional fetch limit diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index 273404c8df31..29a99a8e8886 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -46,7 +46,7 @@ use crate::{ use std::sync::Arc; use crate::expr::{Exists, InSubquery}; -use crate::tree_node::transform_option_vec; +use crate::tree_node::{transform_sort_option_vec, transform_sort_vec}; use datafusion_common::tree_node::{ Transformed, TreeNode, TreeNodeIterator, TreeNodeRecursion, TreeNodeRewriter, TreeNodeVisitor, @@ -481,7 +481,9 @@ impl LogicalPlan { .apply_until_stop(|e| f(&e))? .visit_sibling(|| filter.iter().apply_until_stop(f)) } - LogicalPlan::Sort(Sort { expr, .. }) => expr.iter().apply_until_stop(f), + LogicalPlan::Sort(Sort { expr, .. }) => { + expr.iter().apply_until_stop(|sort| f(&sort.expr)) + } LogicalPlan::Extension(extension) => { // would be nice to avoid this copy -- maybe can // update extension to just observer Exprs @@ -507,7 +509,7 @@ impl LogicalPlan { })) => on_expr .iter() .chain(select_expr.iter()) - .chain(sort_expr.iter().flatten()) + .chain(sort_expr.iter().flatten().map(|sort| &*sort.expr)) .apply_until_stop(f), // plans without expressions LogicalPlan::EmptyRelation(_) @@ -658,10 +660,10 @@ impl LogicalPlan { null_equals_null, }) }), - LogicalPlan::Sort(Sort { expr, input, fetch }) => expr - .into_iter() - .map_until_stop_and_collect(f)? - .update_data(|expr| LogicalPlan::Sort(Sort { expr, input, fetch })), + LogicalPlan::Sort(Sort { expr, input, fetch }) => { + transform_sort_vec(expr, &mut f)? + .update_data(|expr| LogicalPlan::Sort(Sort { expr, input, fetch })) + } LogicalPlan::Extension(Extension { node }) => { // would be nice to avoid this copy -- maybe can // update extension to just observer Exprs @@ -709,7 +711,7 @@ impl LogicalPlan { select_expr, select_expr.into_iter().map_until_stop_and_collect(&mut f), sort_expr, - transform_option_vec(sort_expr, &mut f) + transform_sort_option_vec(sort_expr, &mut f) )? .update_data(|(on_expr, select_expr, sort_expr)| { LogicalPlan::Distinct(Distinct::On(DistinctOn { diff --git a/datafusion/expr/src/tree_node.rs b/datafusion/expr/src/tree_node.rs index 450ebb6c2275..90d61bf63763 100644 --- a/datafusion/expr/src/tree_node.rs +++ b/datafusion/expr/src/tree_node.rs @@ -48,7 +48,6 @@ impl TreeNode for Expr { | Expr::Negative(expr) | Expr::Cast(Cast { expr, .. }) | Expr::TryCast(TryCast { expr, .. }) - | Expr::Sort(Sort { expr, .. }) | Expr::InSubquery(InSubquery{ expr, .. }) => vec![expr.as_ref()], Expr::GroupingSet(GroupingSet::Rollup(exprs)) | Expr::GroupingSet(GroupingSet::Cube(exprs)) => exprs.iter().collect(), @@ -98,7 +97,7 @@ impl TreeNode for Expr { expr_vec.push(f.as_ref()); } if let Some(order_by) = order_by { - expr_vec.extend(order_by); + expr_vec.extend(order_by.iter().map(|sort| sort.expr.as_ref())); } expr_vec } @@ -110,7 +109,7 @@ impl TreeNode for Expr { }) => { let mut expr_vec = args.iter().collect::>(); expr_vec.extend(partition_by); - expr_vec.extend(order_by); + expr_vec.extend(order_by.iter().map(|sort| sort.expr.as_ref())); expr_vec } Expr::InList(InList { expr, list, .. }) => { @@ -265,12 +264,6 @@ impl TreeNode for Expr { .update_data(|be| Expr::Cast(Cast::new(be, data_type))), Expr::TryCast(TryCast { expr, data_type }) => transform_box(expr, &mut f)? .update_data(|be| Expr::TryCast(TryCast::new(be, data_type))), - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => transform_box(expr, &mut f)? - .update_data(|be| Expr::Sort(Sort::new(be, asc, nulls_first))), Expr::ScalarFunction(ScalarFunction { func, args }) => { transform_vec(args, &mut f)?.map_data(|new_args| { Ok(Expr::ScalarFunction(ScalarFunction::new_udf( @@ -290,7 +283,7 @@ impl TreeNode for Expr { partition_by, transform_vec(partition_by, &mut f), order_by, - transform_vec(order_by, &mut f) + transform_sort_vec(order_by, &mut f) )? .update_data(|(new_args, new_partition_by, new_order_by)| { Expr::WindowFunction(WindowFunction::new(fun, new_args)) @@ -313,7 +306,7 @@ impl TreeNode for Expr { filter, transform_option_box(filter, &mut f), order_by, - transform_option_vec(order_by, &mut f) + transform_sort_option_vec(order_by, &mut f) )? .map_data(|(new_args, new_filter, new_order_by)| { Ok(Expr::AggregateFunction(AggregateFunction::new_udf( @@ -386,3 +379,41 @@ fn transform_vec Result>>( ) -> Result>> { ve.into_iter().map_until_stop_and_collect(f) } + +pub fn transform_sort_option_vec Result>>( + sorts_option: Option>, + f: &mut F, +) -> Result>>> { + sorts_option.map_or(Ok(Transformed::no(None)), |sorts| { + Ok(transform_sort_vec(sorts, f)?.update_data(Some)) + }) +} + +pub fn transform_sort_vec Result>>( + sorts: Vec, + mut f: &mut F, +) -> Result>> { + Ok(sorts + .iter() + .map(|sort| (*sort.expr).clone()) + .map_until_stop_and_collect(&mut f)? + .update_data(|transformed_exprs| { + replace_sort_expressions(sorts, transformed_exprs) + })) +} + +pub fn replace_sort_expressions(sorts: Vec, new_expr: Vec) -> Vec { + assert_eq!(sorts.len(), new_expr.len()); + sorts + .into_iter() + .zip(new_expr) + .map(|(sort, expr)| replace_sort_expression(sort, expr)) + .collect() +} + +pub fn replace_sort_expression(sort: Sort, new_expr: Expr) -> Sort { + Sort { + expr: Box::new(new_expr), + ..sort + } +} diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index a01d5ef8973a..b6b1b5660a81 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -296,7 +296,6 @@ pub fn expr_to_columns(expr: &Expr, accum: &mut HashSet) -> Result<()> { | Expr::Case { .. } | Expr::Cast { .. } | Expr::TryCast { .. } - | Expr::Sort { .. } | Expr::ScalarFunction(..) | Expr::WindowFunction { .. } | Expr::AggregateFunction { .. } @@ -461,22 +460,20 @@ pub fn expand_qualified_wildcard( /// (expr, "is the SortExpr for window (either comes from PARTITION BY or ORDER BY columns)") /// if bool is true SortExpr comes from `PARTITION BY` column, if false comes from `ORDER BY` column -type WindowSortKey = Vec<(Expr, bool)>; +type WindowSortKey = Vec<(Sort, bool)>; /// Generate a sort key for a given window expr's partition_by and order_by expr pub fn generate_sort_key( partition_by: &[Expr], - order_by: &[Expr], + order_by: &[Sort], ) -> Result { let normalized_order_by_keys = order_by .iter() - .map(|e| match e { - Expr::Sort(Sort { expr, .. }) => { - Ok(Expr::Sort(Sort::new(expr.clone(), true, false))) - } - _ => plan_err!("Order by only accepts sort expressions"), + .map(|e| { + let Sort { expr, .. } = e; + Sort::new(expr.clone(), true, false) }) - .collect::>>()?; + .collect::>(); let mut final_sort_keys = vec![]; let mut is_partition_flag = vec![]; @@ -512,65 +509,61 @@ pub fn generate_sort_key( /// Compare the sort expr as PostgreSQL's common_prefix_cmp(): /// pub fn compare_sort_expr( - sort_expr_a: &Expr, - sort_expr_b: &Expr, + sort_expr_a: &Sort, + sort_expr_b: &Sort, schema: &DFSchemaRef, ) -> Ordering { - match (sort_expr_a, sort_expr_b) { - ( - Expr::Sort(Sort { - expr: expr_a, - asc: asc_a, - nulls_first: nulls_first_a, - }), - Expr::Sort(Sort { - expr: expr_b, - asc: asc_b, - nulls_first: nulls_first_b, - }), - ) => { - let ref_indexes_a = find_column_indexes_referenced_by_expr(expr_a, schema); - let ref_indexes_b = find_column_indexes_referenced_by_expr(expr_b, schema); - for (idx_a, idx_b) in ref_indexes_a.iter().zip(ref_indexes_b.iter()) { - match idx_a.cmp(idx_b) { - Ordering::Less => { - return Ordering::Less; - } - Ordering::Greater => { - return Ordering::Greater; - } - Ordering::Equal => {} - } + let Sort { + expr: expr_a, + asc: asc_a, + nulls_first: nulls_first_a, + } = sort_expr_a; + + let Sort { + expr: expr_b, + asc: asc_b, + nulls_first: nulls_first_b, + } = sort_expr_b; + + let ref_indexes_a = find_column_indexes_referenced_by_expr(expr_a, schema); + let ref_indexes_b = find_column_indexes_referenced_by_expr(expr_b, schema); + for (idx_a, idx_b) in ref_indexes_a.iter().zip(ref_indexes_b.iter()) { + match idx_a.cmp(idx_b) { + Ordering::Less => { + return Ordering::Less; } - match ref_indexes_a.len().cmp(&ref_indexes_b.len()) { - Ordering::Less => return Ordering::Greater, - Ordering::Greater => { - return Ordering::Less; - } - Ordering::Equal => {} + Ordering::Greater => { + return Ordering::Greater; } - match (asc_a, asc_b) { - (true, false) => { - return Ordering::Greater; - } - (false, true) => { - return Ordering::Less; - } - _ => {} - } - match (nulls_first_a, nulls_first_b) { - (true, false) => { - return Ordering::Less; - } - (false, true) => { - return Ordering::Greater; - } - _ => {} - } - Ordering::Equal + Ordering::Equal => {} } - _ => panic!("Sort expressions must be of type Sort"), } + match ref_indexes_a.len().cmp(&ref_indexes_b.len()) { + Ordering::Less => return Ordering::Greater, + Ordering::Greater => { + return Ordering::Less; + } + Ordering::Equal => {} + } + match (asc_a, asc_b) { + (true, false) => { + return Ordering::Greater; + } + (false, true) => { + return Ordering::Less; + } + _ => {} + } + match (nulls_first_a, nulls_first_b) { + (true, false) => { + return Ordering::Less; + } + (false, true) => { + return Ordering::Greater; + } + _ => {} + } + Ordering::Equal } /// group a slice of window expression expr by their order by expressions @@ -606,14 +599,6 @@ pub fn find_aggregate_exprs(exprs: &[Expr]) -> Vec { }) } -/// Collect all deeply nested `Expr::Sort`. They are returned in order of occurrence -/// (depth first), with duplicates omitted. -pub fn find_sort_exprs(exprs: &[Expr]) -> Vec { - find_exprs_in_exprs(exprs, &|nested_expr| { - matches!(nested_expr, Expr::Sort { .. }) - }) -} - /// Collect all deeply nested `Expr::WindowFunction`. They are returned in order of occurrence /// (depth first), with duplicates omitted. pub fn find_window_exprs(exprs: &[Expr]) -> Vec { @@ -1376,8 +1361,7 @@ mod tests { use crate::{ col, cube, expr, expr_vec_fmt, grouping_set, lit, rollup, test::function_stub::max_udaf, test::function_stub::min_udaf, - test::function_stub::sum_udaf, Cast, ExprFunctionExt, WindowFrame, - WindowFunctionDefinition, + test::function_stub::sum_udaf, Cast, ExprFunctionExt, WindowFunctionDefinition, }; #[test] @@ -1417,10 +1401,9 @@ mod tests { #[test] fn test_group_window_expr_by_sort_keys() -> Result<()> { - let age_asc = Expr::Sort(expr::Sort::new(Box::new(col("age")), true, true)); - let name_desc = Expr::Sort(expr::Sort::new(Box::new(col("name")), false, true)); - let created_at_desc = - Expr::Sort(expr::Sort::new(Box::new(col("created_at")), false, true)); + let age_asc = expr::Sort::new(Box::new(col("age")), true, true); + let name_desc = expr::Sort::new(Box::new(col("name")), false, true); + let created_at_desc = expr::Sort::new(Box::new(col("created_at")), false, true); let max1 = Expr::WindowFunction(expr::WindowFunction::new( WindowFunctionDefinition::AggregateUDF(max_udaf()), vec![col("name")], @@ -1471,43 +1454,6 @@ mod tests { Ok(()) } - #[test] - fn test_find_sort_exprs() -> Result<()> { - let exprs = &[ - Expr::WindowFunction(expr::WindowFunction::new( - WindowFunctionDefinition::AggregateUDF(max_udaf()), - vec![col("name")], - )) - .order_by(vec![ - Expr::Sort(expr::Sort::new(Box::new(col("age")), true, true)), - Expr::Sort(expr::Sort::new(Box::new(col("name")), false, true)), - ]) - .window_frame(WindowFrame::new(Some(false))) - .build() - .unwrap(), - Expr::WindowFunction(expr::WindowFunction::new( - WindowFunctionDefinition::AggregateUDF(sum_udaf()), - vec![col("age")], - )) - .order_by(vec![ - Expr::Sort(expr::Sort::new(Box::new(col("name")), false, true)), - Expr::Sort(expr::Sort::new(Box::new(col("age")), true, true)), - Expr::Sort(expr::Sort::new(Box::new(col("created_at")), false, true)), - ]) - .window_frame(WindowFrame::new(Some(false))) - .build() - .unwrap(), - ]; - let expected = vec![ - Expr::Sort(expr::Sort::new(Box::new(col("age")), true, true)), - Expr::Sort(expr::Sort::new(Box::new(col("name")), false, true)), - Expr::Sort(expr::Sort::new(Box::new(col("created_at")), false, true)), - ]; - let result = find_sort_exprs(exprs); - assert_eq!(expected, result); - Ok(()) - } - #[test] fn avoid_generate_duplicate_sort_keys() -> Result<()> { let asc_or_desc = [true, false]; @@ -1516,41 +1462,41 @@ mod tests { for asc_ in asc_or_desc { for nulls_first_ in nulls_first_or_last { let order_by = &[ - Expr::Sort(Sort { + Sort { expr: Box::new(col("age")), asc: asc_, nulls_first: nulls_first_, - }), - Expr::Sort(Sort { + }, + Sort { expr: Box::new(col("name")), asc: asc_, nulls_first: nulls_first_, - }), + }, ]; let expected = vec![ ( - Expr::Sort(Sort { + Sort { expr: Box::new(col("age")), asc: asc_, nulls_first: nulls_first_, - }), + }, true, ), ( - Expr::Sort(Sort { + Sort { expr: Box::new(col("name")), asc: asc_, nulls_first: nulls_first_, - }), + }, true, ), ( - Expr::Sort(Sort { + Sort { expr: Box::new(col("created_at")), asc: true, nulls_first: false, - }), + }, true, ), ]; diff --git a/datafusion/expr/src/window_frame.rs b/datafusion/expr/src/window_frame.rs index 0e1d917419f8..6c935cdcd121 100644 --- a/datafusion/expr/src/window_frame.rs +++ b/datafusion/expr/src/window_frame.rs @@ -26,7 +26,7 @@ use std::fmt::{self, Formatter}; use std::hash::Hash; -use crate::{lit, Expr}; +use crate::{expr::Sort, lit}; use datafusion_common::{plan_err, sql_err, DataFusionError, Result, ScalarValue}; use sqlparser::ast; @@ -247,7 +247,7 @@ impl WindowFrame { } /// Regularizes the ORDER BY clause of the window frame. - pub fn regularize_order_bys(&self, order_by: &mut Vec) -> Result<()> { + pub fn regularize_order_bys(&self, order_by: &mut Vec) -> Result<()> { match self.units { // Normally, RANGE frames require an ORDER BY clause with exactly // one column. However, an ORDER BY clause may be absent or have diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 2162442f054e..30f5d5b07561 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -32,7 +32,7 @@ use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::{format_state_name, AggregateOrderSensitivity}; use datafusion_expr::{ Accumulator, AggregateUDFImpl, ArrayFunctionSignature, Expr, ExprFunctionExt, - Signature, TypeSignature, Volatility, + Signature, SortExpr, TypeSignature, Volatility, }; use datafusion_functions_aggregate_common::utils::get_sort_options; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; @@ -40,7 +40,7 @@ use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; create_func!(FirstValue, first_value_udaf); /// Returns the first value in a group of values. -pub fn first_value(expression: Expr, order_by: Option>) -> Expr { +pub fn first_value(expression: Expr, order_by: Option>) -> Expr { if let Some(order_by) = order_by { first_value_udaf() .call(vec![expression]) diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index e114efb99960..35d4f91e3b6f 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -229,7 +229,7 @@ mod tests { WindowFunctionDefinition::AggregateUDF(count_udaf()), vec![wildcard()], )) - .order_by(vec![Expr::Sort(Sort::new(Box::new(col("a")), false, true))]) + .order_by(vec![Sort::new(Box::new(col("a")), false, true)]) .window_frame(WindowFrame::new_bounds( WindowFrameUnits::Range, WindowFrameBound::Preceding(ScalarValue::UInt32(Some(6))), diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index a6b9bad6c5d9..61ff4b4fd5a8 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -33,7 +33,7 @@ use datafusion_common::{ }; use datafusion_expr::expr::{ self, Alias, Between, BinaryExpr, Case, Exists, InList, InSubquery, Like, - ScalarFunction, WindowFunction, + ScalarFunction, Sort, WindowFunction, }; use datafusion_expr::expr_rewriter::coerce_plan_expr_for_schema; use datafusion_expr::expr_schema::cast_subquery; @@ -506,7 +506,6 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { | Expr::Negative(_) | Expr::Cast(_) | Expr::TryCast(_) - | Expr::Sort(_) | Expr::Wildcard { .. } | Expr::GroupingSet(_) | Expr::Placeholder(_) @@ -593,12 +592,12 @@ fn coerce_frame_bound( fn coerce_window_frame( window_frame: WindowFrame, schema: &DFSchema, - expressions: &[Expr], + expressions: &[Sort], ) -> Result { let mut window_frame = window_frame; let current_types = expressions .iter() - .map(|e| e.get_type(schema)) + .map(|s| s.expr.get_type(schema)) .collect::>>()?; let target_type = match window_frame.units { WindowFrameUnits::Range => { diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 3a2b190359d4..25bef7e2d0e4 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -36,6 +36,7 @@ use datafusion_expr::expr::{Alias, ScalarFunction}; use datafusion_expr::logical_plan::{ Aggregate, Filter, LogicalPlan, Projection, Sort, Window, }; +use datafusion_expr::tree_node::replace_sort_expressions; use datafusion_expr::{col, BinaryExpr, Case, Expr, ExprSchemable, Operator}; use indexmap::IndexMap; @@ -327,15 +328,17 @@ impl CommonSubexprEliminate { ) -> Result> { let Sort { expr, input, fetch } = sort; let input = Arc::unwrap_or_clone(input); - let new_sort = self.try_unary_plan(expr, input, config)?.update_data( - |(new_expr, new_input)| { + let sort_expressions = + expr.iter().map(|sort| sort.expr.as_ref().clone()).collect(); + let new_sort = self + .try_unary_plan(sort_expressions, input, config)? + .update_data(|(new_expr, new_input)| { LogicalPlan::Sort(Sort { - expr: new_expr, + expr: replace_sort_expressions(expr, new_expr), input: Arc::new(new_input), fetch, }) - }, - ); + }); Ok(new_sort) } @@ -882,7 +885,6 @@ enum ExprMask { /// - [`Columns`](Expr::Column) /// - [`ScalarVariable`](Expr::ScalarVariable) /// - [`Alias`](Expr::Alias) - /// - [`Sort`](Expr::Sort) /// - [`Wildcard`](Expr::Wildcard) /// - [`AggregateFunction`](Expr::AggregateFunction) Normal, @@ -899,7 +901,6 @@ impl ExprMask { | Expr::Column(..) | Expr::ScalarVariable(..) | Expr::Alias(..) - | Expr::Sort { .. } | Expr::Wildcard { .. } ); diff --git a/datafusion/optimizer/src/eliminate_duplicated_expr.rs b/datafusion/optimizer/src/eliminate_duplicated_expr.rs index e9d091d52b00..c460d7a93d26 100644 --- a/datafusion/optimizer/src/eliminate_duplicated_expr.rs +++ b/datafusion/optimizer/src/eliminate_duplicated_expr.rs @@ -22,7 +22,7 @@ use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::tree_node::Transformed; use datafusion_common::Result; use datafusion_expr::logical_plan::LogicalPlan; -use datafusion_expr::{Aggregate, Expr, Sort}; +use datafusion_expr::{Aggregate, Expr, Sort, SortExpr}; use indexmap::IndexSet; use std::hash::{Hash, Hasher}; /// Optimization rule that eliminate duplicated expr. @@ -37,29 +37,15 @@ impl EliminateDuplicatedExpr { } // use this structure to avoid initial clone #[derive(Eq, Clone, Debug)] -struct SortExprWrapper { - expr: Expr, -} +struct SortExprWrapper(SortExpr); impl PartialEq for SortExprWrapper { fn eq(&self, other: &Self) -> bool { - match (&self.expr, &other.expr) { - (Expr::Sort(own_sort), Expr::Sort(other_sort)) => { - own_sort.expr == other_sort.expr - } - _ => self.expr == other.expr, - } + self.0.expr == other.0.expr } } impl Hash for SortExprWrapper { fn hash(&self, state: &mut H) { - match &self.expr { - Expr::Sort(sort) => { - sort.expr.hash(state); - } - _ => { - self.expr.hash(state); - } - } + self.0.expr.hash(state); } } impl OptimizerRule for EliminateDuplicatedExpr { @@ -82,10 +68,10 @@ impl OptimizerRule for EliminateDuplicatedExpr { let unique_exprs: Vec<_> = sort .expr .into_iter() - .map(|e| SortExprWrapper { expr: e }) + .map(SortExprWrapper) .collect::>() .into_iter() - .map(|wrapper| wrapper.expr) + .map(|wrapper| wrapper.0) .collect(); let transformed = if len != unique_exprs.len() { @@ -146,11 +132,11 @@ mod tests { fn eliminate_sort_expr() -> Result<()> { let table_scan = test_table_scan().unwrap(); let plan = LogicalPlanBuilder::from(table_scan) - .sort(vec![col("a"), col("a"), col("b"), col("c")])? + .sort_by(vec![col("a"), col("a"), col("b"), col("c")])? .limit(5, Some(10))? .build()?; let expected = "Limit: skip=5, fetch=10\ - \n Sort: test.a, test.b, test.c\ + \n Sort: test.a ASC NULLS LAST, test.b ASC NULLS LAST, test.c ASC NULLS LAST\ \n TableScan: test"; assert_optimized_plan_eq(plan, expected) } diff --git a/datafusion/optimizer/src/eliminate_limit.rs b/datafusion/optimizer/src/eliminate_limit.rs index e48f37a77cd3..2503475bd8df 100644 --- a/datafusion/optimizer/src/eliminate_limit.rs +++ b/datafusion/optimizer/src/eliminate_limit.rs @@ -182,14 +182,14 @@ mod tests { let plan = LogicalPlanBuilder::from(table_scan) .aggregate(vec![col("a")], vec![sum(col("b"))])? .limit(0, Some(2))? - .sort(vec![col("a")])? + .sort_by(vec![col("a")])? .limit(2, Some(1))? .build()?; // After remove global-state, we don't record the parent // So, bottom don't know parent info, so can't eliminate. let expected = "Limit: skip=2, fetch=1\ - \n Sort: test.a, fetch=3\ + \n Sort: test.a ASC NULLS LAST, fetch=3\ \n Limit: skip=0, fetch=2\ \n Aggregate: groupBy=[[test.a]], aggr=[[sum(test.b)]]\ \n TableScan: test"; @@ -202,12 +202,12 @@ mod tests { let plan = LogicalPlanBuilder::from(table_scan) .aggregate(vec![col("a")], vec![sum(col("b"))])? .limit(0, Some(2))? - .sort(vec![col("a")])? + .sort_by(vec![col("a")])? .limit(0, Some(1))? .build()?; let expected = "Limit: skip=0, fetch=1\ - \n Sort: test.a\ + \n Sort: test.a ASC NULLS LAST\ \n Limit: skip=0, fetch=2\ \n Aggregate: groupBy=[[test.a]], aggr=[[sum(test.b)]]\ \n TableScan: test"; @@ -220,12 +220,12 @@ mod tests { let plan = LogicalPlanBuilder::from(table_scan) .aggregate(vec![col("a")], vec![sum(col("b"))])? .limit(2, Some(1))? - .sort(vec![col("a")])? + .sort_by(vec![col("a")])? .limit(3, Some(1))? .build()?; let expected = "Limit: skip=3, fetch=1\ - \n Sort: test.a\ + \n Sort: test.a ASC NULLS LAST\ \n Limit: skip=2, fetch=1\ \n Aggregate: groupBy=[[test.a]], aggr=[[sum(test.b)]]\ \n TableScan: test"; diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 82149a087e63..33a58a810b08 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -284,8 +284,7 @@ fn can_evaluate_as_join_condition(predicate: &Expr) -> Result { | Expr::TryCast(_) | Expr::InList { .. } | Expr::ScalarFunction(_) => Ok(TreeNodeRecursion::Continue), - Expr::Sort(_) - | Expr::AggregateFunction(_) + Expr::AggregateFunction(_) | Expr::WindowFunction(_) | Expr::Wildcard { .. } | Expr::GroupingSet(_) => internal_err!("Unsupported predicate type"), diff --git a/datafusion/optimizer/src/push_down_limit.rs b/datafusion/optimizer/src/push_down_limit.rs index dff0b61c6b22..ab7880213692 100644 --- a/datafusion/optimizer/src/push_down_limit.rs +++ b/datafusion/optimizer/src/push_down_limit.rs @@ -347,13 +347,13 @@ mod test { let table_scan = test_table_scan()?; let plan = LogicalPlanBuilder::from(table_scan) - .sort(vec![col("a")])? + .sort_by(vec![col("a")])? .limit(0, Some(10))? .build()?; // Should push down limit to sort let expected = "Limit: skip=0, fetch=10\ - \n Sort: test.a, fetch=10\ + \n Sort: test.a ASC NULLS LAST, fetch=10\ \n TableScan: test"; assert_optimized_plan_equal(plan, expected) @@ -364,13 +364,13 @@ mod test { let table_scan = test_table_scan()?; let plan = LogicalPlanBuilder::from(table_scan) - .sort(vec![col("a")])? + .sort_by(vec![col("a")])? .limit(5, Some(10))? .build()?; // Should push down limit to sort let expected = "Limit: skip=5, fetch=10\ - \n Sort: test.a, fetch=15\ + \n Sort: test.a ASC NULLS LAST, fetch=15\ \n TableScan: test"; assert_optimized_plan_equal(plan, expected) diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 7129ceb0fea1..f299d4542c36 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -591,7 +591,6 @@ impl<'a> ConstEvaluator<'a> { | Expr::InSubquery(_) | Expr::ScalarSubquery(_) | Expr::WindowFunction { .. } - | Expr::Sort { .. } | Expr::GroupingSet(_) | Expr::Wildcard { .. } | Expr::Placeholder(_) => false, diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 30cae17eaf9f..dd82b056d0a6 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -624,14 +624,14 @@ mod tests { vec![col("a")], false, None, - Some(vec![col("a")]), + Some(vec![col("a").sort(true, false)]), None, )); let plan = LogicalPlanBuilder::from(table_scan) .aggregate(vec![col("c")], vec![expr, count_distinct(col("b"))])? .build()?; // Do nothing - let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a) ORDER BY [test.a], count(DISTINCT test.b)]] [c:UInt32, sum(test.a) ORDER BY [test.a]:UInt64;N, count(DISTINCT test.b):Int64]\ + let expected = "Aggregate: groupBy=[[test.c]], aggr=[[sum(test.a) ORDER BY [test.a ASC NULLS LAST], count(DISTINCT test.b)]] [c:UInt32, sum(test.a) ORDER BY [test.a ASC NULLS LAST]:UInt64;N, count(DISTINCT test.b):Int64]\ \n TableScan: test [a:UInt32, b:UInt32, c:UInt32]"; assert_optimized_plan_equal(plan, expected) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 826992e132ba..19759a897068 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -75,6 +75,10 @@ message LogicalExprNodeCollection { repeated LogicalExprNode logical_expr_nodes = 1; } +message SortExprNodeCollection { + repeated SortExprNode sort_expr_nodes = 1; +} + message ListingTableScanNode { reserved 1; // was string table_name TableReference table_name = 14; @@ -92,7 +96,7 @@ message ListingTableScanNode { datafusion_common.AvroFormat avro = 12; datafusion_common.NdJsonFormat json = 15; } - repeated LogicalExprNodeCollection file_sort_order = 13; + repeated SortExprNodeCollection file_sort_order = 13; } message ViewTableScanNode { @@ -129,7 +133,7 @@ message SelectionNode { message SortNode { LogicalPlanNode input = 1; - repeated LogicalExprNode expr = 2; + repeated SortExprNode expr = 2; // Maximum number of highest/lowest rows to fetch; negative means no limit int64 fetch = 3; } @@ -160,7 +164,7 @@ message CreateExternalTableNode { repeated string table_partition_cols = 5; bool if_not_exists = 6; string definition = 7; - repeated LogicalExprNodeCollection order_exprs = 10; + repeated SortExprNodeCollection order_exprs = 10; bool unbounded = 11; map options = 8; datafusion_common.Constraints constraints = 12; @@ -245,7 +249,7 @@ message DistinctNode { message DistinctOnNode { repeated LogicalExprNode on_expr = 1; repeated LogicalExprNode select_expr = 2; - repeated LogicalExprNode sort_expr = 3; + repeated SortExprNode sort_expr = 3; LogicalPlanNode input = 4; } @@ -320,7 +324,6 @@ message LogicalExprNode { BetweenNode between = 9; CaseNode case_ = 10; CastNode cast = 11; - SortExprNode sort = 12; NegativeNode negative = 13; InListNode in_list = 14; Wildcard wildcard = 15; @@ -470,7 +473,7 @@ message AggregateUDFExprNode { repeated LogicalExprNode args = 2; bool distinct = 5; LogicalExprNode filter = 3; - repeated LogicalExprNode order_by = 4; + repeated SortExprNode order_by = 4; optional bytes fun_definition = 6; } @@ -503,7 +506,7 @@ message WindowExprNode { } LogicalExprNode expr = 4; repeated LogicalExprNode partition_by = 5; - repeated LogicalExprNode order_by = 6; + repeated SortExprNode order_by = 6; // repeated LogicalExprNode filter = 7; WindowFrame window_frame = 8; optional bytes fun_definition = 10; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index b4d63798f080..cff58d3ddc4a 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -9291,9 +9291,6 @@ impl serde::Serialize for LogicalExprNode { logical_expr_node::ExprType::Cast(v) => { struct_ser.serialize_field("cast", v)?; } - logical_expr_node::ExprType::Sort(v) => { - struct_ser.serialize_field("sort", v)?; - } logical_expr_node::ExprType::Negative(v) => { struct_ser.serialize_field("negative", v)?; } @@ -9384,7 +9381,6 @@ impl<'de> serde::Deserialize<'de> for LogicalExprNode { "case_", "case", "cast", - "sort", "negative", "in_list", "inList", @@ -9433,7 +9429,6 @@ impl<'de> serde::Deserialize<'de> for LogicalExprNode { Between, Case, Cast, - Sort, Negative, InList, Wildcard, @@ -9486,7 +9481,6 @@ impl<'de> serde::Deserialize<'de> for LogicalExprNode { "between" => Ok(GeneratedField::Between), "case" | "case_" => Ok(GeneratedField::Case), "cast" => Ok(GeneratedField::Cast), - "sort" => Ok(GeneratedField::Sort), "negative" => Ok(GeneratedField::Negative), "inList" | "in_list" => Ok(GeneratedField::InList), "wildcard" => Ok(GeneratedField::Wildcard), @@ -9598,13 +9592,6 @@ impl<'de> serde::Deserialize<'de> for LogicalExprNode { return Err(serde::de::Error::duplicate_field("cast")); } expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Cast) -; - } - GeneratedField::Sort => { - if expr_type__.is_some() { - return Err(serde::de::Error::duplicate_field("sort")); - } - expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Sort) ; } GeneratedField::Negative => { @@ -17947,6 +17934,98 @@ impl<'de> serde::Deserialize<'de> for SortExprNode { deserializer.deserialize_struct("datafusion.SortExprNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for SortExprNodeCollection { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if !self.sort_expr_nodes.is_empty() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.SortExprNodeCollection", len)?; + if !self.sort_expr_nodes.is_empty() { + struct_ser.serialize_field("sortExprNodes", &self.sort_expr_nodes)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for SortExprNodeCollection { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "sort_expr_nodes", + "sortExprNodes", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + SortExprNodes, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "sortExprNodes" | "sort_expr_nodes" => Ok(GeneratedField::SortExprNodes), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = SortExprNodeCollection; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.SortExprNodeCollection") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut sort_expr_nodes__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::SortExprNodes => { + if sort_expr_nodes__.is_some() { + return Err(serde::de::Error::duplicate_field("sortExprNodes")); + } + sort_expr_nodes__ = Some(map_.next_value()?); + } + } + } + Ok(SortExprNodeCollection { + sort_expr_nodes: sort_expr_nodes__.unwrap_or_default(), + }) + } + } + deserializer.deserialize_struct("datafusion.SortExprNodeCollection", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for SortNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 875d2af75dd7..2ce8004e3248 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -97,6 +97,12 @@ pub struct LogicalExprNodeCollection { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct SortExprNodeCollection { + #[prost(message, repeated, tag = "1")] + pub sort_expr_nodes: ::prost::alloc::vec::Vec, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct ListingTableScanNode { #[prost(message, optional, tag = "14")] pub table_name: ::core::option::Option, @@ -117,7 +123,7 @@ pub struct ListingTableScanNode { #[prost(uint32, tag = "9")] pub target_partitions: u32, #[prost(message, repeated, tag = "13")] - pub file_sort_order: ::prost::alloc::vec::Vec, + pub file_sort_order: ::prost::alloc::vec::Vec, #[prost(oneof = "listing_table_scan_node::FileFormatType", tags = "10, 11, 12, 15")] pub file_format_type: ::core::option::Option< listing_table_scan_node::FileFormatType, @@ -200,7 +206,7 @@ pub struct SortNode { #[prost(message, optional, boxed, tag = "1")] pub input: ::core::option::Option<::prost::alloc::boxed::Box>, #[prost(message, repeated, tag = "2")] - pub expr: ::prost::alloc::vec::Vec, + pub expr: ::prost::alloc::vec::Vec, /// Maximum number of highest/lowest rows to fetch; negative means no limit #[prost(int64, tag = "3")] pub fetch: i64, @@ -256,7 +262,7 @@ pub struct CreateExternalTableNode { #[prost(string, tag = "7")] pub definition: ::prost::alloc::string::String, #[prost(message, repeated, tag = "10")] - pub order_exprs: ::prost::alloc::vec::Vec, + pub order_exprs: ::prost::alloc::vec::Vec, #[prost(bool, tag = "11")] pub unbounded: bool, #[prost(map = "string, string", tag = "8")] @@ -402,7 +408,7 @@ pub struct DistinctOnNode { #[prost(message, repeated, tag = "2")] pub select_expr: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "3")] - pub sort_expr: ::prost::alloc::vec::Vec, + pub sort_expr: ::prost::alloc::vec::Vec, #[prost(message, optional, boxed, tag = "4")] pub input: ::core::option::Option<::prost::alloc::boxed::Box>, } @@ -488,7 +494,7 @@ pub struct SubqueryAliasNode { pub struct LogicalExprNode { #[prost( oneof = "logical_expr_node::ExprType", - tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 17, 18, 19, 20, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35" + tags = "1, 2, 3, 4, 6, 7, 8, 9, 10, 11, 13, 14, 15, 17, 18, 19, 20, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35" )] pub expr_type: ::core::option::Option, } @@ -521,8 +527,6 @@ pub mod logical_expr_node { Case(::prost::alloc::boxed::Box), #[prost(message, tag = "11")] Cast(::prost::alloc::boxed::Box), - #[prost(message, tag = "12")] - Sort(::prost::alloc::boxed::Box), #[prost(message, tag = "13")] Negative(::prost::alloc::boxed::Box), #[prost(message, tag = "14")] @@ -740,7 +744,7 @@ pub struct AggregateUdfExprNode { #[prost(message, optional, boxed, tag = "3")] pub filter: ::core::option::Option<::prost::alloc::boxed::Box>, #[prost(message, repeated, tag = "4")] - pub order_by: ::prost::alloc::vec::Vec, + pub order_by: ::prost::alloc::vec::Vec, #[prost(bytes = "vec", optional, tag = "6")] pub fun_definition: ::core::option::Option<::prost::alloc::vec::Vec>, } @@ -762,7 +766,7 @@ pub struct WindowExprNode { #[prost(message, repeated, tag = "5")] pub partition_by: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "6")] - pub order_by: ::prost::alloc::vec::Vec, + pub order_by: ::prost::alloc::vec::Vec, /// repeated LogicalExprNode filter = 7; #[prost(message, optional, tag = "8")] pub window_frame: ::core::option::Option, @@ -869,8 +873,8 @@ pub struct TryCastNode { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct SortExprNode { - #[prost(message, optional, boxed, tag = "1")] - pub expr: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, optional, tag = "1")] + pub expr: ::core::option::Option, #[prost(bool, tag = "2")] pub asc: bool, #[prost(bool, tag = "3")] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index acda1298dd80..3ba1cb945e9c 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -22,11 +22,11 @@ use datafusion_common::{ exec_datafusion_err, internal_err, plan_datafusion_err, Result, ScalarValue, TableReference, UnnestOptions, }; -use datafusion_expr::expr::{Alias, Placeholder}; +use datafusion_expr::expr::{Alias, Placeholder, Sort}; use datafusion_expr::expr::{Unnest, WildcardOptions}; use datafusion_expr::ExprFunctionExt; use datafusion_expr::{ - expr::{self, InList, Sort, WindowFunction}, + expr::{self, InList, WindowFunction}, logical_plan::{PlanType, StringifiedPlan}, Between, BinaryExpr, BuiltInWindowFunction, Case, Cast, Expr, GroupingSet, GroupingSet::GroupingSets, @@ -267,7 +267,7 @@ pub fn parse_expr( .as_ref() .ok_or_else(|| Error::required("window_function"))?; let partition_by = parse_exprs(&expr.partition_by, registry, codec)?; - let mut order_by = parse_exprs(&expr.order_by, registry, codec)?; + let mut order_by = parse_sorts(&expr.order_by, registry, codec)?; let window_frame = expr .window_frame .as_ref() @@ -524,16 +524,6 @@ pub fn parse_expr( let data_type = cast.arrow_type.as_ref().required("arrow_type")?; Ok(Expr::TryCast(TryCast::new(expr, data_type))) } - ExprType::Sort(sort) => Ok(Expr::Sort(Sort::new( - Box::new(parse_required_expr( - sort.expr.as_deref(), - registry, - "expr", - codec, - )?), - sort.asc, - sort.nulls_first, - ))), ExprType::Negative(negative) => Ok(Expr::Negative(Box::new( parse_required_expr(negative.expr.as_deref(), registry, "expr", codec)?, ))), @@ -588,7 +578,7 @@ pub fn parse_expr( parse_optional_expr(pb.filter.as_deref(), registry, codec)?.map(Box::new), match pb.order_by.len() { 0 => None, - _ => Some(parse_exprs(&pb.order_by, registry, codec)?), + _ => Some(parse_sorts(&pb.order_by, registry, codec)?), }, None, ))) @@ -635,6 +625,37 @@ where Ok(res) } +pub fn parse_sorts<'a, I>( + protos: I, + registry: &dyn FunctionRegistry, + codec: &dyn LogicalExtensionCodec, +) -> Result, Error> +where + I: IntoIterator, +{ + protos + .into_iter() + .map(|sort| parse_sort(sort, registry, codec)) + .collect::, Error>>() +} + +pub fn parse_sort( + sort: &protobuf::SortExprNode, + registry: &dyn FunctionRegistry, + codec: &dyn LogicalExtensionCodec, +) -> Result { + Ok(Sort::new( + Box::new(parse_required_expr( + sort.expr.as_ref(), + registry, + "expr", + codec, + )?), + sort.asc, + sort.nulls_first, + )) +} + /// Parse an optional escape_char for Like, ILike, SimilarTo fn parse_escape_char(s: &str) -> Result> { match s.len() { diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 67977b1795a6..bf5394ec01de 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -20,7 +20,7 @@ use std::fmt::Debug; use std::sync::Arc; use crate::protobuf::logical_plan_node::LogicalPlanType::CustomScan; -use crate::protobuf::{CustomTableScanNode, LogicalExprNodeCollection}; +use crate::protobuf::{CustomTableScanNode, SortExprNodeCollection}; use crate::{ convert_required, into_required, protobuf::{ @@ -62,11 +62,13 @@ use datafusion_expr::{ EmptyRelation, Extension, Join, JoinConstraint, Limit, Prepare, Projection, Repartition, Sort, SubqueryAlias, TableScan, Values, Window, }, - DistinctOn, DropView, Expr, LogicalPlan, LogicalPlanBuilder, ScalarUDF, WindowUDF, + DistinctOn, DropView, Expr, LogicalPlan, LogicalPlanBuilder, ScalarUDF, SortExpr, + WindowUDF, }; use datafusion_expr::{AggregateUDF, Unnest}; use self::to_proto::{serialize_expr, serialize_exprs}; +use crate::logical_plan::to_proto::serialize_sorts; use prost::bytes::BufMut; use prost::Message; @@ -347,8 +349,8 @@ impl AsLogicalPlan for LogicalPlanNode { let mut all_sort_orders = vec![]; for order in &scan.file_sort_order { - all_sort_orders.push(from_proto::parse_exprs( - &order.logical_expr_nodes, + all_sort_orders.push(from_proto::parse_sorts( + &order.sort_expr_nodes, ctx, extension_codec, )?) @@ -476,8 +478,8 @@ impl AsLogicalPlan for LogicalPlanNode { LogicalPlanType::Sort(sort) => { let input: LogicalPlan = into_logical_plan!(sort.input, ctx, extension_codec)?; - let sort_expr: Vec = - from_proto::parse_exprs(&sort.expr, ctx, extension_codec)?; + let sort_expr: Vec = + from_proto::parse_sorts(&sort.expr, ctx, extension_codec)?; LogicalPlanBuilder::from(input).sort(sort_expr)?.build() } LogicalPlanType::Repartition(repartition) => { @@ -536,8 +538,8 @@ impl AsLogicalPlan for LogicalPlanNode { let mut order_exprs = vec![]; for expr in &create_extern_table.order_exprs { - order_exprs.push(from_proto::parse_exprs( - &expr.logical_expr_nodes, + order_exprs.push(from_proto::parse_sorts( + &expr.sort_expr_nodes, ctx, extension_codec, )?); @@ -772,7 +774,7 @@ impl AsLogicalPlan for LogicalPlanNode { )?; let sort_expr = match distinct_on.sort_expr.len() { 0 => None, - _ => Some(from_proto::parse_exprs( + _ => Some(from_proto::parse_sorts( &distinct_on.sort_expr, ctx, extension_codec, @@ -981,10 +983,10 @@ impl AsLogicalPlan for LogicalPlanNode { let options = listing_table.options(); - let mut exprs_vec: Vec = vec![]; + let mut exprs_vec: Vec = vec![]; for order in &options.file_sort_order { - let expr_vec = LogicalExprNodeCollection { - logical_expr_nodes: serialize_exprs(order, extension_codec)?, + let expr_vec = SortExprNodeCollection { + sort_expr_nodes: serialize_sorts(order, extension_codec)?, }; exprs_vec.push(expr_vec); } @@ -1114,7 +1116,7 @@ impl AsLogicalPlan for LogicalPlanNode { )?; let sort_expr = match sort_expr { None => vec![], - Some(sort_expr) => serialize_exprs(sort_expr, extension_codec)?, + Some(sort_expr) => serialize_sorts(sort_expr, extension_codec)?, }; Ok(protobuf::LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::DistinctOn(Box::new( @@ -1258,13 +1260,13 @@ impl AsLogicalPlan for LogicalPlanNode { input.as_ref(), extension_codec, )?; - let selection_expr: Vec = - serialize_exprs(expr, extension_codec)?; + let sort_expr: Vec = + serialize_sorts(expr, extension_codec)?; Ok(protobuf::LogicalPlanNode { logical_plan_type: Some(LogicalPlanType::Sort(Box::new( protobuf::SortNode { input: Some(Box::new(input)), - expr: selection_expr, + expr: sort_expr, fetch: fetch.map(|f| f as i64).unwrap_or(-1i64), }, ))), @@ -1334,10 +1336,10 @@ impl AsLogicalPlan for LogicalPlanNode { column_defaults, }, )) => { - let mut converted_order_exprs: Vec = vec![]; + let mut converted_order_exprs: Vec = vec![]; for order in order_exprs { - let temp = LogicalExprNodeCollection { - logical_expr_nodes: serialize_exprs(order, extension_codec)?, + let temp = SortExprNodeCollection { + sort_expr_nodes: serialize_sorts(order, extension_codec)?, }; converted_order_exprs.push(temp); } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index bb7bf84a3387..b937c03f79d9 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -22,12 +22,12 @@ use datafusion_common::{TableReference, UnnestOptions}; use datafusion_expr::expr::{ self, Alias, Between, BinaryExpr, Cast, GroupingSet, InList, Like, Placeholder, - ScalarFunction, Sort, Unnest, + ScalarFunction, Unnest, }; use datafusion_expr::{ logical_plan::PlanType, logical_plan::StringifiedPlan, BuiltInWindowFunction, Expr, - JoinConstraint, JoinType, TryCast, WindowFrame, WindowFrameBound, WindowFrameUnits, - WindowFunctionDefinition, + JoinConstraint, JoinType, SortExpr, TryCast, WindowFrame, WindowFrameBound, + WindowFrameUnits, WindowFunctionDefinition, }; use crate::protobuf::{ @@ -343,7 +343,7 @@ pub fn serialize_expr( None }; let partition_by = serialize_exprs(partition_by, codec)?; - let order_by = serialize_exprs(order_by, codec)?; + let order_by = serialize_sorts(order_by, codec)?; let window_frame: Option = Some(window_frame.try_into()?); @@ -380,7 +380,7 @@ pub fn serialize_expr( None => None, }, order_by: match order_by { - Some(e) => serialize_exprs(e, codec)?, + Some(e) => serialize_sorts(e, codec)?, None => vec![], }, fun_definition: (!buf.is_empty()).then_some(buf), @@ -537,20 +537,6 @@ pub fn serialize_expr( expr_type: Some(ExprType::TryCast(expr)), } } - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => { - let expr = Box::new(protobuf::SortExprNode { - expr: Some(Box::new(serialize_expr(expr.as_ref(), codec)?)), - asc: *asc, - nulls_first: *nulls_first, - }); - protobuf::LogicalExprNode { - expr_type: Some(ExprType::Sort(expr)), - } - } Expr::Negative(expr) => { let expr = Box::new(protobuf::NegativeNode { expr: Some(Box::new(serialize_expr(expr.as_ref(), codec)?)), @@ -635,6 +621,30 @@ pub fn serialize_expr( Ok(expr_node) } +pub fn serialize_sorts<'a, I>( + sorts: I, + codec: &dyn LogicalExtensionCodec, +) -> Result, Error> +where + I: IntoIterator, +{ + sorts + .into_iter() + .map(|sort| { + let SortExpr { + expr, + asc, + nulls_first, + } = sort; + Ok(protobuf::SortExprNode { + expr: Some(serialize_expr(expr.as_ref(), codec)?), + asc: *asc, + nulls_first: *nulls_first, + }) + }) + .collect::, Error>>() +} + impl From for protobuf::TableReference { fn from(t: TableReference) -> Self { use protobuf::table_reference::TableReferenceEnum; diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 94ac913e1968..e174d1b50713 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -59,7 +59,7 @@ use datafusion_common::{ use datafusion_expr::dml::CopyTo; use datafusion_expr::expr::{ self, Between, BinaryExpr, Case, Cast, GroupingSet, InList, Like, ScalarFunction, - Sort, Unnest, WildcardOptions, + Unnest, WildcardOptions, }; use datafusion_expr::logical_plan::{Extension, UserDefinedLogicalNodeCore}; use datafusion_expr::{ @@ -1937,14 +1937,6 @@ fn roundtrip_try_cast() { roundtrip_expr_test(test_expr, ctx); } -#[test] -fn roundtrip_sort_expr() { - let test_expr = Expr::Sort(Sort::new(Box::new(lit(1.0_f32)), true, true)); - - let ctx = SessionContext::new(); - roundtrip_expr_test(test_expr, ctx); -} - #[test] fn roundtrip_negative() { let test_expr = Expr::Negative(Box::new(lit(1.0_f32))); diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 71e40c20b80a..9c768eb73c2e 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -282,22 +282,17 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let func_deps = schema.functional_dependencies(); // Find whether ties are possible in the given ordering let is_ordering_strict = order_by.iter().find_map(|orderby_expr| { - if let Expr::Sort(sort_expr) = orderby_expr { - if let Expr::Column(col) = sort_expr.expr.as_ref() { - let idx = schema.index_of_column(col).ok()?; - return if func_deps.iter().any(|dep| { - dep.source_indices == vec![idx] - && dep.mode == Dependency::Single - }) { - Some(true) - } else { - Some(false) - }; - } - Some(false) - } else { - panic!("order_by expression must be of type Sort"); + if let Expr::Column(col) = orderby_expr.expr.as_ref() { + let idx = schema.index_of_column(col).ok()?; + return if func_deps.iter().any(|dep| { + dep.source_indices == vec![idx] && dep.mode == Dependency::Single + }) { + Some(true) + } else { + Some(false) + }; } + Some(false) }); let window_frame = window diff --git a/datafusion/sql/src/expr/order_by.rs b/datafusion/sql/src/expr/order_by.rs index 7fb32f714cfa..cdaa787cedd0 100644 --- a/datafusion/sql/src/expr/order_by.rs +++ b/datafusion/sql/src/expr/order_by.rs @@ -20,7 +20,7 @@ use datafusion_common::{ not_impl_err, plan_datafusion_err, plan_err, Column, DFSchema, Result, }; use datafusion_expr::expr::Sort; -use datafusion_expr::Expr; +use datafusion_expr::{Expr, SortExpr}; use sqlparser::ast::{Expr as SQLExpr, OrderByExpr, Value}; impl<'a, S: ContextProvider> SqlToRel<'a, S> { @@ -44,7 +44,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { planner_context: &mut PlannerContext, literal_to_column: bool, additional_schema: Option<&DFSchema>, - ) -> Result> { + ) -> Result> { if exprs.is_empty() { return Ok(vec![]); } @@ -99,13 +99,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } }; let asc = asc.unwrap_or(true); - expr_vec.push(Expr::Sort(Sort::new( + expr_vec.push(Sort::new( Box::new(expr), asc, // when asc is true, by default nulls last to be consistent with postgres // postgres rule: https://www.postgresql.org/docs/current/queries-order.html nulls_first.unwrap_or(!asc), - ))) + )) } Ok(expr_vec) } diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index ba2b41bb6ecf..71328cfd018c 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -20,6 +20,7 @@ use std::sync::Arc; use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use datafusion_common::{not_impl_err, plan_err, Constraints, Result, ScalarValue}; +use datafusion_expr::expr::Sort; use datafusion_expr::{ CreateMemoryTable, DdlStatement, Distinct, Expr, LogicalPlan, LogicalPlanBuilder, Operator, @@ -119,7 +120,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { pub(super) fn order_by( &self, plan: LogicalPlan, - order_by: Vec, + order_by: Vec, ) -> Result { if order_by.is_empty() { return Ok(plan); diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 384893bfa94c..8a26671fcb6c 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -31,7 +31,7 @@ use datafusion_common::UnnestOptions; use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_expr::expr::{Alias, PlannedReplaceSelectItem, WildcardOptions}; use datafusion_expr::expr_rewriter::{ - normalize_col, normalize_col_with_schemas_and_ambiguity_check, normalize_cols, + normalize_col, normalize_col_with_schemas_and_ambiguity_check, normalize_sorts, }; use datafusion_expr::utils::{ expr_as_column_expr, expr_to_columns, find_aggregate_exprs, find_window_exprs, @@ -107,7 +107,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { true, Some(base_plan.schema().as_ref()), )?; - let order_by_rex = normalize_cols(order_by_rex, &projected_plan)?; + let order_by_rex = normalize_sorts(order_by_rex, &projected_plan)?; // this alias map is resolved and looked up in both having exprs and group by exprs let alias_map = extract_aliases(&select_exprs); diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index e75a96e78d48..3dfc379b039a 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -48,9 +48,10 @@ use datafusion_expr::{ CreateIndex as PlanCreateIndex, CreateMemoryTable, CreateView, DescribeTable, DmlStatement, DropCatalogSchema, DropFunction, DropTable, DropView, EmptyRelation, Explain, Expr, ExprSchemable, Filter, LogicalPlan, LogicalPlanBuilder, - OperateFunctionArg, PlanType, Prepare, SetVariable, Statement as PlanStatement, - ToStringifiedPlan, TransactionAccessMode, TransactionConclusion, TransactionEnd, - TransactionIsolationLevel, TransactionStart, Volatility, WriteOp, + OperateFunctionArg, PlanType, Prepare, SetVariable, SortExpr, + Statement as PlanStatement, ToStringifiedPlan, TransactionAccessMode, + TransactionConclusion, TransactionEnd, TransactionIsolationLevel, TransactionStart, + Volatility, WriteOp, }; use sqlparser::ast; use sqlparser::ast::{ @@ -952,7 +953,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { order_exprs: Vec, schema: &DFSchemaRef, planner_context: &mut PlannerContext, - ) -> Result>> { + ) -> Result>> { // Ask user to provide a schema if schema is empty. if !order_exprs.is_empty() && schema.fields().is_empty() { return plan_err!( @@ -966,8 +967,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let expr_vec = self.order_by_to_sort_expr(expr, schema, planner_context, true, None)?; // Verify that columns of all SortExprs exist in the schema: - for expr in expr_vec.iter() { - for column in expr.column_refs().iter() { + for sort in expr_vec.iter() { + for column in sort.expr.column_refs().iter() { if !schema.has_column(column) { // Return an error if any column is not in the schema: return plan_err!("Column {column} is not in schema"); diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 0dbcba162bc0..9a3f139fdee8 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -15,8 +15,6 @@ // specific language governing permissions and limitations // under the License. -use core::fmt; - use datafusion_expr::ScalarUDF; use sqlparser::ast::Value::SingleQuotedString; use sqlparser::ast::{ @@ -24,7 +22,7 @@ use sqlparser::ast::{ ObjectName, TimezoneInfo, UnaryOperator, }; use std::sync::Arc; -use std::{fmt::Display, vec}; +use std::vec; use super::dialect::{DateFieldExtractStyle, IntervalStyle}; use super::Unparser; @@ -46,33 +44,6 @@ use datafusion_expr::{ Between, BinaryExpr, Case, Cast, Expr, GroupingSet, Like, Operator, TryCast, }; -/// DataFusion's Exprs can represent either an `Expr` or an `OrderByExpr` -pub enum Unparsed { - // SQL Expression - Expr(ast::Expr), - // SQL ORDER BY expression (e.g. `col ASC NULLS FIRST`) - OrderByExpr(ast::OrderByExpr), -} - -impl Unparsed { - pub fn into_order_by_expr(self) -> Result { - if let Unparsed::OrderByExpr(order_by_expr) = self { - Ok(order_by_expr) - } else { - internal_err!("Expected Sort expression to be converted an OrderByExpr") - } - } -} - -impl Display for Unparsed { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - Unparsed::Expr(expr) => write!(f, "{}", expr), - Unparsed::OrderByExpr(order_by_expr) => write!(f, "{}", order_by_expr), - } - } -} - /// Convert a DataFusion [`Expr`] to [`ast::Expr`] /// /// This function is the opposite of [`SqlToRel::sql_to_expr`] and can be used @@ -106,13 +77,9 @@ pub fn expr_to_sql(expr: &Expr) -> Result { unparser.expr_to_sql(expr) } -/// Convert a DataFusion [`Expr`] to [`Unparsed`] -/// -/// This function is similar to expr_to_sql, but it supports converting more [`Expr`] types like -/// `Sort` expressions to `OrderByExpr` expressions. -pub fn expr_to_unparsed(expr: &Expr) -> Result { +pub fn sort_to_sql(sort: &Sort) -> Result { let unparser = Unparser::default(); - unparser.expr_to_unparsed(expr) + unparser.sort_to_sql(sort) } const LOWEST: &BinaryOperator = &BinaryOperator::Or; @@ -286,7 +253,7 @@ impl Unparser<'_> { }; let order_by: Vec = order_by .iter() - .map(|expr| expr_to_unparsed(expr)?.into_order_by_expr()) + .map(sort_to_sql) .collect::>>()?; let start_bound = self.convert_bound(&window_frame.start_bound)?; @@ -413,11 +380,6 @@ impl Unparser<'_> { negated: *negated, }) } - Expr::Sort(Sort { - expr: _, - asc: _, - nulls_first: _, - }) => plan_err!("Sort expression should be handled by expr_to_unparsed"), Expr::IsNull(expr) => { Ok(ast::Expr::IsNull(Box::new(self.expr_to_sql_inner(expr)?))) } @@ -534,36 +496,26 @@ impl Unparser<'_> { } } - /// This function can convert more [`Expr`] types than `expr_to_sql`, - /// returning an [`Unparsed`] like `Sort` expressions to `OrderByExpr` - /// expressions. - pub fn expr_to_unparsed(&self, expr: &Expr) -> Result { - match expr { - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => { - let sql_parser_expr = self.expr_to_sql(expr)?; + pub fn sort_to_sql(&self, sort: &Sort) -> Result { + let Sort { + expr, + asc, + nulls_first, + } = sort; + let sql_parser_expr = self.expr_to_sql(expr)?; - let nulls_first = if self.dialect.supports_nulls_first_in_sort() { - Some(*nulls_first) - } else { - None - }; + let nulls_first = if self.dialect.supports_nulls_first_in_sort() { + Some(*nulls_first) + } else { + None + }; - Ok(Unparsed::OrderByExpr(ast::OrderByExpr { - expr: sql_parser_expr, - asc: Some(*asc), - nulls_first, - with_fill: None, - })) - } - _ => { - let sql_parser_expr = self.expr_to_sql(expr)?; - Ok(Unparsed::Expr(sql_parser_expr)) - } - } + Ok(ast::OrderByExpr { + expr: sql_parser_expr, + asc: Some(*asc), + nulls_first, + with_fill: None, + }) } fn scalar_function_to_sql_overrides( @@ -1809,11 +1761,7 @@ mod tests { fun: WindowFunctionDefinition::AggregateUDF(count_udaf()), args: vec![wildcard()], partition_by: vec![], - order_by: vec![Expr::Sort(Sort::new( - Box::new(col("a")), - false, - true, - ))], + order_by: vec![Sort::new(Box::new(col("a")), false, true)], window_frame: WindowFrame::new_bounds( datafusion_expr::WindowFrameUnits::Range, datafusion_expr::WindowFrameBound::Preceding( @@ -1941,24 +1889,6 @@ mod tests { Ok(()) } - #[test] - fn expr_to_unparsed_ok() -> Result<()> { - let tests: Vec<(Expr, &str)> = vec![ - ((col("a") + col("b")).gt(lit(4)), r#"((a + b) > 4)"#), - (col("a").sort(true, true), r#"a ASC NULLS FIRST"#), - ]; - - for (expr, expected) in tests { - let ast = expr_to_unparsed(&expr)?; - - let actual = format!("{}", ast); - - assert_eq!(actual, expected); - } - - Ok(()) - } - #[test] fn custom_dialect_with_identifier_quote_style() -> Result<()> { let dialect = CustomDialectBuilder::new() @@ -2047,7 +1977,7 @@ mod tests { #[test] fn customer_dialect_support_nulls_first_in_ort() -> Result<()> { - let tests: Vec<(Expr, &str, bool)> = vec![ + let tests: Vec<(Sort, &str, bool)> = vec![ (col("a").sort(true, true), r#"a ASC NULLS FIRST"#, true), (col("a").sort(true, true), r#"a ASC"#, false), ]; @@ -2057,7 +1987,7 @@ mod tests { .with_supports_nulls_first_in_sort(supports_nulls_first_in_sort) .build(); let unparser = Unparser::new(&dialect); - let ast = unparser.expr_to_unparsed(&expr)?; + let ast = unparser.sort_to_sql(&expr)?; let actual = format!("{}", ast); diff --git a/datafusion/sql/src/unparser/mod.rs b/datafusion/sql/src/unparser/mod.rs index b2fd32566aa8..83ae64ba238b 100644 --- a/datafusion/sql/src/unparser/mod.rs +++ b/datafusion/sql/src/unparser/mod.rs @@ -29,8 +29,6 @@ pub use plan::plan_to_sql; use self::dialect::{DefaultDialect, Dialect}; pub mod dialect; -pub use expr::Unparsed; - /// Convert a DataFusion [`Expr`] to [`sqlparser::ast::Expr`] /// /// See [`expr_to_sql`] for background. `Unparser` allows greater control of diff --git a/datafusion/sql/src/unparser/plan.rs b/datafusion/sql/src/unparser/plan.rs index 106705c322fc..509c5dd52cd4 100644 --- a/datafusion/sql/src/unparser/plan.rs +++ b/datafusion/sql/src/unparser/plan.rs @@ -15,11 +15,10 @@ // specific language governing permissions and limitations // under the License. -use datafusion_common::{ - internal_err, not_impl_err, plan_err, Column, DataFusionError, Result, -}; +use datafusion_common::{internal_err, not_impl_err, Column, DataFusionError, Result}; use datafusion_expr::{ expr::Alias, Distinct, Expr, JoinConstraint, JoinType, LogicalPlan, Projection, + SortExpr, }; use sqlparser::ast::{self, Ident, SetExpr}; @@ -318,7 +317,7 @@ impl Unparser<'_> { return self.derive(plan, relation); } if let Some(query_ref) = query { - query_ref.order_by(self.sort_to_sql(sort.expr.clone())?); + query_ref.order_by(self.sorts_to_sql(sort.expr.clone())?); } else { return internal_err!( "Sort operator only valid in a statement context." @@ -361,7 +360,7 @@ impl Unparser<'_> { .collect::>>()?; if let Some(sort_expr) = &on.sort_expr { if let Some(query_ref) = query { - query_ref.order_by(self.sort_to_sql(sort_expr.clone())?); + query_ref.order_by(self.sorts_to_sql(sort_expr.clone())?); } else { return internal_err!( "Sort operator only valid in a statement context." @@ -525,14 +524,10 @@ impl Unparser<'_> { } } - fn sort_to_sql(&self, sort_exprs: Vec) -> Result> { + fn sorts_to_sql(&self, sort_exprs: Vec) -> Result> { sort_exprs .iter() - .map(|expr: &Expr| { - self.expr_to_unparsed(expr)? - .into_order_by_expr() - .or(plan_err!("Expecting Sort expr")) - }) + .map(|sort_expr| self.sort_to_sql(sort_expr)) .collect::>>() } diff --git a/datafusion/sql/src/unparser/rewrite.rs b/datafusion/sql/src/unparser/rewrite.rs index 9e1adcf4df31..522a08af8546 100644 --- a/datafusion/sql/src/unparser/rewrite.rs +++ b/datafusion/sql/src/unparser/rewrite.rs @@ -21,10 +21,11 @@ use std::{ }; use datafusion_common::{ - tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeIterator}, + tree_node::{Transformed, TransformedResult, TreeNode}, Result, }; -use datafusion_expr::{Expr, LogicalPlan, Projection, Sort}; +use datafusion_expr::tree_node::transform_sort_vec; +use datafusion_expr::{Expr, LogicalPlan, Projection, Sort, SortExpr}; use sqlparser::ast::Ident; /// Normalize the schema of a union plan to remove qualifiers from the schema fields and sort expressions. @@ -83,20 +84,18 @@ pub(super) fn normalize_union_schema(plan: &LogicalPlan) -> Result } /// Rewrite sort expressions that have a UNION plan as their input to remove the table reference. -fn rewrite_sort_expr_for_union(exprs: Vec) -> Result> { - let sort_exprs: Vec = exprs - .into_iter() - .map_until_stop_and_collect(|expr| { - expr.transform_up(|expr| { - if let Expr::Column(mut col) = expr { - col.relation = None; - Ok(Transformed::yes(Expr::Column(col))) - } else { - Ok(Transformed::no(expr)) - } - }) +fn rewrite_sort_expr_for_union(exprs: Vec) -> Result> { + let sort_exprs = transform_sort_vec(exprs, &mut |expr| { + expr.transform_up(|expr| { + if let Expr::Column(mut col) = expr { + col.relation = None; + Ok(Transformed::yes(Expr::Column(col))) + } else { + Ok(Transformed::no(expr)) + } }) - .data()?; + }) + .data()?; Ok(sort_exprs) } @@ -158,12 +157,8 @@ pub(super) fn rewrite_plan_for_sort_on_non_projected_fields( .collect::>(); let mut collects = p.expr.clone(); - for expr in &sort.expr { - if let Expr::Sort(s) = expr { - collects.push(s.expr.as_ref().clone()); - } else { - panic!("sort expression must be of type Sort"); - } + for sort in &sort.expr { + collects.push(sort.expr.as_ref().clone()); } // Compare outer collects Expr::to_string with inner collected transformed values diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index b1b510f1792d..05903bb56cfe 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -31,7 +31,7 @@ use datafusion::logical_expr::expr::{Exists, InSubquery, Sort}; use datafusion::logical_expr::{ expr::find_df_window_func, Aggregate, BinaryExpr, Case, EmptyRelation, Expr, - ExprSchemable, LogicalPlan, Operator, Projection, Values, + ExprSchemable, LogicalPlan, Operator, Projection, SortExpr, Values, }; use substrait::proto::expression::subquery::set_predicate::PredicateOp; use url::Url; @@ -900,8 +900,8 @@ pub async fn from_substrait_sorts( substrait_sorts: &Vec, input_schema: &DFSchema, extensions: &Extensions, -) -> Result> { - let mut sorts: Vec = vec![]; +) -> Result> { + let mut sorts: Vec = vec![]; for s in substrait_sorts { let expr = from_substrait_rex(ctx, s.expr.as_ref().unwrap(), input_schema, extensions) @@ -935,11 +935,11 @@ pub async fn from_substrait_sorts( None => not_impl_err!("Sort without sort kind is invalid"), }; let (asc, nulls_first) = asc_nullfirst.unwrap(); - sorts.push(Expr::Sort(Sort { + sorts.push(Sort { expr: Box::new(expr), asc, nulls_first, - })); + }); } Ok(sorts) } @@ -986,7 +986,7 @@ pub async fn from_substrait_agg_func( input_schema: &DFSchema, extensions: &Extensions, filter: Option>, - order_by: Option>, + order_by: Option>, distinct: bool, ) -> Result> { let args = diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 72b6760be29c..592390a285ba 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -808,31 +808,26 @@ pub fn to_substrait_agg_measure( /// Converts sort expression to corresponding substrait `SortField` fn to_substrait_sort_field( ctx: &SessionContext, - expr: &Expr, + sort: &Sort, schema: &DFSchemaRef, extensions: &mut Extensions, ) -> Result { - match expr { - Expr::Sort(sort) => { - let sort_kind = match (sort.asc, sort.nulls_first) { - (true, true) => SortDirection::AscNullsFirst, - (true, false) => SortDirection::AscNullsLast, - (false, true) => SortDirection::DescNullsFirst, - (false, false) => SortDirection::DescNullsLast, - }; - Ok(SortField { - expr: Some(to_substrait_rex( - ctx, - sort.expr.deref(), - schema, - 0, - extensions, - )?), - sort_kind: Some(SortKind::Direction(sort_kind.into())), - }) - } - _ => exec_err!("expects to receive sort expression"), - } + let sort_kind = match (sort.asc, sort.nulls_first) { + (true, true) => SortDirection::AscNullsFirst, + (true, false) => SortDirection::AscNullsLast, + (false, true) => SortDirection::DescNullsFirst, + (false, false) => SortDirection::DescNullsLast, + }; + Ok(SortField { + expr: Some(to_substrait_rex( + ctx, + sort.expr.deref(), + schema, + 0, + extensions, + )?), + sort_kind: Some(SortKind::Direction(sort_kind.into())), + }) } /// Return Substrait scalar function with two arguments @@ -2107,30 +2102,26 @@ fn try_to_substrait_field_reference( fn substrait_sort_field( ctx: &SessionContext, - expr: &Expr, + sort: &Sort, schema: &DFSchemaRef, extensions: &mut Extensions, ) -> Result { - match expr { - Expr::Sort(Sort { - expr, - asc, - nulls_first, - }) => { - let e = to_substrait_rex(ctx, expr, schema, 0, extensions)?; - let d = match (asc, nulls_first) { - (true, true) => SortDirection::AscNullsFirst, - (true, false) => SortDirection::AscNullsLast, - (false, true) => SortDirection::DescNullsFirst, - (false, false) => SortDirection::DescNullsLast, - }; - Ok(SortField { - expr: Some(e), - sort_kind: Some(SortKind::Direction(d as i32)), - }) - } - _ => not_impl_err!("Expecting sort expression but got {expr:?}"), - } + let Sort { + expr, + asc, + nulls_first, + } = sort; + let e = to_substrait_rex(ctx, expr, schema, 0, extensions)?; + let d = match (asc, nulls_first) { + (true, true) => SortDirection::AscNullsFirst, + (true, false) => SortDirection::AscNullsLast, + (false, true) => SortDirection::DescNullsFirst, + (false, false) => SortDirection::DescNullsLast, + }; + Ok(SortField { + expr: Some(e), + sort_kind: Some(SortKind::Direction(d as i32)), + }) } fn substrait_field_ref(index: usize) -> Result { diff --git a/docs/source/library-user-guide/using-the-dataframe-api.md b/docs/source/library-user-guide/using-the-dataframe-api.md index 3bd47ef50e51..7f3e28c255c6 100644 --- a/docs/source/library-user-guide/using-the-dataframe-api.md +++ b/docs/source/library-user-guide/using-the-dataframe-api.md @@ -263,14 +263,14 @@ async fn main() -> Result<()>{ let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; // Create a new DataFrame sorted by `id`, `bank_account` let new_df = df.select(vec![col("a"), col("b")])? - .sort(vec![col("a")])?; + .sort_by(vec![col("a")])?; // Build the same plan using the LogicalPlanBuilder // Similar to `SELECT a, b FROM example.csv ORDER BY a` let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; let (_state, plan) = df.into_parts(); // get the DataFrame's LogicalPlan let plan = LogicalPlanBuilder::from(plan) .project(vec![col("a"), col("b")])? - .sort(vec![col("a")])? + .sort_by(vec![col("a")])? .build()?; // prove they are the same assert_eq!(new_df.logical_plan(), &plan); From 095feb9cfc03bcfa0c34ad74e77b570174fc0244 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 29 Aug 2024 17:39:35 +0200 Subject: [PATCH 26/75] Remove normalize_with_schemas function (#12233) It was deprecated since 20.0.0. --- datafusion/common/src/column.rs | 75 +----------------------- datafusion/expr/src/expr_rewriter/mod.rs | 2 +- 2 files changed, 2 insertions(+), 75 deletions(-) diff --git a/datafusion/common/src/column.rs b/datafusion/common/src/column.rs index 2e2bfff40340..d855198fa7c6 100644 --- a/datafusion/common/src/column.rs +++ b/datafusion/common/src/column.rs @@ -26,7 +26,6 @@ use std::collections::HashSet; use std::convert::Infallible; use std::fmt; use std::str::FromStr; -use std::sync::Arc; /// A named reference to a qualified field in a schema. #[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] @@ -156,79 +155,6 @@ impl Column { } } - /// Qualify column if not done yet. - /// - /// If this column already has a [relation](Self::relation), it will be returned as is and the given parameters are - /// ignored. Otherwise this will search through the given schemas to find the column. This will use the first schema - /// that matches. - /// - /// A schema matches if there is a single column that -- when unqualified -- matches this column. There is an - /// exception for `USING` statements, see below. - /// - /// # Using columns - /// Take the following SQL statement: - /// - /// ```sql - /// SELECT id FROM t1 JOIN t2 USING(id) - /// ``` - /// - /// In this case, both `t1.id` and `t2.id` will match unqualified column `id`. To express this possibility, use - /// `using_columns`. Each entry in this array is a set of columns that are bound together via a `USING` clause. So - /// in this example this would be `[{t1.id, t2.id}]`. - #[deprecated( - since = "20.0.0", - note = "use normalize_with_schemas_and_ambiguity_check instead" - )] - pub fn normalize_with_schemas( - self, - schemas: &[&Arc], - using_columns: &[HashSet], - ) -> Result { - if self.relation.is_some() { - return Ok(self); - } - - for schema in schemas { - let qualified_fields = - schema.qualified_fields_with_unqualified_name(&self.name); - match qualified_fields.len() { - 0 => continue, - 1 => { - return Ok(Column::from(qualified_fields[0])); - } - _ => { - // More than 1 fields in this schema have their names set to self.name. - // - // This should only happen when a JOIN query with USING constraint references - // join columns using unqualified column name. For example: - // - // ```sql - // SELECT id FROM t1 JOIN t2 USING(id) - // ``` - // - // In this case, both `t1.id` and `t2.id` will match unqualified column `id`. - // We will use the relation from the first matched field to normalize self. - - // Compare matched fields with one USING JOIN clause at a time - let columns = schema.columns_with_unqualified_name(&self.name); - for using_col in using_columns { - let all_matched = columns.iter().all(|f| using_col.contains(f)); - // All matched fields belong to the same using column set, in orther words - // the same join clause. We simply pick the qualifier from the first match. - if all_matched { - return Ok(columns[0].clone()); - } - } - } - } - } - - _schema_err!(SchemaError::FieldNotFound { - field: Box::new(Column::new(self.relation.clone(), self.name)), - valid_fields: schemas.iter().flat_map(|s| s.columns()).collect(), - }) - } - /// Qualify column if not done yet. /// /// If this column already has a [relation](Self::relation), it will be returned as is and the given parameters are @@ -381,6 +307,7 @@ mod tests { use super::*; use arrow::datatypes::DataType; use arrow_schema::SchemaBuilder; + use std::sync::Arc; fn create_qualified_schema(qualifier: &str, names: Vec<&str>) -> Result { let mut schema_builder = SchemaBuilder::new(); diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index b809b015d929..5e7fedb4cbd8 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -60,7 +60,7 @@ pub trait FunctionRewrite { ) -> Result>; } -/// Recursively call [`Column::normalize_with_schemas`] on all [`Column`] expressions +/// Recursively call `LogicalPlanBuilder::normalize` on all [`Column`] expressions /// in the `expr` expression tree. pub fn normalize_col(expr: Expr, plan: &LogicalPlan) -> Result { expr.transform(|expr| { From 827d7e125b8876e0c706d1f7edaaabf6ce9e37b0 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 29 Aug 2024 17:40:47 +0200 Subject: [PATCH 27/75] Update AWS dependencies in CLI (#12229) --- datafusion-cli/Cargo.lock | 355 ++++++++++++--------------- datafusion-cli/Cargo.toml | 4 +- datafusion-cli/src/object_storage.rs | 3 +- 3 files changed, 160 insertions(+), 202 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 1e89bb3af87e..ddc6242977d3 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -473,160 +473,155 @@ checksum = "0c4b4d0bd25bd0b74681c0ad21497610ce1b7c91b1022cd21c80c6fbdd9476b0" [[package]] name = "aws-config" -version = "0.55.3" +version = "1.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bcdcf0d683fe9c23d32cf5b53c9918ea0a500375a9fb20109802552658e576c9" +checksum = "4e95816a168520d72c0e7680c405a5a8c1fb6a035b4bc4b9d7b0de8e1a941697" dependencies = [ "aws-credential-types", - "aws-http", + "aws-runtime", "aws-sdk-sso", + "aws-sdk-ssooidc", "aws-sdk-sts", "aws-smithy-async", - "aws-smithy-client", "aws-smithy-http", - "aws-smithy-http-tower", "aws-smithy-json", + "aws-smithy-runtime", + "aws-smithy-runtime-api", "aws-smithy-types", "aws-types", "bytes", - "fastrand 1.9.0", + "fastrand", "hex", "http 0.2.12", - "hyper 0.14.30", - "ring 0.16.20", + "ring", "time", "tokio", - "tower", "tracing", + "url", "zeroize", ] [[package]] name = "aws-credential-types" -version = "0.55.3" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fcdb2f7acbc076ff5ad05e7864bdb191ca70a6fd07668dc3a1a8bcd051de5ae" +checksum = "e16838e6c9e12125face1c1eff1343c75e3ff540de98ff7ebd61874a89bcfeb9" dependencies = [ "aws-smithy-async", + "aws-smithy-runtime-api", "aws-smithy-types", - "fastrand 1.9.0", - "tokio", - "tracing", "zeroize", ] [[package]] -name = "aws-endpoint" -version = "0.55.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8cce1c41a6cfaa726adee9ebb9a56fcd2bbfd8be49fd8a04c5e20fd968330b04" -dependencies = [ - "aws-smithy-http", - "aws-smithy-types", - "aws-types", - "http 0.2.12", - "regex", - "tracing", -] - -[[package]] -name = "aws-http" -version = "0.55.3" +name = "aws-runtime" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aadbc44e7a8f3e71c8b374e03ecd972869eb91dd2bc89ed018954a52ba84bc44" +checksum = "f42c2d4218de4dcd890a109461e2f799a1a2ba3bcd2cde9af88360f5df9266c6" dependencies = [ "aws-credential-types", + "aws-sigv4", + "aws-smithy-async", "aws-smithy-http", + "aws-smithy-runtime-api", "aws-smithy-types", "aws-types", "bytes", + "fastrand", "http 0.2.12", "http-body 0.4.6", - "lazy_static", + "once_cell", "percent-encoding", "pin-project-lite", "tracing", + "uuid", ] [[package]] name = "aws-sdk-sso" -version = "0.28.0" +version = "1.39.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8b812340d86d4a766b2ca73f740dfd47a97c2dff0c06c8517a16d88241957e4" +checksum = "11822090cf501c316c6f75711d77b96fba30658e3867a7762e5e2f5d32d31e81" dependencies = [ "aws-credential-types", - "aws-endpoint", - "aws-http", - "aws-sig-auth", + "aws-runtime", "aws-smithy-async", - "aws-smithy-client", "aws-smithy-http", - "aws-smithy-http-tower", "aws-smithy-json", + "aws-smithy-runtime", + "aws-smithy-runtime-api", "aws-smithy-types", "aws-types", "bytes", "http 0.2.12", - "regex", - "tokio-stream", - "tower", + "once_cell", + "regex-lite", "tracing", ] [[package]] -name = "aws-sdk-sts" -version = "0.28.0" +name = "aws-sdk-ssooidc" +version = "1.40.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "265fac131fbfc188e5c3d96652ea90ecc676a934e3174eaaee523c6cec040b3b" +checksum = "78a2a06ff89176123945d1bbe865603c4d7101bea216a550bb4d2e4e9ba74d74" dependencies = [ "aws-credential-types", - "aws-endpoint", - "aws-http", - "aws-sig-auth", + "aws-runtime", "aws-smithy-async", - "aws-smithy-client", "aws-smithy-http", - "aws-smithy-http-tower", "aws-smithy-json", - "aws-smithy-query", + "aws-smithy-runtime", + "aws-smithy-runtime-api", "aws-smithy-types", - "aws-smithy-xml", "aws-types", "bytes", "http 0.2.12", - "regex", - "tower", + "once_cell", + "regex-lite", "tracing", ] [[package]] -name = "aws-sig-auth" -version = "0.55.3" +name = "aws-sdk-sts" +version = "1.39.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3b94acb10af0c879ecd5c7bdf51cda6679a0a4f4643ce630905a77673bfa3c61" +checksum = "a20a91795850826a6f456f4a48eff1dfa59a0e69bdbf5b8c50518fd372106574" dependencies = [ "aws-credential-types", - "aws-sigv4", + "aws-runtime", + "aws-smithy-async", "aws-smithy-http", + "aws-smithy-json", + "aws-smithy-query", + "aws-smithy-runtime", + "aws-smithy-runtime-api", + "aws-smithy-types", + "aws-smithy-xml", "aws-types", "http 0.2.12", + "once_cell", + "regex-lite", "tracing", ] [[package]] name = "aws-sigv4" -version = "0.55.3" +version = "1.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d2ce6f507be68e968a33485ced670111d1cbad161ddbbab1e313c03d37d8f4c" +checksum = "5df1b0fa6be58efe9d4ccc257df0a53b89cd8909e86591a13ca54817c87517be" dependencies = [ + "aws-credential-types", "aws-smithy-http", + "aws-smithy-runtime-api", + "aws-smithy-types", + "bytes", "form_urlencoded", "hex", "hmac", "http 0.2.12", + "http 1.1.0", "once_cell", "percent-encoding", - "regex", "sha2", "time", "tracing", @@ -634,53 +629,28 @@ dependencies = [ [[package]] name = "aws-smithy-async" -version = "0.55.3" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13bda3996044c202d75b91afeb11a9afae9db9a721c6a7a427410018e286b880" +checksum = "62220bc6e97f946ddd51b5f1361f78996e704677afc518a4ff66b7a72ea1378c" dependencies = [ "futures-util", "pin-project-lite", "tokio", - "tokio-stream", -] - -[[package]] -name = "aws-smithy-client" -version = "0.55.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a86aa6e21e86c4252ad6a0e3e74da9617295d8d6e374d552be7d3059c41cedd" -dependencies = [ - "aws-smithy-async", - "aws-smithy-http", - "aws-smithy-http-tower", - "aws-smithy-types", - "bytes", - "fastrand 1.9.0", - "http 0.2.12", - "http-body 0.4.6", - "hyper 0.14.30", - "hyper-rustls 0.23.2", - "lazy_static", - "pin-project-lite", - "rustls 0.20.9", - "tokio", - "tower", - "tracing", ] [[package]] name = "aws-smithy-http" -version = "0.55.3" +version = "0.60.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b3b693869133551f135e1f2c77cb0b8277d9e3e17feaf2213f735857c4f0d28" +checksum = "01dbcb6e2588fd64cfb6d7529661b06466419e4c54ed1c62d6510d2d0350a728" dependencies = [ + "aws-smithy-runtime-api", "aws-smithy-types", "bytes", "bytes-utils", "futures-core", "http 0.2.12", "http-body 0.4.6", - "hyper 0.14.30", "once_cell", "percent-encoding", "pin-project-lite", @@ -689,74 +659,110 @@ dependencies = [ ] [[package]] -name = "aws-smithy-http-tower" -version = "0.55.3" +name = "aws-smithy-json" +version = "0.60.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ae4f6c5798a247fac98a867698197d9ac22643596dc3777f0c76b91917616b9" +checksum = "4683df9469ef09468dad3473d129960119a0d3593617542b7d52086c8486f2d6" dependencies = [ - "aws-smithy-http", "aws-smithy-types", - "bytes", - "http 0.2.12", - "http-body 0.4.6", - "pin-project-lite", - "tower", - "tracing", ] [[package]] -name = "aws-smithy-json" -version = "0.55.3" +name = "aws-smithy-query" +version = "0.60.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23f9f42fbfa96d095194a632fbac19f60077748eba536eb0b9fecc28659807f8" +checksum = "f2fbd61ceb3fe8a1cb7352e42689cec5335833cd9f94103a61e98f9bb61c64bb" dependencies = [ "aws-smithy-types", + "urlencoding", ] [[package]] -name = "aws-smithy-query" -version = "0.55.3" +name = "aws-smithy-runtime" +version = "1.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "98819eb0b04020a1c791903533b638534ae6c12e2aceda3e6e6fba015608d51d" +checksum = "d1ce695746394772e7000b39fe073095db6d45a862d0767dd5ad0ac0d7f8eb87" dependencies = [ + "aws-smithy-async", + "aws-smithy-http", + "aws-smithy-runtime-api", "aws-smithy-types", - "urlencoding", + "bytes", + "fastrand", + "h2 0.3.26", + "http 0.2.12", + "http-body 0.4.6", + "http-body 1.0.1", + "httparse", + "hyper 0.14.30", + "hyper-rustls 0.24.2", + "once_cell", + "pin-project-lite", + "pin-utils", + "rustls 0.21.12", + "tokio", + "tracing", +] + +[[package]] +name = "aws-smithy-runtime-api" +version = "1.7.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e086682a53d3aa241192aa110fa8dfce98f2f5ac2ead0de84d41582c7e8fdb96" +dependencies = [ + "aws-smithy-async", + "aws-smithy-types", + "bytes", + "http 0.2.12", + "http 1.1.0", + "pin-project-lite", + "tokio", + "tracing", + "zeroize", ] [[package]] name = "aws-smithy-types" -version = "0.55.3" +version = "1.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16a3d0bf4f324f4ef9793b86a1701d9700fbcdbd12a846da45eed104c634c6e8" +checksum = "273dcdfd762fae3e1650b8024624e7cd50e484e37abdab73a7a706188ad34543" dependencies = [ "base64-simd", + "bytes", + "bytes-utils", + "http 0.2.12", + "http 1.1.0", + "http-body 0.4.6", + "http-body 1.0.1", + "http-body-util", "itoa", "num-integer", + "pin-project-lite", + "pin-utils", "ryu", + "serde", "time", ] [[package]] name = "aws-smithy-xml" -version = "0.55.3" +version = "0.60.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1b9d12875731bd07e767be7baad95700c3137b56730ec9ddeedb52a5e5ca63b" +checksum = "d123fbc2a4adc3c301652ba8e149bf4bc1d1725affb9784eb20c953ace06bf55" dependencies = [ "xmlparser", ] [[package]] name = "aws-types" -version = "0.55.3" +version = "1.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6dd209616cc8d7bfb82f87811a5c655dc97537f592689b18743bddf5dc5c4829" +checksum = "5221b91b3e441e6675310829fd8984801b772cb1546ef6c0e54dec9f1ac13fef" dependencies = [ "aws-credential-types", "aws-smithy-async", - "aws-smithy-client", - "aws-smithy-http", + "aws-smithy-runtime-api", "aws-smithy-types", - "http 0.2.12", "rustc_version", "tracing", ] @@ -1689,15 +1695,6 @@ dependencies = [ "str-buf", ] -[[package]] -name = "fastrand" -version = "1.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e51093e27b0797c359783294ca4f0a911c270184cb10f85783b118614a1501be" -dependencies = [ - "instant", -] - [[package]] name = "fastrand" version = "2.1.0" @@ -2114,17 +2111,18 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.23.2" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1788965e61b367cd03a62950836d5cd41560c3577d90e40e0819373194d1661c" +checksum = "ec3efd23720e2049821a693cbc7e65ea87c72f1c58ff2f9522ff332b1491e590" dependencies = [ + "futures-util", "http 0.2.12", "hyper 0.14.30", "log", - "rustls 0.20.9", + "rustls 0.21.12", "rustls-native-certs 0.6.3", "tokio", - "tokio-rustls 0.23.4", + "tokio-rustls 0.24.1", ] [[package]] @@ -2645,7 +2643,7 @@ dependencies = [ "quick-xml", "rand", "reqwest", - "ring 0.17.8", + "ring", "rustls-pemfile 2.1.3", "serde", "serde_json", @@ -2945,7 +2943,7 @@ checksum = "ba92fb39ec7ad06ca2582c0ca834dfeadcaf06ddfc8e635c80aa7e1c05315fdd" dependencies = [ "bytes", "rand", - "ring 0.17.8", + "ring", "rustc-hash", "rustls 0.23.12", "slab", @@ -3116,21 +3114,6 @@ dependencies = [ "windows-registry", ] -[[package]] -name = "ring" -version = "0.16.20" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3053cf52e236a3ed746dfc745aa9cacf1b791d846bdaf412f60a8d7d6e17c8fc" -dependencies = [ - "cc", - "libc", - "once_cell", - "spin 0.5.2", - "untrusted 0.7.1", - "web-sys", - "winapi", -] - [[package]] name = "ring" version = "0.17.8" @@ -3141,8 +3124,8 @@ dependencies = [ "cfg-if", "getrandom", "libc", - "spin 0.9.8", - "untrusted 0.9.0", + "spin", + "untrusted", "windows-sys 0.52.0", ] @@ -3214,14 +3197,14 @@ dependencies = [ [[package]] name = "rustls" -version = "0.20.9" +version = "0.21.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b80e3dec595989ea8510028f30c408a4630db12c9cbb8de34203b89d6577e99" +checksum = "3f56a14d1f48b391359b22f731fd4bd7e43c97f3c50eee276f3aa09c94784d3e" dependencies = [ "log", - "ring 0.16.20", + "ring", + "rustls-webpki 0.101.7", "sct", - "webpki", ] [[package]] @@ -3231,9 +3214,9 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c58f8c84392efc0a126acce10fa59ff7b3d2ac06ab451a33f2741989b806b044" dependencies = [ "once_cell", - "ring 0.17.8", + "ring", "rustls-pki-types", - "rustls-webpki", + "rustls-webpki 0.102.6", "subtle", "zeroize", ] @@ -3288,15 +3271,25 @@ version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fc0a2ce646f8655401bb81e7927b812614bd5d91dbc968696be50603510fcaf0" +[[package]] +name = "rustls-webpki" +version = "0.101.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b6275d1ee7a1cd780b64aca7726599a1dbc893b1e64144529e55c3c2f745765" +dependencies = [ + "ring", + "untrusted", +] + [[package]] name = "rustls-webpki" version = "0.102.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8e6b52d4fda176fd835fdc55a835d4a89b8499cad995885a21149d5ad62f852e" dependencies = [ - "ring 0.17.8", + "ring", "rustls-pki-types", - "untrusted 0.9.0", + "untrusted", ] [[package]] @@ -3364,8 +3357,8 @@ version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "da046153aa2352493d6cb7da4b6e5c0c057d8a1d0a9aa8560baffdd945acd414" dependencies = [ - "ring 0.17.8", - "untrusted 0.9.0", + "ring", + "untrusted", ] [[package]] @@ -3532,12 +3525,6 @@ dependencies = [ "windows-sys 0.52.0", ] -[[package]] -name = "spin" -version = "0.5.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" - [[package]] name = "spin" version = "0.9.8" @@ -3668,7 +3655,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "04cbcdd0c794ebb0d4cf35e88edd2f7d2c4c3e9a5a6dab322839b321c6a87a64" dependencies = [ "cfg-if", - "fastrand 2.1.0", + "fastrand", "once_cell", "rustix", "windows-sys 0.59.0", @@ -3805,13 +3792,12 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.23.4" +version = "0.24.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c43ee83903113e03984cb9e5cebe6c04a5116269e900e3ddba8f068a62adda59" +checksum = "c28327cf380ac148141087fbfb9de9d7bd4e84ab5d2c28fbc911d753de8a7081" dependencies = [ - "rustls 0.20.9", + "rustls 0.21.12", "tokio", - "webpki", ] [[package]] @@ -3825,17 +3811,6 @@ dependencies = [ "tokio", ] -[[package]] -name = "tokio-stream" -version = "0.1.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "267ac89e0bec6e691e5813911606935d77c476ff49024f98abcea3e7b15e37af" -dependencies = [ - "futures-core", - "pin-project-lite", - "tokio", -] - [[package]] name = "tokio-util" version = "0.7.11" @@ -3862,7 +3837,6 @@ dependencies = [ "tokio", "tower-layer", "tower-service", - "tracing", ] [[package]] @@ -3883,7 +3857,6 @@ version = "0.1.40" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c3523ab5a71916ccf420eebdf5521fcef02141234bbc0b8a49f2fdc4544364ef" dependencies = [ - "log", "pin-project-lite", "tracing-attributes", "tracing-core", @@ -3984,12 +3957,6 @@ version = "0.1.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0336d538f7abc86d282a4189614dfaa90810dfc2c6f6427eaf88e16311dd225d" -[[package]] -name = "untrusted" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a156c684c91ea7d62626509bce3cb4e1d9ed5c4d978f7b4352658f96a4c26b4a" - [[package]] name = "untrusted" version = "0.9.0" @@ -4165,16 +4132,6 @@ dependencies = [ "wasm-bindgen", ] -[[package]] -name = "webpki" -version = "0.22.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ed63aea5ce73d0ff405984102c42de94fc55a6b75765d621c65262469b3c9b53" -dependencies = [ - "ring 0.17.8", - "untrusted 0.9.0", -] - [[package]] name = "winapi" version = "0.3.9" diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 0a4523a1c04e..f477bad69a2c 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -32,8 +32,8 @@ readme = "README.md" [dependencies] arrow = { version = "52.2.0" } async-trait = "0.1.73" -aws-config = "0.55" -aws-credential-types = "0.55" +aws-config = "1.5.5" +aws-credential-types = "1.2.0" clap = { version = "4.5.16", features = ["derive", "cargo"] } datafusion = { path = "../datafusion/core", version = "41.0.0", features = [ "avro", diff --git a/datafusion-cli/src/object_storage.rs b/datafusion-cli/src/object_storage.rs index 87eb04d113de..e8d60e4f0926 100644 --- a/datafusion-cli/src/object_storage.rs +++ b/datafusion-cli/src/object_storage.rs @@ -27,6 +27,7 @@ use datafusion::error::{DataFusionError, Result}; use datafusion::execution::context::SessionState; use async_trait::async_trait; +use aws_config::BehaviorVersion; use aws_credential_types::provider::ProvideCredentials; use object_store::aws::{AmazonS3Builder, AwsCredential}; use object_store::gcp::GoogleCloudStorageBuilder; @@ -61,7 +62,7 @@ pub async fn get_s3_object_store_builder( builder = builder.with_token(session_token); } } else { - let config = aws_config::from_env().load().await; + let config = aws_config::defaults(BehaviorVersion::latest()).load().await; if let Some(region) = config.region() { builder = builder.with_region(region.to_string()); } From 0f16849ea2025cfecb25d23eee68f8ec25edadd0 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 29 Aug 2024 17:47:35 +0200 Subject: [PATCH 28/75] Avoid Arc::clone when serializing physical expressions (#12235) `Arc::clone` is indispensable when passing shared references between threads. For synchronous code`&` and `&Arc` can be (and often are) used, with the latter being future-compatible, should the code start to need `Arc::clone` (e.g. due to parallelization). --- datafusion/proto/src/physical_plan/mod.rs | 56 +++++------ .../proto/src/physical_plan/to_proto.rs | 93 ++++++------------- .../tests/cases/roundtrip_physical_plan.rs | 3 +- 3 files changed, 55 insertions(+), 97 deletions(-) diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 78f370c714cc..04cbf8b537b3 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1122,7 +1122,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { let expr = exec .expr() .iter() - .map(|expr| serialize_physical_expr(Arc::clone(&expr.0), extension_codec)) + .map(|expr| serialize_physical_expr(&expr.0, extension_codec)) .collect::>>()?; let expr_name = exec.expr().iter().map(|expr| expr.1.clone()).collect(); return Ok(protobuf::PhysicalPlanNode { @@ -1163,7 +1163,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { protobuf::FilterExecNode { input: Some(Box::new(input)), expr: Some(serialize_physical_expr( - Arc::clone(exec.predicate()), + exec.predicate(), extension_codec, )?), default_filter_selectivity: exec.default_selectivity() as u32, @@ -1220,8 +1220,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .on() .iter() .map(|tuple| { - let l = serialize_physical_expr(tuple.0.to_owned(), extension_codec)?; - let r = serialize_physical_expr(tuple.1.to_owned(), extension_codec)?; + let l = serialize_physical_expr(&tuple.0, extension_codec)?; + let r = serialize_physical_expr(&tuple.1, extension_codec)?; Ok::<_, DataFusionError>(protobuf::JoinOn { left: Some(l), right: Some(r), @@ -1233,10 +1233,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .filter() .as_ref() .map(|f| { - let expression = serialize_physical_expr( - f.expression().to_owned(), - extension_codec, - )?; + let expression = + serialize_physical_expr(f.expression(), extension_codec)?; let column_indices = f .column_indices() .iter() @@ -1294,8 +1292,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .on() .iter() .map(|tuple| { - let l = serialize_physical_expr(tuple.0.to_owned(), extension_codec)?; - let r = serialize_physical_expr(tuple.1.to_owned(), extension_codec)?; + let l = serialize_physical_expr(&tuple.0, extension_codec)?; + let r = serialize_physical_expr(&tuple.1, extension_codec)?; Ok::<_, DataFusionError>(protobuf::JoinOn { left: Some(l), right: Some(r), @@ -1307,10 +1305,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .filter() .as_ref() .map(|f| { - let expression = serialize_physical_expr( - f.expression().to_owned(), - extension_codec, - )?; + let expression = + serialize_physical_expr(f.expression(), extension_codec)?; let column_indices = f .column_indices() .iter() @@ -1348,7 +1344,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .map(|expr| { Ok(protobuf::PhysicalSortExprNode { expr: Some(Box::new(serialize_physical_expr( - expr.expr.to_owned(), + &expr.expr, extension_codec, )?)), asc: !expr.options.descending, @@ -1368,7 +1364,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .map(|expr| { Ok(protobuf::PhysicalSortExprNode { expr: Some(Box::new(serialize_physical_expr( - expr.expr.to_owned(), + &expr.expr, extension_codec, )?)), asc: !expr.options.descending, @@ -1475,14 +1471,14 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .group_expr() .null_expr() .iter() - .map(|expr| serialize_physical_expr(expr.0.to_owned(), extension_codec)) + .map(|expr| serialize_physical_expr(&expr.0, extension_codec)) .collect::>>()?; let group_expr = exec .group_expr() .expr() .iter() - .map(|expr| serialize_physical_expr(expr.0.to_owned(), extension_codec)) + .map(|expr| serialize_physical_expr(&expr.0, extension_codec)) .collect::>>()?; let limit = exec.limit().map(|value| protobuf::AggLimit { @@ -1581,7 +1577,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { if let Some(exec) = plan.downcast_ref::() { let predicate = exec .predicate() - .map(|pred| serialize_physical_expr(Arc::clone(pred), extension_codec)) + .map(|pred| serialize_physical_expr(pred, extension_codec)) .transpose()?; return Ok(protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::ParquetScan( @@ -1653,7 +1649,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .map(|expr| { let sort_expr = Box::new(protobuf::PhysicalSortExprNode { expr: Some(Box::new(serialize_physical_expr( - expr.expr.to_owned(), + &expr.expr, extension_codec, )?)), asc: !expr.options.descending, @@ -1722,7 +1718,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .map(|expr| { let sort_expr = Box::new(protobuf::PhysicalSortExprNode { expr: Some(Box::new(serialize_physical_expr( - expr.expr.to_owned(), + &expr.expr, extension_codec, )?)), asc: !expr.options.descending, @@ -1761,10 +1757,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .filter() .as_ref() .map(|f| { - let expression = serialize_physical_expr( - f.expression().to_owned(), - extension_codec, - )?; + let expression = + serialize_physical_expr(f.expression(), extension_codec)?; let column_indices = f .column_indices() .iter() @@ -1806,13 +1800,13 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { let window_expr = exec .window_expr() .iter() - .map(|e| serialize_physical_window_expr(Arc::clone(e), extension_codec)) + .map(|e| serialize_physical_window_expr(e, extension_codec)) .collect::>>()?; let partition_keys = exec .partition_keys .iter() - .map(|e| serialize_physical_expr(Arc::clone(e), extension_codec)) + .map(|e| serialize_physical_expr(e, extension_codec)) .collect::>>()?; return Ok(protobuf::PhysicalPlanNode { @@ -1836,13 +1830,13 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { let window_expr = exec .window_expr() .iter() - .map(|e| serialize_physical_window_expr(Arc::clone(e), extension_codec)) + .map(|e| serialize_physical_window_expr(e, extension_codec)) .collect::>>()?; let partition_keys = exec .partition_keys .iter() - .map(|e| serialize_physical_expr(Arc::clone(e), extension_codec)) + .map(|e| serialize_physical_expr(e, extension_codec)) .collect::>>()?; let input_order_mode = match &exec.input_order_mode { @@ -1886,7 +1880,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { let expr: PhysicalSortExpr = requirement.to_owned().into(); let sort_expr = protobuf::PhysicalSortExprNode { expr: Some(Box::new(serialize_physical_expr( - expr.expr.to_owned(), + &expr.expr, extension_codec, )?)), asc: !expr.options.descending, @@ -2025,7 +2019,7 @@ pub trait PhysicalExtensionCodec: Debug + Send + Sync { fn try_encode_expr( &self, - _node: Arc, + _node: &Arc, _buf: &mut Vec, ) -> Result<()> { not_impl_err!("PhysicalExtensionCodec is not provided") diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 555ad22a9bc1..25be7de61cc3 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -52,7 +52,7 @@ pub fn serialize_physical_aggr_expr( aggr_expr: Arc, codec: &dyn PhysicalExtensionCodec, ) -> Result { - let expressions = serialize_physical_exprs(aggr_expr.expressions(), codec)?; + let expressions = serialize_physical_exprs(&aggr_expr.expressions(), codec)?; let ordering_req = aggr_expr.order_bys().unwrap_or(&[]).to_vec(); let ordering_req = serialize_physical_sort_exprs(ordering_req, codec)?; @@ -96,7 +96,7 @@ fn serialize_physical_window_aggr_expr( } pub fn serialize_physical_window_expr( - window_expr: Arc, + window_expr: &Arc, codec: &dyn PhysicalExtensionCodec, ) -> Result { let expr = window_expr.as_any(); @@ -187,9 +187,8 @@ pub fn serialize_physical_window_expr( return not_impl_err!("WindowExpr not supported: {window_expr:?}"); }; - let args = serialize_physical_exprs(args, codec)?; - let partition_by = - serialize_physical_exprs(window_expr.partition_by().to_vec(), codec)?; + let args = serialize_physical_exprs(&args, codec)?; + let partition_by = serialize_physical_exprs(window_expr.partition_by(), codec)?; let order_by = serialize_physical_sort_exprs(window_expr.order_by().to_vec(), codec)?; let window_frame: protobuf::WindowFrame = window_frame .as_ref() @@ -225,7 +224,7 @@ pub fn serialize_physical_sort_expr( codec: &dyn PhysicalExtensionCodec, ) -> Result { let PhysicalSortExpr { expr, options } = sort_expr; - let expr = serialize_physical_expr(expr, codec)?; + let expr = serialize_physical_expr(&expr, codec)?; Ok(PhysicalSortExprNode { expr: Some(Box::new(expr)), asc: !options.descending, @@ -233,12 +232,12 @@ pub fn serialize_physical_sort_expr( }) } -pub fn serialize_physical_exprs( +pub fn serialize_physical_exprs<'a, I>( values: I, codec: &dyn PhysicalExtensionCodec, ) -> Result> where - I: IntoIterator>, + I: IntoIterator>, { values .into_iter() @@ -251,7 +250,7 @@ where /// If required, a [`PhysicalExtensionCodec`] can be provided which can handle /// serialization of udfs requiring specialized serialization (see [`PhysicalExtensionCodec::try_encode_udf`]) pub fn serialize_physical_expr( - value: Arc, + value: &Arc, codec: &dyn PhysicalExtensionCodec, ) -> Result { let expr = value.as_any(); @@ -267,14 +266,8 @@ pub fn serialize_physical_expr( }) } else if let Some(expr) = expr.downcast_ref::() { let binary_expr = Box::new(protobuf::PhysicalBinaryExprNode { - l: Some(Box::new(serialize_physical_expr( - Arc::clone(expr.left()), - codec, - )?)), - r: Some(Box::new(serialize_physical_expr( - Arc::clone(expr.right()), - codec, - )?)), + l: Some(Box::new(serialize_physical_expr(expr.left(), codec)?)), + r: Some(Box::new(serialize_physical_expr(expr.right(), codec)?)), op: format!("{:?}", expr.op()), }); @@ -292,8 +285,7 @@ pub fn serialize_physical_expr( expr: expr .expr() .map(|exp| { - serialize_physical_expr(Arc::clone(exp), codec) - .map(Box::new) + serialize_physical_expr(exp, codec).map(Box::new) }) .transpose()?, when_then_expr: expr @@ -308,10 +300,7 @@ pub fn serialize_physical_expr( >>()?, else_expr: expr .else_expr() - .map(|a| { - serialize_physical_expr(Arc::clone(a), codec) - .map(Box::new) - }) + .map(|a| serialize_physical_expr(a, codec).map(Box::new)) .transpose()?, }, ), @@ -322,10 +311,7 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::NotExpr(Box::new( protobuf::PhysicalNot { - expr: Some(Box::new(serialize_physical_expr( - expr.arg().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(expr.arg(), codec)?)), }, ))), }) @@ -333,10 +319,7 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::IsNullExpr( Box::new(protobuf::PhysicalIsNull { - expr: Some(Box::new(serialize_physical_expr( - expr.arg().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(expr.arg(), codec)?)), }), )), }) @@ -344,10 +327,7 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::IsNotNullExpr( Box::new(protobuf::PhysicalIsNotNull { - expr: Some(Box::new(serialize_physical_expr( - expr.arg().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(expr.arg(), codec)?)), }), )), }) @@ -355,11 +335,8 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::InList(Box::new( protobuf::PhysicalInListNode { - expr: Some(Box::new(serialize_physical_expr( - expr.expr().to_owned(), - codec, - )?)), - list: serialize_physical_exprs(expr.list().to_vec(), codec)?, + expr: Some(Box::new(serialize_physical_expr(expr.expr(), codec)?)), + list: serialize_physical_exprs(expr.list(), codec)?, negated: expr.negated(), }, ))), @@ -368,10 +345,7 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::Negative(Box::new( protobuf::PhysicalNegativeNode { - expr: Some(Box::new(serialize_physical_expr( - expr.arg().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(expr.arg(), codec)?)), }, ))), }) @@ -385,10 +359,7 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::Cast(Box::new( protobuf::PhysicalCastNode { - expr: Some(Box::new(serialize_physical_expr( - cast.expr().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(cast.expr(), codec)?)), arrow_type: Some(cast.cast_type().try_into()?), }, ))), @@ -397,10 +368,7 @@ pub fn serialize_physical_expr( Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::TryCast(Box::new( protobuf::PhysicalTryCastNode { - expr: Some(Box::new(serialize_physical_expr( - cast.expr().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(cast.expr(), codec)?)), arrow_type: Some(cast.cast_type().try_into()?), }, ))), @@ -412,7 +380,7 @@ pub fn serialize_physical_expr( expr_type: Some(protobuf::physical_expr_node::ExprType::ScalarUdf( protobuf::PhysicalScalarUdfNode { name: expr.name().to_string(), - args: serialize_physical_exprs(expr.args().to_vec(), codec)?, + args: serialize_physical_exprs(expr.args(), codec)?, fun_definition: (!buf.is_empty()).then_some(buf), return_type: Some(expr.return_type().try_into()?), }, @@ -424,12 +392,9 @@ pub fn serialize_physical_expr( protobuf::PhysicalLikeExprNode { negated: expr.negated(), case_insensitive: expr.case_insensitive(), - expr: Some(Box::new(serialize_physical_expr( - expr.expr().to_owned(), - codec, - )?)), + expr: Some(Box::new(serialize_physical_expr(expr.expr(), codec)?)), pattern: Some(Box::new(serialize_physical_expr( - expr.pattern().to_owned(), + expr.pattern(), codec, )?)), }, @@ -437,12 +402,12 @@ pub fn serialize_physical_expr( }) } else { let mut buf: Vec = vec![]; - match codec.try_encode_expr(Arc::clone(&value), &mut buf) { + match codec.try_encode_expr(value, &mut buf) { Ok(_) => { let inputs: Vec = value .children() .into_iter() - .map(|e| serialize_physical_expr(Arc::clone(e), codec)) + .map(|e| serialize_physical_expr(e, codec)) .collect::>()?; Ok(protobuf::PhysicalExprNode { expr_type: Some(protobuf::physical_expr_node::ExprType::Extension( @@ -468,7 +433,7 @@ pub fn serialize_partitioning( )), }, Partitioning::Hash(exprs, partition_count) => { - let serialized_exprs = serialize_physical_exprs(exprs.clone(), codec)?; + let serialized_exprs = serialize_physical_exprs(exprs, codec)?; protobuf::Partitioning { partition_method: Some(protobuf::partitioning::PartitionMethod::Hash( protobuf::PhysicalHashRepartition { @@ -493,8 +458,8 @@ fn serialize_when_then_expr( codec: &dyn PhysicalExtensionCodec, ) -> Result { Ok(protobuf::PhysicalWhenThen { - when_expr: Some(serialize_physical_expr(Arc::clone(when_expr), codec)?), - then_expr: Some(serialize_physical_expr(Arc::clone(then_expr), codec)?), + when_expr: Some(serialize_physical_expr(when_expr, codec)?), + then_expr: Some(serialize_physical_expr(then_expr, codec)?), }) } @@ -608,7 +573,7 @@ pub fn serialize_maybe_filter( match expr { None => Ok(protobuf::MaybeFilter { expr: None }), Some(expr) => Ok(protobuf::MaybeFilter { - expr: Some(serialize_physical_expr(expr, codec)?), + expr: Some(serialize_physical_expr(&expr, codec)?), }), } } diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 3e49dc24fd5a..b2ded88dfaf4 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -821,11 +821,10 @@ fn roundtrip_parquet_exec_with_custom_predicate_expr() -> Result<()> { fn try_encode_expr( &self, - node: Arc, + node: &Arc, buf: &mut Vec, ) -> Result<()> { if node - .as_ref() .as_any() .downcast_ref::() .is_some() From 650dfdc96a6e4e641befd520580cfdac22ff9902 Mon Sep 17 00:00:00 2001 From: Edmondo Porcu Date: Thu, 29 Aug 2024 12:13:36 -0400 Subject: [PATCH 29/75] Adding alias to custom udf (#12219) --- datafusion/proto/tests/cases/mod.rs | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/datafusion/proto/tests/cases/mod.rs b/datafusion/proto/tests/cases/mod.rs index 1f837b7f42e8..fbb2cd8f1e83 100644 --- a/datafusion/proto/tests/cases/mod.rs +++ b/datafusion/proto/tests/cases/mod.rs @@ -34,12 +34,17 @@ struct MyRegexUdf { signature: Signature, // regex as original string pattern: String, + aliases: Vec, } impl MyRegexUdf { fn new(pattern: String) -> Self { let signature = Signature::exact(vec![DataType::Utf8], Volatility::Immutable); - Self { signature, pattern } + Self { + signature, + pattern, + aliases: vec!["aggregate_udf_alias".to_string()], + } } } @@ -67,6 +72,9 @@ impl ScalarUDFImpl for MyRegexUdf { ) -> datafusion_common::Result { unimplemented!() } + fn aliases(&self) -> &[String] { + &self.aliases + } } #[derive(Clone, PartialEq, ::prost::Message)] From b691b35f6904942c3544ae2af80aef927dfc8d11 Mon Sep 17 00:00:00 2001 From: Austin Liu Date: Fri, 30 Aug 2024 09:25:07 +0800 Subject: [PATCH 30/75] Minor: fix link title (#12246) Signed-off-by: Austin Liu --- docs/source/user-guide/sql/scalar_functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 9569f4e65ff3..47e35d2e72e3 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -3262,7 +3262,7 @@ _Alias of [array_distance](#array_distance)._ ### `list_distinct` -_Alias of [array_dims](#array_distinct)._ +_Alias of [array_distinct](#array_distinct)._ ### `list_element` From f9b14693228cfefdbf7bbc7f8a41179c2c0bcd64 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 30 Aug 2024 04:02:29 +0200 Subject: [PATCH 31/75] Remove TableSource::supports_filter_pushdown function (#12239) It was deprecated since 20.0.0. --- datafusion/expr/src/table_source.rs | 18 +++------------ .../src/analyzer/inline_table_scan.rs | 22 +++++++++++-------- datafusion/optimizer/src/push_down_filter.rs | 10 +++++---- 3 files changed, 22 insertions(+), 28 deletions(-) diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index 8b8d2dfcf2df..bdb602d48dee 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -98,27 +98,15 @@ pub trait TableSource: Sync + Send { TableType::Base } - /// Tests whether the table provider can make use of a filter expression - /// to optimise data retrieval. - #[deprecated(since = "20.0.0", note = "use supports_filters_pushdown instead")] - fn supports_filter_pushdown( - &self, - _filter: &Expr, - ) -> Result { - Ok(TableProviderFilterPushDown::Unsupported) - } - /// Tests whether the table provider can make use of any or all filter expressions /// to optimise data retrieval. - #[allow(deprecated)] fn supports_filters_pushdown( &self, filters: &[&Expr], ) -> Result> { - filters - .iter() - .map(|f| self.supports_filter_pushdown(f)) - .collect() + Ok((0..filters.len()) + .map(|_| TableProviderFilterPushDown::Unsupported) + .collect()) } /// Get the Logical plan of this table provider, if available. diff --git a/datafusion/optimizer/src/analyzer/inline_table_scan.rs b/datafusion/optimizer/src/analyzer/inline_table_scan.rs index d5b3648725b9..2fc9b91da6cb 100644 --- a/datafusion/optimizer/src/analyzer/inline_table_scan.rs +++ b/datafusion/optimizer/src/analyzer/inline_table_scan.rs @@ -109,7 +109,7 @@ mod tests { use crate::test::assert_analyzed_plan_eq; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_expr::{col, lit, LogicalPlan, LogicalPlanBuilder, TableSource}; + use datafusion_expr::{col, lit, Expr, LogicalPlan, LogicalPlanBuilder, TableSource}; pub struct RawTableSource {} @@ -125,12 +125,14 @@ mod tests { ])) } - fn supports_filter_pushdown( + fn supports_filters_pushdown( &self, - _filter: &datafusion_expr::Expr, - ) -> datafusion_common::Result + filters: &[&Expr], + ) -> datafusion_common::Result> { - Ok(datafusion_expr::TableProviderFilterPushDown::Inexact) + Ok((0..filters.len()) + .map(|_| datafusion_expr::TableProviderFilterPushDown::Inexact) + .collect()) } } @@ -154,12 +156,14 @@ mod tests { self } - fn supports_filter_pushdown( + fn supports_filters_pushdown( &self, - _filter: &datafusion_expr::Expr, - ) -> datafusion_common::Result + filters: &[&Expr], + ) -> datafusion_common::Result> { - Ok(datafusion_expr::TableProviderFilterPushDown::Exact) + Ok((0..filters.len()) + .map(|_| datafusion_expr::TableProviderFilterPushDown::Exact) + .collect()) } fn schema(&self) -> arrow::datatypes::SchemaRef { diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 33a58a810b08..0625652a6be1 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -2416,11 +2416,13 @@ mod tests { TableType::Base } - fn supports_filter_pushdown( + fn supports_filters_pushdown( &self, - _e: &Expr, - ) -> Result { - Ok(self.filter_support.clone()) + filters: &[&Expr], + ) -> Result> { + Ok((0..filters.len()) + .map(|_| self.filter_support.clone()) + .collect()) } fn as_any(&self) -> &dyn std::any::Any { From 94d178ebe9674669b32ecd7896b5597f49e90791 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 30 Aug 2024 04:08:12 +0200 Subject: [PATCH 32/75] Remove Box from Sort (#12207) `expr::Sort` had `Box` because Sort was also an expression (via `expr::Expr::Sort`). This has been removed, obsoleting need to use a `Box`. --- datafusion/core/src/datasource/mod.rs | 2 +- datafusion/core/tests/dataframe/mod.rs | 20 +++++++++---------- .../tests/user_defined/user_defined_plan.rs | 2 +- datafusion/expr/src/expr.rs | 6 +++--- datafusion/expr/src/expr_rewriter/mod.rs | 4 ++-- datafusion/expr/src/expr_rewriter/order_by.rs | 2 +- datafusion/expr/src/logical_plan/builder.rs | 8 ++++---- datafusion/expr/src/logical_plan/plan.rs | 2 +- datafusion/expr/src/logical_plan/tree_node.rs | 2 +- datafusion/expr/src/tree_node.rs | 8 ++++---- datafusion/expr/src/utils.rs | 16 +++++++-------- .../src/analyzer/count_wildcard_rule.rs | 2 +- .../optimizer/src/common_subexpr_eliminate.rs | 3 +-- .../proto/src/logical_plan/from_proto.rs | 7 +------ datafusion/proto/src/logical_plan/to_proto.rs | 2 +- datafusion/sql/src/expr/function.rs | 2 +- datafusion/sql/src/expr/order_by.rs | 2 +- datafusion/sql/src/unparser/expr.rs | 2 +- datafusion/sql/src/unparser/rewrite.rs | 2 +- .../substrait/src/logical_plan/consumer.rs | 2 +- .../substrait/src/logical_plan/producer.rs | 9 +-------- 21 files changed, 46 insertions(+), 59 deletions(-) diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 55e88e572be1..529bb799e23d 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -63,7 +63,7 @@ fn create_ordering( // Construct PhysicalSortExpr objects from Expr objects: let mut sort_exprs = vec![]; for sort in exprs { - match sort.expr.as_ref() { + match &sort.expr { Expr::Column(col) => match expressions::col(&col.name, schema) { Ok(expr) => { sort_exprs.push(PhysicalSortExpr { diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index c5b9db7588e9..19ce9294cfad 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -184,7 +184,7 @@ async fn test_count_wildcard_on_window() -> Result<()> { WindowFunctionDefinition::AggregateUDF(count_udaf()), vec![wildcard()], )) - .order_by(vec![Sort::new(Box::new(col("a")), false, true)]) + .order_by(vec![Sort::new(col("a"), false, true)]) .window_frame(WindowFrame::new_bounds( WindowFrameUnits::Range, WindowFrameBound::Preceding(ScalarValue::UInt32(Some(6))), @@ -352,7 +352,7 @@ async fn sort_on_unprojected_columns() -> Result<()> { .unwrap() .select(vec![col("a")]) .unwrap() - .sort(vec![Sort::new(Box::new(col("b")), false, true)]) + .sort(vec![Sort::new(col("b"), false, true)]) .unwrap(); let results = df.collect().await.unwrap(); @@ -396,7 +396,7 @@ async fn sort_on_distinct_columns() -> Result<()> { .unwrap() .distinct() .unwrap() - .sort(vec![Sort::new(Box::new(col("a")), false, true)]) + .sort(vec![Sort::new(col("a"), false, true)]) .unwrap(); let results = df.collect().await.unwrap(); @@ -435,7 +435,7 @@ async fn sort_on_distinct_unprojected_columns() -> Result<()> { .await? .select(vec![col("a")])? .distinct()? - .sort(vec![Sort::new(Box::new(col("b")), false, true)]) + .sort(vec![Sort::new(col("b"), false, true)]) .unwrap_err(); assert_eq!(err.strip_backtrace(), "Error during planning: For SELECT DISTINCT, ORDER BY expressions b must appear in select list"); Ok(()) @@ -599,8 +599,8 @@ async fn test_grouping_sets() -> Result<()> { .await? .aggregate(vec![grouping_set_expr], vec![count(col("a"))])? .sort(vec![ - Sort::new(Box::new(col("a")), false, true), - Sort::new(Box::new(col("b")), false, true), + Sort::new(col("a"), false, true), + Sort::new(col("b"), false, true), ])?; let results = df.collect().await?; @@ -640,8 +640,8 @@ async fn test_grouping_sets_count() -> Result<()> { .await? .aggregate(vec![grouping_set_expr], vec![count(lit(1))])? .sort(vec![ - Sort::new(Box::new(col("c1")), false, true), - Sort::new(Box::new(col("c2")), false, true), + Sort::new(col("c1"), false, true), + Sort::new(col("c2"), false, true), ])?; let results = df.collect().await?; @@ -687,8 +687,8 @@ async fn test_grouping_set_array_agg_with_overflow() -> Result<()> { ], )? .sort(vec![ - Sort::new(Box::new(col("c1")), false, true), - Sort::new(Box::new(col("c2")), false, true), + Sort::new(col("c1"), false, true), + Sort::new(col("c2"), false, true), ])?; let results = df.collect().await?; diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index da27cf8869d1..56edeab443c7 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -419,7 +419,7 @@ impl UserDefinedLogicalNodeCore for TopKPlanNode { } fn expressions(&self) -> Vec { - vec![self.expr.expr.as_ref().clone()] + vec![self.expr.expr.clone()] } /// For example: `TopK: k=10` diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index b81c02ccd0b7..8914214d084f 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -602,7 +602,7 @@ impl TryCast { #[derive(Clone, PartialEq, Eq, Hash, Debug)] pub struct Sort { /// The expression to sort on - pub expr: Box, + pub expr: Expr, /// The direction of the sort pub asc: bool, /// Whether to put Nulls before all other data values @@ -611,7 +611,7 @@ pub struct Sort { impl Sort { /// Create a new Sort expression - pub fn new(expr: Box, asc: bool, nulls_first: bool) -> Self { + pub fn new(expr: Expr, asc: bool, nulls_first: bool) -> Self { Self { expr, asc, @@ -1368,7 +1368,7 @@ impl Expr { /// let sort_expr = col("foo").sort(true, true); // SORT ASC NULLS_FIRST /// ``` pub fn sort(self, asc: bool, nulls_first: bool) -> Sort { - Sort::new(Box::new(self), asc, nulls_first) + Sort::new(self, asc, nulls_first) } /// Return `IsTrue(Box(self))` diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index 5e7fedb4cbd8..61b0f6d9bb2b 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -125,8 +125,8 @@ pub fn normalize_sorts( .into_iter() .map(|e| { let sort = e.into(); - normalize_col(*sort.expr, plan) - .map(|expr| Sort::new(Box::new(expr), sort.asc, sort.nulls_first)) + normalize_col(sort.expr, plan) + .map(|expr| Sort::new(expr, sort.asc, sort.nulls_first)) }) .collect() } diff --git a/datafusion/expr/src/expr_rewriter/order_by.rs b/datafusion/expr/src/expr_rewriter/order_by.rs index af5b8c4f9177..48d380cd59e2 100644 --- a/datafusion/expr/src/expr_rewriter/order_by.rs +++ b/datafusion/expr/src/expr_rewriter/order_by.rs @@ -35,7 +35,7 @@ pub fn rewrite_sort_cols_by_aggs( .map(|e| { let sort = e.into(); Ok(Sort::new( - Box::new(rewrite_sort_col_by_aggs(*sort.expr, plan)?), + rewrite_sort_col_by_aggs(sort.expr, plan)?, sort.asc, sort.nulls_first, )) diff --git a/datafusion/expr/src/logical_plan/builder.rs b/datafusion/expr/src/logical_plan/builder.rs index f5770167861b..fc961b83f7b5 100644 --- a/datafusion/expr/src/logical_plan/builder.rs +++ b/datafusion/expr/src/logical_plan/builder.rs @@ -1720,8 +1720,8 @@ mod tests { let plan = table_scan(Some("employee_csv"), &employee_schema(), Some(vec![3, 4]))? .sort(vec![ - expr::Sort::new(Box::new(col("state")), true, true), - expr::Sort::new(Box::new(col("salary")), false, false), + expr::Sort::new(col("state"), true, true), + expr::Sort::new(col("salary"), false, false), ])? .build()?; @@ -2147,8 +2147,8 @@ mod tests { let plan = table_scan(Some("employee_csv"), &employee_schema(), Some(vec![3, 4]))? .sort(vec![ - expr::Sort::new(Box::new(col("state")), true, true), - expr::Sort::new(Box::new(col("salary")), false, false), + expr::Sort::new(col("state"), true, true), + expr::Sort::new(col("salary"), false, false), ])? .build()?; diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 8e6ec762f549..5bd6ab10331a 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -2616,7 +2616,7 @@ impl DistinctOn { // Check that the left-most sort expressions are the same as the `ON` expressions. let mut matched = true; for (on, sort) in self.on_expr.iter().zip(sort_expr.iter()) { - if on != &*sort.expr { + if on != &sort.expr { matched = false; break; } diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index 29a99a8e8886..0964fb601879 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -509,7 +509,7 @@ impl LogicalPlan { })) => on_expr .iter() .chain(select_expr.iter()) - .chain(sort_expr.iter().flatten().map(|sort| &*sort.expr)) + .chain(sort_expr.iter().flatten().map(|sort| &sort.expr)) .apply_until_stop(f), // plans without expressions LogicalPlan::EmptyRelation(_) diff --git a/datafusion/expr/src/tree_node.rs b/datafusion/expr/src/tree_node.rs index 90d61bf63763..c7c498dd3f01 100644 --- a/datafusion/expr/src/tree_node.rs +++ b/datafusion/expr/src/tree_node.rs @@ -97,7 +97,7 @@ impl TreeNode for Expr { expr_vec.push(f.as_ref()); } if let Some(order_by) = order_by { - expr_vec.extend(order_by.iter().map(|sort| sort.expr.as_ref())); + expr_vec.extend(order_by.iter().map(|sort| &sort.expr)); } expr_vec } @@ -109,7 +109,7 @@ impl TreeNode for Expr { }) => { let mut expr_vec = args.iter().collect::>(); expr_vec.extend(partition_by); - expr_vec.extend(order_by.iter().map(|sort| sort.expr.as_ref())); + expr_vec.extend(order_by.iter().map(|sort| &sort.expr)); expr_vec } Expr::InList(InList { expr, list, .. }) => { @@ -395,7 +395,7 @@ pub fn transform_sort_vec Result>>( ) -> Result>> { Ok(sorts .iter() - .map(|sort| (*sort.expr).clone()) + .map(|sort| sort.expr.clone()) .map_until_stop_and_collect(&mut f)? .update_data(|transformed_exprs| { replace_sort_expressions(sorts, transformed_exprs) @@ -413,7 +413,7 @@ pub fn replace_sort_expressions(sorts: Vec, new_expr: Vec) -> Vec Sort { Sort { - expr: Box::new(new_expr), + expr: new_expr, ..sort } } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index b6b1b5660a81..c4c6b076e5ba 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -1401,9 +1401,9 @@ mod tests { #[test] fn test_group_window_expr_by_sort_keys() -> Result<()> { - let age_asc = expr::Sort::new(Box::new(col("age")), true, true); - let name_desc = expr::Sort::new(Box::new(col("name")), false, true); - let created_at_desc = expr::Sort::new(Box::new(col("created_at")), false, true); + let age_asc = expr::Sort::new(col("age"), true, true); + let name_desc = expr::Sort::new(col("name"), false, true); + let created_at_desc = expr::Sort::new(col("created_at"), false, true); let max1 = Expr::WindowFunction(expr::WindowFunction::new( WindowFunctionDefinition::AggregateUDF(max_udaf()), vec![col("name")], @@ -1463,12 +1463,12 @@ mod tests { for nulls_first_ in nulls_first_or_last { let order_by = &[ Sort { - expr: Box::new(col("age")), + expr: col("age"), asc: asc_, nulls_first: nulls_first_, }, Sort { - expr: Box::new(col("name")), + expr: col("name"), asc: asc_, nulls_first: nulls_first_, }, @@ -1477,7 +1477,7 @@ mod tests { let expected = vec![ ( Sort { - expr: Box::new(col("age")), + expr: col("age"), asc: asc_, nulls_first: nulls_first_, }, @@ -1485,7 +1485,7 @@ mod tests { ), ( Sort { - expr: Box::new(col("name")), + expr: col("name"), asc: asc_, nulls_first: nulls_first_, }, @@ -1493,7 +1493,7 @@ mod tests { ), ( Sort { - expr: Box::new(col("created_at")), + expr: col("created_at"), asc: true, nulls_first: false, }, diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 35d4f91e3b6f..0036f6df43f6 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -229,7 +229,7 @@ mod tests { WindowFunctionDefinition::AggregateUDF(count_udaf()), vec![wildcard()], )) - .order_by(vec![Sort::new(Box::new(col("a")), false, true)]) + .order_by(vec![Sort::new(col("a"), false, true)]) .window_frame(WindowFrame::new_bounds( WindowFrameUnits::Range, WindowFrameBound::Preceding(ScalarValue::UInt32(Some(6))), diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 25bef7e2d0e4..22e9d220d324 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -328,8 +328,7 @@ impl CommonSubexprEliminate { ) -> Result> { let Sort { expr, input, fetch } = sort; let input = Arc::unwrap_or_clone(input); - let sort_expressions = - expr.iter().map(|sort| sort.expr.as_ref().clone()).collect(); + let sort_expressions = expr.iter().map(|sort| sort.expr.clone()).collect(); let new_sort = self .try_unary_plan(sort_expressions, input, config)? .update_data(|(new_expr, new_input)| { diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 3ba1cb945e9c..893255ccc8ce 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -645,12 +645,7 @@ pub fn parse_sort( codec: &dyn LogicalExtensionCodec, ) -> Result { Ok(Sort::new( - Box::new(parse_required_expr( - sort.expr.as_ref(), - registry, - "expr", - codec, - )?), + parse_required_expr(sort.expr.as_ref(), registry, "expr", codec)?, sort.asc, sort.nulls_first, )) diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index b937c03f79d9..63d1a007c1e5 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -637,7 +637,7 @@ where nulls_first, } = sort; Ok(protobuf::SortExprNode { - expr: Some(serialize_expr(expr.as_ref(), codec)?), + expr: Some(serialize_expr(expr, codec)?), asc: *asc, nulls_first: *nulls_first, }) diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 9c768eb73c2e..190a7e918928 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -282,7 +282,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let func_deps = schema.functional_dependencies(); // Find whether ties are possible in the given ordering let is_ordering_strict = order_by.iter().find_map(|orderby_expr| { - if let Expr::Column(col) = orderby_expr.expr.as_ref() { + if let Expr::Column(col) = &orderby_expr.expr { let idx = schema.index_of_column(col).ok()?; return if func_deps.iter().any(|dep| { dep.source_indices == vec![idx] && dep.mode == Dependency::Single diff --git a/datafusion/sql/src/expr/order_by.rs b/datafusion/sql/src/expr/order_by.rs index cdaa787cedd0..6a3a4d6ccbb7 100644 --- a/datafusion/sql/src/expr/order_by.rs +++ b/datafusion/sql/src/expr/order_by.rs @@ -100,7 +100,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { }; let asc = asc.unwrap_or(true); expr_vec.push(Sort::new( - Box::new(expr), + expr, asc, // when asc is true, by default nulls last to be consistent with postgres // postgres rule: https://www.postgresql.org/docs/current/queries-order.html diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 9a3f139fdee8..549635a31aef 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -1761,7 +1761,7 @@ mod tests { fun: WindowFunctionDefinition::AggregateUDF(count_udaf()), args: vec![wildcard()], partition_by: vec![], - order_by: vec![Sort::new(Box::new(col("a")), false, true)], + order_by: vec![Sort::new(col("a"), false, true)], window_frame: WindowFrame::new_bounds( datafusion_expr::WindowFrameUnits::Range, datafusion_expr::WindowFrameBound::Preceding( diff --git a/datafusion/sql/src/unparser/rewrite.rs b/datafusion/sql/src/unparser/rewrite.rs index 522a08af8546..2529385849e0 100644 --- a/datafusion/sql/src/unparser/rewrite.rs +++ b/datafusion/sql/src/unparser/rewrite.rs @@ -158,7 +158,7 @@ pub(super) fn rewrite_plan_for_sort_on_non_projected_fields( let mut collects = p.expr.clone(); for sort in &sort.expr { - collects.push(sort.expr.as_ref().clone()); + collects.push(sort.expr.clone()); } // Compare outer collects Expr::to_string with inner collected transformed values diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 05903bb56cfe..21bef3c2c98e 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -936,7 +936,7 @@ pub async fn from_substrait_sorts( }; let (asc, nulls_first) = asc_nullfirst.unwrap(); sorts.push(Sort { - expr: Box::new(expr), + expr, asc, nulls_first, }); diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 592390a285ba..e71cf04cd341 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -16,7 +16,6 @@ // under the License. use itertools::Itertools; -use std::ops::Deref; use std::sync::Arc; use arrow_buffer::ToByteSlice; @@ -819,13 +818,7 @@ fn to_substrait_sort_field( (false, false) => SortDirection::DescNullsLast, }; Ok(SortField { - expr: Some(to_substrait_rex( - ctx, - sort.expr.deref(), - schema, - 0, - extensions, - )?), + expr: Some(to_substrait_rex(ctx, &sort.expr, schema, 0, extensions)?), sort_kind: Some(SortKind::Direction(sort_kind.into())), }) } From e60318553d6ac36b2d07466acebef861fde2936e Mon Sep 17 00:00:00 2001 From: Jonah Gao Date: Fri, 30 Aug 2024 14:08:29 +0800 Subject: [PATCH 33/75] Reuse `NamePreserver` in `SimplifyExpressions` (#12238) --- datafusion/expr/src/expr_rewriter/mod.rs | 2 ++ .../src/simplify_expressions/simplify_exprs.rs | 18 +++++++----------- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/datafusion/expr/src/expr_rewriter/mod.rs b/datafusion/expr/src/expr_rewriter/mod.rs index 61b0f6d9bb2b..35c573836419 100644 --- a/datafusion/expr/src/expr_rewriter/mod.rs +++ b/datafusion/expr/src/expr_rewriter/mod.rs @@ -311,6 +311,8 @@ impl NamePreserver { /// Create a new NamePreserver for rewriting the `expr` that is part of the specified plan pub fn new(plan: &LogicalPlan) -> Self { Self { + // The schema of Filter and Join nodes comes from their inputs rather than their output expressions, + // so there is no need to use aliases to preserve expression names. use_alias: !matches!(plan, LogicalPlan::Filter(_) | LogicalPlan::Join(_)), } } diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index e13ce382a3e0..1c39313ea132 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -27,6 +27,7 @@ use datafusion_expr::simplify::SimplifyContext; use datafusion_expr::utils::merge_schema; use crate::optimizer::ApplyOrder; +use crate::utils::NamePreserver; use crate::{OptimizerConfig, OptimizerRule}; use super::ExprSimplifier; @@ -119,18 +120,13 @@ impl SimplifyExpressions { simplifier }; - // the output schema of a filter or join is the input schema. Thus they - // can't handle aliased expressions - let use_alias = !matches!(plan, LogicalPlan::Filter(_) | LogicalPlan::Join(_)); + // Preserve expression names to avoid changing the schema of the plan. + let name_preserver = NamePreserver::new(&plan); plan.map_expressions(|e| { - let new_e = if use_alias { - // TODO: unify with `rewrite_preserving_name` - let original_name = e.name_for_alias()?; - simplifier.simplify(e)?.alias_if_changed(original_name) - } else { - simplifier.simplify(e) - }?; - + let original_name = name_preserver.save(&e)?; + let new_e = simplifier + .simplify(e) + .and_then(|expr| original_name.restore(expr))?; // TODO it would be nice to have a way to know if the expression was simplified // or not. For now conservatively return Transformed::yes Ok(Transformed::yes(new_e)) From 88dd30504cbe9b9e4059f592d8688029c6d35b22 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 30 Aug 2024 23:19:55 +0200 Subject: [PATCH 34/75] Remove redundant argument and improve error message (#12217) * Remove redundant argument and improve error message * Update datafusion/functions/src/string/common.rs Co-authored-by: Oleks V * Update datafusion/functions/src/string/common.rs Co-authored-by: Oleks V --------- Co-authored-by: Oleks V --- datafusion/functions/src/string/common.rs | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/datafusion/functions/src/string/common.rs b/datafusion/functions/src/string/common.rs index a5dc22b4d9e4..9738cb812f90 100644 --- a/datafusion/functions/src/string/common.rs +++ b/datafusion/functions/src/string/common.rs @@ -73,15 +73,14 @@ pub(crate) fn general_trim( }; if use_string_view { - string_view_trim::(trim_type, func, args) + string_view_trim::(func, args) } else { - string_trim::(trim_type, func, args) + string_trim::(func, args) } } // removing 'a will cause compiler complaining lifetime of `func` fn string_view_trim<'a, T: OffsetSizeTrait>( - trim_type: TrimType, func: fn(&'a str, &'a str) -> &'a str, args: &'a [ArrayRef], ) -> Result { @@ -129,14 +128,13 @@ fn string_view_trim<'a, T: OffsetSizeTrait>( } other => { exec_err!( - "{trim_type} was called with {other} arguments. It requires at least 1 and at most 2." + "Function TRIM was called with {other} arguments. It requires at least 1 and at most 2." ) } } } fn string_trim<'a, T: OffsetSizeTrait>( - trim_type: TrimType, func: fn(&'a str, &'a str) -> &'a str, args: &'a [ArrayRef], ) -> Result { @@ -183,7 +181,7 @@ fn string_trim<'a, T: OffsetSizeTrait>( } other => { exec_err!( - "{trim_type} was called with {other} arguments. It requires at least 1 and at most 2." + "Function TRIM was called with {other} arguments. It requires at least 1 and at most 2." ) } } From 442d9bfcc5f8706a15e92f16a6fbc5c108c97a78 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Sat, 31 Aug 2024 23:18:54 +0200 Subject: [PATCH 35/75] Remove deprecated from_plan function (#12265) * Remove deprecated from_plan function Deprecated since 31.0.0. * empty --- datafusion/expr/src/utils.rs | 31 ------------------------------- 1 file changed, 31 deletions(-) diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index c4c6b076e5ba..9b64bca8121e 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -679,37 +679,6 @@ where err } -/// Returns a new logical plan based on the original one with inputs -/// and expressions replaced. -/// -/// The exprs correspond to the same order of expressions returned by -/// `LogicalPlan::expressions`. This function is used in optimizers in -/// the following way: -/// -/// ```text -/// let new_inputs = optimize_children(..., plan, props); -/// -/// // get the plans expressions to optimize -/// let exprs = plan.expressions(); -/// -/// // potentially rewrite plan expressions -/// let rewritten_exprs = rewrite_exprs(exprs); -/// -/// // create new plan using rewritten_exprs in same position -/// let new_plan = from_plan(&plan, rewritten_exprs, new_inputs); -/// ``` -/// -/// Notice: sometimes [from_plan] will use schema of original plan, it don't change schema! -/// Such as `Projection/Aggregate/Window` -#[deprecated(since = "31.0.0", note = "use LogicalPlan::with_new_exprs instead")] -pub fn from_plan( - plan: &LogicalPlan, - expr: &[Expr], - inputs: &[LogicalPlan], -) -> Result { - plan.with_new_exprs(expr.to_vec(), inputs.to_vec()) -} - /// Create field meta-data from an expression, for use in a result set schema pub fn exprlist_to_fields<'a>( exprs: impl IntoIterator, From 016ed03cf18c3f60594cf6f8fedd202561c27a91 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Sun, 1 Sep 2024 15:58:53 +0800 Subject: [PATCH 36/75] Remove redundant result of AggregateFunctionExpr::field (#12258) --- datafusion/physical-expr/src/aggregate.rs | 12 +++--------- datafusion/physical-expr/src/window/aggregate.rs | 4 ++-- .../physical-expr/src/window/sliding_aggregate.rs | 4 ++-- .../physical-optimizer/src/aggregate_statistics.rs | 4 ++-- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- datafusion/proto/src/physical_plan/mod.rs | 7 ++----- 6 files changed, 12 insertions(+), 21 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 5c1216f2a386..d62dc27ece86 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -253,12 +253,8 @@ impl AggregateFunctionExpr { } /// the field of the final result of this aggregation. - pub fn field(&self) -> Result { - Ok(Field::new( - &self.name, - self.data_type.clone(), - self.is_nullable, - )) + pub fn field(&self) -> Field { + Field::new(&self.name, self.data_type.clone(), self.is_nullable) } /// the accumulator used to accumulate values from the expressions. @@ -523,9 +519,7 @@ impl AggregateFunctionExpr { /// /// Note: this is used to use special aggregate implementations in certain conditions pub fn get_minmax_desc(&self) -> Option<(Field, bool)> { - self.fun - .is_descending() - .and_then(|flag| self.field().ok().map(|f| (f, flag))) + self.fun.is_descending().map(|flag| (self.field(), flag)) } /// Returns default value of the function given the input is Null diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 5439e140502a..1cc08a4e99aa 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -79,7 +79,7 @@ impl WindowExpr for PlainAggregateWindowExpr { } fn field(&self) -> Result { - self.aggregate.field() + Ok(self.aggregate.field()) } fn name(&self) -> &str { @@ -177,7 +177,7 @@ impl AggregateWindowExpr for PlainAggregateWindowExpr { ) -> Result { if cur_range.start == cur_range.end { self.aggregate - .default_value(self.aggregate.field()?.data_type()) + .default_value(self.aggregate.field().data_type()) } else { // Accumulate any new rows that have entered the window: let update_bound = cur_range.end - last_range.end; diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index ac3a4f4c09ec..b3848e15ee42 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -81,7 +81,7 @@ impl WindowExpr for SlidingAggregateWindowExpr { } fn field(&self) -> Result { - self.aggregate.field() + Ok(self.aggregate.field()) } fn name(&self) -> &str { @@ -183,7 +183,7 @@ impl AggregateWindowExpr for SlidingAggregateWindowExpr { ) -> Result { if cur_range.start == cur_range.end { self.aggregate - .default_value(self.aggregate.field()?.data_type()) + .default_value(self.aggregate.field().data_type()) } else { // Accumulate any new rows that have entered the window: let update_bound = cur_range.end - last_range.end; diff --git a/datafusion/physical-optimizer/src/aggregate_statistics.rs b/datafusion/physical-optimizer/src/aggregate_statistics.rs index 2b8725b5bac7..863c5ab2d288 100644 --- a/datafusion/physical-optimizer/src/aggregate_statistics.rs +++ b/datafusion/physical-optimizer/src/aggregate_statistics.rs @@ -183,7 +183,7 @@ fn take_optimizable_min( // MIN/MAX with 0 rows is always null if is_min(agg_expr) { if let Ok(min_data_type) = - ScalarValue::try_from(agg_expr.field().unwrap().data_type()) + ScalarValue::try_from(agg_expr.field().data_type()) { return Some((min_data_type, agg_expr.name().to_string())); } @@ -229,7 +229,7 @@ fn take_optimizable_max( // MIN/MAX with 0 rows is always null if is_max(agg_expr) { if let Ok(max_data_type) = - ScalarValue::try_from(agg_expr.field().unwrap().data_type()) + ScalarValue::try_from(agg_expr.field().data_type()) { return Some((max_data_type, agg_expr.name().to_string())); } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 0f33a9d7b992..2f974f1ef4e0 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -821,7 +821,7 @@ fn create_schema( | AggregateMode::SinglePartitioned => { // in final mode, the field with the final result of the accumulator for expr in aggr_expr { - fields.push(expr.field()?) + fields.push(expr.field()) } } } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 04cbf8b537b3..e622af745062 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1444,11 +1444,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { let agg_names = exec .aggr_expr() .iter() - .map(|expr| match expr.field() { - Ok(field) => Ok(field.name().clone()), - Err(e) => Err(e), - }) - .collect::>()?; + .map(|expr| expr.name().to_string()) + .collect::>(); let agg_mode = match exec.mode() { AggregateMode::Partial => protobuf::AggregateMode::Partial, From 8746e073b7a7ffcf86fac19d5ea1984ee6970d20 Mon Sep 17 00:00:00 2001 From: Dharan Aditya Date: Sun, 1 Sep 2024 14:10:10 +0530 Subject: [PATCH 37/75] Support `map_keys` & `map_values` for MAP type (#12194) * impl map_keys * rename field name * add logic tests * one more * owned to clone * more tests * typo * impl * add logic tests * chore * add docs * trying to make prettier happy * Update scalar_functions.md Co-authored-by: Alex Huang * reface signature * format docs * Update map_values.rs Co-authored-by: Alex Huang --------- Co-authored-by: Alex Huang --- datafusion/common/src/utils/mod.rs | 17 +-- datafusion/functions-nested/src/lib.rs | 6 ++ .../functions-nested/src/map_extract.rs | 3 +- datafusion/functions-nested/src/map_keys.rs | 102 ++++++++++++++++++ datafusion/functions-nested/src/map_values.rs | 102 ++++++++++++++++++ datafusion/functions-nested/src/utils.rs | 19 +++- datafusion/sqllogictest/test_files/map.slt | 97 ++++++++++++++++- .../source/user-guide/sql/scalar_functions.md | 52 +++++++++ 8 files changed, 377 insertions(+), 21 deletions(-) create mode 100644 datafusion/functions-nested/src/map_keys.rs create mode 100644 datafusion/functions-nested/src/map_values.rs diff --git a/datafusion/common/src/utils/mod.rs b/datafusion/common/src/utils/mod.rs index 839f890bf077..418ea380bc2c 100644 --- a/datafusion/common/src/utils/mod.rs +++ b/datafusion/common/src/utils/mod.rs @@ -35,7 +35,7 @@ use arrow_array::{ Array, FixedSizeListArray, LargeListArray, ListArray, OffsetSizeTrait, RecordBatchOptions, }; -use arrow_schema::{DataType, Fields}; +use arrow_schema::DataType; use sqlparser::ast::Ident; use sqlparser::dialect::GenericDialect; use sqlparser::parser::Parser; @@ -754,21 +754,6 @@ pub fn combine_limit( (combined_skip, combined_fetch) } -pub fn get_map_entry_field(data_type: &DataType) -> Result<&Fields> { - match data_type { - DataType::Map(field, _) => { - let field_data_type = field.data_type(); - match field_data_type { - DataType::Struct(fields) => Ok(fields), - _ => { - _internal_err!("Expected a Struct type, got {:?}", field_data_type) - } - } - } - _ => _internal_err!("Expected a Map type, got {:?}", data_type), - } -} - #[cfg(test)] mod tests { use crate::ScalarValue::Null; diff --git a/datafusion/functions-nested/src/lib.rs b/datafusion/functions-nested/src/lib.rs index 863b5a876adc..b548cf6db8b1 100644 --- a/datafusion/functions-nested/src/lib.rs +++ b/datafusion/functions-nested/src/lib.rs @@ -44,6 +44,8 @@ pub mod length; pub mod make_array; pub mod map; pub mod map_extract; +pub mod map_keys; +pub mod map_values; pub mod planner; pub mod position; pub mod range; @@ -85,6 +87,8 @@ pub mod expr_fn { pub use super::length::array_length; pub use super::make_array::make_array; pub use super::map_extract::map_extract; + pub use super::map_keys::map_keys; + pub use super::map_values::map_values; pub use super::position::array_position; pub use super::position::array_positions; pub use super::range::gen_series; @@ -149,6 +153,8 @@ pub fn all_default_nested_functions() -> Vec> { replace::array_replace_udf(), map::map_udf(), map_extract::map_extract_udf(), + map_keys::map_keys_udf(), + map_values::map_values_udf(), ] } diff --git a/datafusion/functions-nested/src/map_extract.rs b/datafusion/functions-nested/src/map_extract.rs index 82f0d8d6c15e..9f0c4ad29c60 100644 --- a/datafusion/functions-nested/src/map_extract.rs +++ b/datafusion/functions-nested/src/map_extract.rs @@ -24,7 +24,6 @@ use arrow::datatypes::DataType; use arrow_array::{Array, MapArray}; use arrow_buffer::OffsetBuffer; use arrow_schema::Field; -use datafusion_common::utils::get_map_entry_field; use datafusion_common::{cast::as_map_array, exec_err, Result}; use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; @@ -32,7 +31,7 @@ use std::any::Any; use std::sync::Arc; use std::vec; -use crate::utils::make_scalar_function; +use crate::utils::{get_map_entry_field, make_scalar_function}; // Create static instances of ScalarUDFs for each function make_udf_expr_and_func!( diff --git a/datafusion/functions-nested/src/map_keys.rs b/datafusion/functions-nested/src/map_keys.rs new file mode 100644 index 000000000000..0b1cebb27c86 --- /dev/null +++ b/datafusion/functions-nested/src/map_keys.rs @@ -0,0 +1,102 @@ +// 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. + +//! [`ScalarUDFImpl`] definitions for map_keys function. + +use crate::utils::{get_map_entry_field, make_scalar_function}; +use arrow_array::{Array, ArrayRef, ListArray}; +use arrow_schema::{DataType, Field}; +use datafusion_common::{cast::as_map_array, exec_err, Result}; +use datafusion_expr::{ + ArrayFunctionSignature, ColumnarValue, ScalarUDFImpl, Signature, TypeSignature, + Volatility, +}; +use std::any::Any; +use std::sync::Arc; + +make_udf_expr_and_func!( + MapKeysFunc, + map_keys, + map, + "Return a list of all keys in the map.", + map_keys_udf +); + +#[derive(Debug)] +pub(crate) struct MapKeysFunc { + signature: Signature, +} + +impl MapKeysFunc { + pub fn new() -> Self { + Self { + signature: Signature::new( + TypeSignature::ArraySignature(ArrayFunctionSignature::MapArray), + Volatility::Immutable, + ), + } + } +} + +impl ScalarUDFImpl for MapKeysFunc { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "map_keys" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> datafusion_common::Result { + if arg_types.len() != 1 { + return exec_err!("map_keys expects single argument"); + } + let map_type = &arg_types[0]; + let map_fields = get_map_entry_field(map_type)?; + Ok(DataType::List(Arc::new(Field::new( + "item", + map_fields.first().unwrap().data_type().clone(), + false, + )))) + } + + fn invoke(&self, args: &[ColumnarValue]) -> datafusion_common::Result { + make_scalar_function(map_keys_inner)(args) + } +} + +fn map_keys_inner(args: &[ArrayRef]) -> Result { + if args.len() != 1 { + return exec_err!("map_keys expects single argument"); + } + + let map_array = match args[0].data_type() { + DataType::Map(_, _) => as_map_array(&args[0])?, + _ => return exec_err!("Argument for map_keys should be a map"), + }; + + Ok(Arc::new(ListArray::new( + Arc::new(Field::new("item", map_array.key_type().clone(), false)), + map_array.offsets().clone(), + Arc::clone(map_array.keys()), + None, + ))) +} diff --git a/datafusion/functions-nested/src/map_values.rs b/datafusion/functions-nested/src/map_values.rs new file mode 100644 index 000000000000..58c0d74eed5f --- /dev/null +++ b/datafusion/functions-nested/src/map_values.rs @@ -0,0 +1,102 @@ +// 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. + +//! [`ScalarUDFImpl`] definitions for map_values function. + +use crate::utils::{get_map_entry_field, make_scalar_function}; +use arrow_array::{Array, ArrayRef, ListArray}; +use arrow_schema::{DataType, Field}; +use datafusion_common::{cast::as_map_array, exec_err, Result}; +use datafusion_expr::{ + ArrayFunctionSignature, ColumnarValue, ScalarUDFImpl, Signature, TypeSignature, + Volatility, +}; +use std::any::Any; +use std::sync::Arc; + +make_udf_expr_and_func!( + MapValuesFunc, + map_values, + map, + "Return a list of all values in the map.", + map_values_udf +); + +#[derive(Debug)] +pub(crate) struct MapValuesFunc { + signature: Signature, +} + +impl MapValuesFunc { + pub fn new() -> Self { + Self { + signature: Signature::new( + TypeSignature::ArraySignature(ArrayFunctionSignature::MapArray), + Volatility::Immutable, + ), + } + } +} + +impl ScalarUDFImpl for MapValuesFunc { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "map_values" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> datafusion_common::Result { + if arg_types.len() != 1 { + return exec_err!("map_values expects single argument"); + } + let map_type = &arg_types[0]; + let map_fields = get_map_entry_field(map_type)?; + Ok(DataType::List(Arc::new(Field::new( + "item", + map_fields.last().unwrap().data_type().clone(), + true, + )))) + } + + fn invoke(&self, args: &[ColumnarValue]) -> datafusion_common::Result { + make_scalar_function(map_values_inner)(args) + } +} + +fn map_values_inner(args: &[ArrayRef]) -> Result { + if args.len() != 1 { + return exec_err!("map_values expects single argument"); + } + + let map_array = match args[0].data_type() { + DataType::Map(_, _) => as_map_array(&args[0])?, + _ => return exec_err!("Argument for map_values should be a map"), + }; + + Ok(Arc::new(ListArray::new( + Arc::new(Field::new("item", map_array.value_type().clone(), true)), + map_array.offsets().clone(), + Arc::clone(map_array.values()), + None, + ))) +} diff --git a/datafusion/functions-nested/src/utils.rs b/datafusion/functions-nested/src/utils.rs index 3d5b261618d5..0765f6cd237d 100644 --- a/datafusion/functions-nested/src/utils.rs +++ b/datafusion/functions-nested/src/utils.rs @@ -26,9 +26,9 @@ use arrow_array::{ UInt32Array, }; use arrow_buffer::OffsetBuffer; -use arrow_schema::Field; +use arrow_schema::{Field, Fields}; use datafusion_common::cast::{as_large_list_array, as_list_array}; -use datafusion_common::{exec_err, plan_err, Result, ScalarValue}; +use datafusion_common::{exec_err, internal_err, plan_err, Result, ScalarValue}; use core::any::type_name; use datafusion_common::DataFusionError; @@ -253,6 +253,21 @@ pub(crate) fn compute_array_dims( } } +pub(crate) fn get_map_entry_field(data_type: &DataType) -> Result<&Fields> { + match data_type { + DataType::Map(field, _) => { + let field_data_type = field.data_type(); + match field_data_type { + DataType::Struct(fields) => Ok(fields), + _ => { + internal_err!("Expected a Struct type, got {:?}", field_data_type) + } + } + } + _ => internal_err!("Expected a Map type, got {:?}", data_type), + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 270e4beccc52..c66334c4de2a 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -568,8 +568,103 @@ select map_extract(column1, 1), map_extract(column1, 5), map_extract(column1, 7) [] [[4, , 6]] [] [] [] [[1, , 3]] +# Tests for map_keys + +query ? +SELECT map_keys(MAP { 'a': 1, 2: 3 }); +---- +[a, 2] + +query ? +SELECT map_keys(MAP {'a':1, 'b':2, 'c':3 }) FROM t; +---- +[a, b, c] +[a, b, c] +[a, b, c] + +query ? +SELECT map_keys(Map{column1: column2, column3: column4}) FROM t; +---- +[a, k1] +[b, k3] +[d, k5] + +query ? +SELECT map_keys(map(column5, column6)) FROM t; +---- +[k1, k2] +[k3] +[k5] + +query ? +SELECT map_keys(map(column8, column9)) FROM t; +---- +[[1, 2, 3]] +[[4]] +[[1, 2]] + +query ? +SELECT map_keys(Map{}); +---- +[] + +query ? +SELECT map_keys(column1) from map_array_table_1; +---- +[1, 2, 3] +[4, 5, 6] +[7, 8, 9] + + +# Tests for map_values + +query ? +SELECT map_values(MAP { 'a': 1, 2: 3 }); +---- +[1, 3] + +query ? +SELECT map_values(MAP {'a':1, 'b':2, 'c':3 }) FROM t; +---- +[1, 2, 3] +[1, 2, 3] +[1, 2, 3] + +query ? +SELECT map_values(Map{column1: column2, column3: column4}) FROM t; +---- +[1, 10] +[2, 30] +[4, 50] + +query ? +SELECT map_values(map(column5, column6)) FROM t; +---- +[1, 2] +[3] +[5] + +query ? +SELECT map_values(map(column8, column9)) FROM t; +---- +[a] +[b] +[c] + +query ? +SELECT map_values(Map{}); +---- +[] + +query ? +SELECT map_values(column1) from map_array_table_1; +---- +[[1, , 3], [4, , 6], [7, 8, 9]] +[[1, , 3], [4, , 6], [7, 8, 9]] +[[1, , 3], [9, , 6], [7, 8, 9]] + statement ok drop table map_array_table_1; statement ok -drop table map_array_table_2; \ No newline at end of file +drop table map_array_table_2; diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 47e35d2e72e3..80b61f8242ef 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -3677,6 +3677,8 @@ Unwraps struct fields into columns. - [map](#map) - [make_map](#make_map) - [map_extract](#map_extract) +- [map_keys](#map_keys) +- [map_values](#map_values) ### `map` @@ -3765,6 +3767,56 @@ SELECT map_extract(MAP {'a': 1, 'b': NULL, 'c': 3}, 'a'); - element_at +### `map_keys` + +Return a list of all keys in the map. + +``` +map_keys(map) +``` + +#### Arguments + +- `map`: Map expression. + Can be a constant, column, or function, and any combination of map operators. + +#### Example + +``` +SELECT map_keys(MAP {'a': 1, 'b': NULL, 'c': 3}); +---- +[a, b, c] + +select map_keys(map([100, 5], [42,43])); +---- +[100, 5] +``` + +### `map_values` + +Return a list of all values in the map. + +``` +map_values(map) +``` + +#### Arguments + +- `map`: Map expression. + Can be a constant, column, or function, and any combination of map operators. + +#### Example + +``` +SELECT map_values(MAP {'a': 1, 'b': NULL, 'c': 3}); +---- +[1, , 3] + +select map_values(map([100, 5], [42,43])); +---- +[42, 43] +``` + ## Hashing Functions - [digest](#digest) From cefccc7be922dbfa1a0308ef1ca2952c008a3fcf Mon Sep 17 00:00:00 2001 From: wiedld Date: Sun, 1 Sep 2024 05:43:33 -0700 Subject: [PATCH 38/75] test: define current arrow_cast behavior for BinaryView (#12200) --- .../sqllogictest/test_files/arrow_typeof.slt | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/datafusion/sqllogictest/test_files/arrow_typeof.slt b/datafusion/sqllogictest/test_files/arrow_typeof.slt index b9ceb5bf05f1..bae6dc33c8cf 100644 --- a/datafusion/sqllogictest/test_files/arrow_typeof.slt +++ b/datafusion/sqllogictest/test_files/arrow_typeof.slt @@ -102,7 +102,7 @@ query error Error unrecognized word: unknown SELECT arrow_cast('1', 'unknown') # Round Trip tests: -query TTTTTTTTTTTTTTTTTTTTTTTT +query TTTTTTTTTTTTTTTTTTTTTTTTT SELECT arrow_typeof(arrow_cast(1, 'Int8')) as col_i8, arrow_typeof(arrow_cast(1, 'Int16')) as col_i16, @@ -117,6 +117,7 @@ SELECT arrow_typeof(arrow_cast(1, 'Float64')) as col_f64, arrow_typeof(arrow_cast('foo', 'Utf8')) as col_utf8, arrow_typeof(arrow_cast('foo', 'LargeUtf8')) as col_large_utf8, + arrow_typeof(arrow_cast('foo', 'Utf8View')) as col_utf8_view, arrow_typeof(arrow_cast('foo', 'Binary')) as col_binary, arrow_typeof(arrow_cast('foo', 'LargeBinary')) as col_large_binary, arrow_typeof(arrow_cast(to_timestamp('2020-01-02 01:01:11.1234567890Z'), 'Timestamp(Second, None)')) as col_ts_s, @@ -129,7 +130,7 @@ SELECT arrow_typeof(arrow_cast(to_timestamp('2020-01-02 01:01:11.1234567890Z'), 'Timestamp(Nanosecond, Some("+08:00"))')) as col_tstz_ns, arrow_typeof(arrow_cast('foo', 'Dictionary(Int32, Utf8)')) as col_dict ---- -Int8 Int16 Int32 Int64 UInt8 UInt16 UInt32 UInt64 Float16 Float32 Float64 Utf8 LargeUtf8 Binary LargeBinary Timestamp(Second, None) Timestamp(Millisecond, None) Timestamp(Microsecond, None) Timestamp(Nanosecond, None) Timestamp(Second, Some("+08:00")) Timestamp(Millisecond, Some("+08:00")) Timestamp(Microsecond, Some("+08:00")) Timestamp(Nanosecond, Some("+08:00")) Dictionary(Int32, Utf8) +Int8 Int16 Int32 Int64 UInt8 UInt16 UInt32 UInt64 Float16 Float32 Float64 Utf8 LargeUtf8 Utf8View Binary LargeBinary Timestamp(Second, None) Timestamp(Millisecond, None) Timestamp(Microsecond, None) Timestamp(Nanosecond, None) Timestamp(Second, Some("+08:00")) Timestamp(Millisecond, Some("+08:00")) Timestamp(Microsecond, Some("+08:00")) Timestamp(Nanosecond, Some("+08:00")) Dictionary(Int32, Utf8) @@ -211,21 +212,23 @@ statement ok create table foo as select arrow_cast('foo', 'Utf8') as col_utf8, arrow_cast('foo', 'LargeUtf8') as col_large_utf8, + arrow_cast('foo', 'Utf8View') as col_utf8_view, arrow_cast('foo', 'Binary') as col_binary, arrow_cast('foo', 'LargeBinary') as col_large_binary ; ## Ensure each column in the table has the expected type -query TTTT +query TTTTT SELECT arrow_typeof(col_utf8), arrow_typeof(col_large_utf8), + arrow_typeof(col_utf8_view), arrow_typeof(col_binary), arrow_typeof(col_large_binary) FROM foo; ---- -Utf8 LargeUtf8 Binary LargeBinary +Utf8 LargeUtf8 Utf8View Binary LargeBinary statement ok @@ -427,6 +430,7 @@ select arrow_cast('MyAwesomeString', 'Utf8View'), arrow_typeof(arrow_cast('MyAwe ---- MyAwesomeString Utf8View -# Fails until we update arrow-rs with support for https://github.com/apache/arrow-rs/pull/5894 -query error DataFusion error: SQL error: ParserError\("Expected: an SQL statement, found: arrow_cast"\) -arrow_cast('MyAwesomeString', 'BinaryView'), arrow_typeof(arrow_cast('MyAwesomeString', 'BinaryView')) +# Fails until we update to use the arrow-cast release with support for casting utf8 types to BinaryView +# refer to merge commit https://github.com/apache/arrow-rs/commit/4bd737dab2aa17aca200259347909d48ed793ba1 +query error DataFusion error: This feature is not implemented: Unsupported CAST from Utf8 to BinaryView +select arrow_cast('MyAwesomeString', 'BinaryView'), arrow_typeof(arrow_cast('MyAwesomeString', 'BinaryView')) From 5e9a5be84789f77fbf3757c721f60d7e38cb4bbf Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Sun, 1 Sep 2024 14:48:33 +0200 Subject: [PATCH 39/75] Update prost dependency (#12237) --- Cargo.toml | 2 ++ datafusion-examples/Cargo.toml | 2 +- datafusion/proto-common/Cargo.toml | 2 +- datafusion/proto/Cargo.toml | 2 +- datafusion/substrait/Cargo.toml | 1 + 5 files changed, 6 insertions(+), 3 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 50d234c39576..877cead93673 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -129,6 +129,8 @@ parquet = { version = "52.2.0", default-features = false, features = [ "async", "object_store", ] } +# Should match arrow-flight's version of prost. +prost = "0.12.3" rand = "0.8" regex = "1.8" rstest = "0.22.0" diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 626c365af21c..57b9930177d4 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -72,7 +72,7 @@ log = { workspace = true } mimalloc = { version = "0.1", default-features = false } num_cpus = { workspace = true } object_store = { workspace = true, features = ["aws", "http"] } -prost = { version = "0.12", default-features = false } +prost = { workspace = true } prost-derive = { version = "0.13", default-features = false } serde = { version = "1.0.136", features = ["derive"] } serde_json = { workspace = true } diff --git a/datafusion/proto-common/Cargo.toml b/datafusion/proto-common/Cargo.toml index e5d65827cdec..33a7ecd2daab 100644 --- a/datafusion/proto-common/Cargo.toml +++ b/datafusion/proto-common/Cargo.toml @@ -45,7 +45,7 @@ chrono = { workspace = true } datafusion-common = { workspace = true } object_store = { workspace = true } pbjson = { version = "0.6.0", optional = true } -prost = "0.12.0" +prost = { workspace = true } serde = { version = "1.0", optional = true } serde_json = { workspace = true, optional = true } diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 95d9e6700a50..2804ed019b61 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -53,7 +53,7 @@ datafusion-expr = { workspace = true } datafusion-proto-common = { workspace = true } object_store = { workspace = true } pbjson = { version = "0.6.0", optional = true } -prost = "0.12.0" +prost = { workspace = true } serde = { version = "1.0", optional = true } serde_json = { workspace = true, optional = true } diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index ff02ef8c7ef6..f379ac6042dd 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -39,6 +39,7 @@ datafusion = { workspace = true, default-features = true } itertools = { workspace = true } object_store = { workspace = true } pbjson-types = "0.7" +# TODO use workspace version prost = "0.13" substrait = { version = "0.41", features = ["serde"] } url = { workspace = true } From a8bca7548cb35352123d1f03bc48b04d75ddd772 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 1 Sep 2024 08:49:36 -0400 Subject: [PATCH 40/75] Bump webpack in /datafusion/wasmtest/datafusion-wasm-app (#12236) Bumps [webpack](https://github.com/webpack/webpack) from 5.88.2 to 5.94.0. - [Release notes](https://github.com/webpack/webpack/releases) - [Commits](https://github.com/webpack/webpack/compare/v5.88.2...v5.94.0) --- updated-dependencies: - dependency-name: webpack dependency-type: direct:development ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../datafusion-wasm-app/package-lock.json | 456 ++++++++---------- .../wasmtest/datafusion-wasm-app/package.json | 2 +- 2 files changed, 208 insertions(+), 250 deletions(-) diff --git a/datafusion/wasmtest/datafusion-wasm-app/package-lock.json b/datafusion/wasmtest/datafusion-wasm-app/package-lock.json index 995984a13ebc..ebb03e417e5d 100644 --- a/datafusion/wasmtest/datafusion-wasm-app/package-lock.json +++ b/datafusion/wasmtest/datafusion-wasm-app/package-lock.json @@ -13,7 +13,7 @@ }, "devDependencies": { "copy-webpack-plugin": "6.4.1", - "webpack": "5.88.2", + "webpack": "5.94.0", "webpack-cli": "5.1.4", "webpack-dev-server": "4.15.1" } @@ -38,57 +38,57 @@ "dev": true }, "node_modules/@jridgewell/gen-mapping": { - "version": "0.3.3", - "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.3.tgz", - "integrity": "sha512-HLhSWOLRi875zjjMG/r+Nv0oCW8umGb0BgEhyX3dDX3egwZtB8PqLnjz3yedt8R5StBrzcg4aBpnh8UA9D1BoQ==", + "version": "0.3.5", + "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.5.tgz", + "integrity": "sha512-IzL8ZoEDIBRWEzlCcRhOaCupYyN5gdIK+Q6fbFdPDg6HqX6jpkItn7DFIpW9LQzXG6Df9sA7+OKnq0qlz/GaQg==", "dev": true, "dependencies": { - "@jridgewell/set-array": "^1.0.1", + "@jridgewell/set-array": "^1.2.1", "@jridgewell/sourcemap-codec": "^1.4.10", - "@jridgewell/trace-mapping": "^0.3.9" + "@jridgewell/trace-mapping": "^0.3.24" }, "engines": { "node": ">=6.0.0" } }, "node_modules/@jridgewell/resolve-uri": { - "version": "3.1.1", - "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.1.tgz", - "integrity": "sha512-dSYZh7HhCDtCKm4QakX0xFpsRDqjjtZf/kjI/v3T3Nwt5r8/qz/M19F9ySyOqU94SXBmeG9ttTul+YnR4LOxFA==", + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.2.tgz", + "integrity": "sha512-bRISgCIjP20/tbWSPWMEi54QVPRZExkuD9lJL+UIxUKtwVJA8wW1Trb1jMs1RFXo1CBTNZ/5hpC9QvmKWdopKw==", "dev": true, "engines": { "node": ">=6.0.0" } }, "node_modules/@jridgewell/set-array": { - "version": "1.1.2", - "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.1.2.tgz", - "integrity": "sha512-xnkseuNADM0gt2bs+BvhO0p78Mk762YnZdsuzFV018NoG1Sj1SCQvpSqa7XUaTam5vAGasABV9qXASMKnFMwMw==", + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.2.1.tgz", + "integrity": "sha512-R8gLRTZeyp03ymzP/6Lil/28tGeGEzhx1q2k703KGWRAI1VdvPIXdG70VJc2pAMw3NA6JKL5hhFu1sJX0Mnn/A==", "dev": true, "engines": { "node": ">=6.0.0" } }, "node_modules/@jridgewell/source-map": { - "version": "0.3.5", - "resolved": "https://registry.npmjs.org/@jridgewell/source-map/-/source-map-0.3.5.tgz", - "integrity": "sha512-UTYAUj/wviwdsMfzoSJspJxbkH5o1snzwX0//0ENX1u/55kkZZkcTZP6u9bwKGkv+dkk9at4m1Cpt0uY80kcpQ==", + "version": "0.3.6", + "resolved": "https://registry.npmjs.org/@jridgewell/source-map/-/source-map-0.3.6.tgz", + "integrity": "sha512-1ZJTZebgqllO79ue2bm3rIGud/bOe0pP5BjSRCRxxYkEZS8STV7zN84UBbiYu7jy+eCKSnVIUgoWWE/tt+shMQ==", "dev": true, "dependencies": { - "@jridgewell/gen-mapping": "^0.3.0", - "@jridgewell/trace-mapping": "^0.3.9" + "@jridgewell/gen-mapping": "^0.3.5", + "@jridgewell/trace-mapping": "^0.3.25" } }, "node_modules/@jridgewell/sourcemap-codec": { - "version": "1.4.15", - "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.15.tgz", - "integrity": "sha512-eF2rxCRulEKXHTRiDrDy6erMYWqNw4LPdQ8UQA4huuxaQsVeRPFl2oM8oDGxMFhJUWZf9McpLtJasDDZb/Bpeg==", + "version": "1.5.0", + "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.5.0.tgz", + "integrity": "sha512-gv3ZRaISU3fjPAgNsriBRqGWQL6quFx04YMPW/zD8XMLsU32mhCCbfbO6KZFLjvYpCZ8zyDEgqsgf+PwPaM7GQ==", "dev": true }, "node_modules/@jridgewell/trace-mapping": { - "version": "0.3.19", - "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.19.tgz", - "integrity": "sha512-kf37QtfW+Hwx/buWGMPcR60iF9ziHa6r/CZJIHbmcm4+0qrXiVdxegAH0F6yddEVQ7zdkjcGCgCzUu+BcbhQxw==", + "version": "0.3.25", + "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.25.tgz", + "integrity": "sha512-vNk6aEwybGtawWmy/PzwnGDOjCkLWSD2wqvjGGAgOAwCGWySYXfYoxt00IJkTF+8Lb57DwOb3Aa0o9CApepiYQ==", "dev": true, "dependencies": { "@jridgewell/resolve-uri": "^3.1.0", @@ -198,30 +198,10 @@ "@types/node": "*" } }, - "node_modules/@types/eslint": { - "version": "8.44.2", - "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-8.44.2.tgz", - "integrity": "sha512-sdPRb9K6iL5XZOmBubg8yiFp5yS/JdUDQsq5e6h95km91MCYMuvp7mh1fjPEYUhvHepKpZOjnEaMBR4PxjWDzg==", - "dev": true, - "dependencies": { - "@types/estree": "*", - "@types/json-schema": "*" - } - }, - "node_modules/@types/eslint-scope": { - "version": "3.7.4", - "resolved": "https://registry.npmjs.org/@types/eslint-scope/-/eslint-scope-3.7.4.tgz", - "integrity": "sha512-9K4zoImiZc3HlIp6AVUDE4CWYx22a+lhSZMYNpbjW04+YF0KWj4pJXnEMjdnFTiQibFFmElcsasJXDbdI/EPhA==", - "dev": true, - "dependencies": { - "@types/eslint": "*", - "@types/estree": "*" - } - }, "node_modules/@types/estree": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@types/estree/-/estree-1.0.1.tgz", - "integrity": "sha512-LG4opVs2ANWZ1TJoKc937iMmNstM/d0ae1vNbnBvBhqCSezgVUOzcLCqbI5elV8Vy6WKwKjaqR+zO9VKirBBCA==", + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/@types/estree/-/estree-1.0.5.tgz", + "integrity": "sha512-/kYRxGDLWzHOB7q+wtSUQlFrtcdUccpfy+X+9iMBpHK8QLLhx2wIPYuS5DYtR9Wa/YlZAbIovy7qVdB1Aq6Lyw==", "dev": true }, "node_modules/@types/express": { @@ -348,9 +328,9 @@ } }, "node_modules/@webassemblyjs/ast": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/ast/-/ast-1.11.6.tgz", - "integrity": "sha512-IN1xI7PwOvLPgjcf180gC1bqn3q/QaOCwYUahIOhbYUu8KA/3tw2RT/T0Gidi1l7Hhj5D/INhJxiICObqpMu4Q==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/ast/-/ast-1.12.1.tgz", + "integrity": "sha512-EKfMUOPRRUTy5UII4qJDGPpqfwjOmZ5jeGFwid9mnoqIFK+e0vqoi1qH56JpmZSzEL53jKnNzScdmftJyG5xWg==", "dev": true, "dependencies": { "@webassemblyjs/helper-numbers": "1.11.6", @@ -370,9 +350,9 @@ "dev": true }, "node_modules/@webassemblyjs/helper-buffer": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-buffer/-/helper-buffer-1.11.6.tgz", - "integrity": "sha512-z3nFzdcp1mb8nEOFFk8DrYLpHvhKC3grJD2ardfKOzmbmJvEf/tPIqCY+sNcwZIY8ZD7IkB2l7/pqhUhqm7hLA==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-buffer/-/helper-buffer-1.12.1.tgz", + "integrity": "sha512-nzJwQw99DNDKr9BVCOZcLuJJUlqkJh+kVzVl6Fmq/tI5ZtEyWT1KZMyOXltXLZJmDtvLCDgwsyrkohEtopTXCw==", "dev": true }, "node_modules/@webassemblyjs/helper-numbers": { @@ -393,15 +373,15 @@ "dev": true }, "node_modules/@webassemblyjs/helper-wasm-section": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.11.6.tgz", - "integrity": "sha512-LPpZbSOwTpEC2cgn4hTydySy1Ke+XEu+ETXuoyvuyezHO3Kjdu90KK95Sh9xTbmjrCsUwvWwCOQQNta37VrS9g==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.12.1.tgz", + "integrity": "sha512-Jif4vfB6FJlUlSbgEMHUyk1j234GTNG9dBJ4XJdOySoj518Xj0oGsNi59cUQF4RRMS9ouBUxDDdyBVfPTypa5g==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.11.6", - "@webassemblyjs/helper-buffer": "1.11.6", + "@webassemblyjs/ast": "1.12.1", + "@webassemblyjs/helper-buffer": "1.12.1", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", - "@webassemblyjs/wasm-gen": "1.11.6" + "@webassemblyjs/wasm-gen": "1.12.1" } }, "node_modules/@webassemblyjs/ieee754": { @@ -429,28 +409,28 @@ "dev": true }, "node_modules/@webassemblyjs/wasm-edit": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-edit/-/wasm-edit-1.11.6.tgz", - "integrity": "sha512-Ybn2I6fnfIGuCR+Faaz7YcvtBKxvoLV3Lebn1tM4o/IAJzmi9AWYIPWpyBfU8cC+JxAO57bk4+zdsTjJR+VTOw==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-edit/-/wasm-edit-1.12.1.tgz", + "integrity": "sha512-1DuwbVvADvS5mGnXbE+c9NfA8QRcZ6iKquqjjmR10k6o+zzsRVesil54DKexiowcFCPdr/Q0qaMgB01+SQ1u6g==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.11.6", - "@webassemblyjs/helper-buffer": "1.11.6", + "@webassemblyjs/ast": "1.12.1", + "@webassemblyjs/helper-buffer": "1.12.1", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", - "@webassemblyjs/helper-wasm-section": "1.11.6", - "@webassemblyjs/wasm-gen": "1.11.6", - "@webassemblyjs/wasm-opt": "1.11.6", - "@webassemblyjs/wasm-parser": "1.11.6", - "@webassemblyjs/wast-printer": "1.11.6" + "@webassemblyjs/helper-wasm-section": "1.12.1", + "@webassemblyjs/wasm-gen": "1.12.1", + "@webassemblyjs/wasm-opt": "1.12.1", + "@webassemblyjs/wasm-parser": "1.12.1", + "@webassemblyjs/wast-printer": "1.12.1" } }, "node_modules/@webassemblyjs/wasm-gen": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-gen/-/wasm-gen-1.11.6.tgz", - "integrity": "sha512-3XOqkZP/y6B4F0PBAXvI1/bky7GryoogUtfwExeP/v7Nzwo1QLcq5oQmpKlftZLbT+ERUOAZVQjuNVak6UXjPA==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-gen/-/wasm-gen-1.12.1.tgz", + "integrity": "sha512-TDq4Ojh9fcohAw6OIMXqiIcTq5KUXTGRkVxbSo1hQnSy6lAM5GSdfwWeSxpAo0YzgsgF182E/U0mDNhuA0tW7w==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.11.6", + "@webassemblyjs/ast": "1.12.1", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", "@webassemblyjs/ieee754": "1.11.6", "@webassemblyjs/leb128": "1.11.6", @@ -458,24 +438,24 @@ } }, "node_modules/@webassemblyjs/wasm-opt": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-opt/-/wasm-opt-1.11.6.tgz", - "integrity": "sha512-cOrKuLRE7PCe6AsOVl7WasYf3wbSo4CeOk6PkrjS7g57MFfVUF9u6ysQBBODX0LdgSvQqRiGz3CXvIDKcPNy4g==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-opt/-/wasm-opt-1.12.1.tgz", + "integrity": "sha512-Jg99j/2gG2iaz3hijw857AVYekZe2SAskcqlWIZXjji5WStnOpVoat3gQfT/Q5tb2djnCjBtMocY/Su1GfxPBg==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.11.6", - "@webassemblyjs/helper-buffer": "1.11.6", - "@webassemblyjs/wasm-gen": "1.11.6", - "@webassemblyjs/wasm-parser": "1.11.6" + "@webassemblyjs/ast": "1.12.1", + "@webassemblyjs/helper-buffer": "1.12.1", + "@webassemblyjs/wasm-gen": "1.12.1", + "@webassemblyjs/wasm-parser": "1.12.1" } }, "node_modules/@webassemblyjs/wasm-parser": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-parser/-/wasm-parser-1.11.6.tgz", - "integrity": "sha512-6ZwPeGzMJM3Dqp3hCsLgESxBGtT/OeCvCZ4TA1JUPYgmhAx38tTPR9JaKy0S5H3evQpO/h2uWs2j6Yc/fjkpTQ==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-parser/-/wasm-parser-1.12.1.tgz", + "integrity": "sha512-xikIi7c2FHXysxXe3COrVUPSheuBtpcfhbpFj4gmu7KRLYOzANztwUU0IbsqvMqzuNK2+glRGWCEqZo1WCLyAQ==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.11.6", + "@webassemblyjs/ast": "1.12.1", "@webassemblyjs/helper-api-error": "1.11.6", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", "@webassemblyjs/ieee754": "1.11.6", @@ -484,12 +464,12 @@ } }, "node_modules/@webassemblyjs/wast-printer": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-printer/-/wast-printer-1.11.6.tgz", - "integrity": "sha512-JM7AhRcE+yW2GWYaKeHL5vt4xqee5N2WcezptmgyhNS+ScggqcT1OtXykhAb13Sn5Yas0j2uv9tHgrjwvzAP4A==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-printer/-/wast-printer-1.12.1.tgz", + "integrity": "sha512-+X4WAlOisVWQMikjbcvY2e0rwPsKQ9F688lksZhBcPycBBuii3O7m8FACbDMWDojpAqvjIncrG8J0XHKyQfVeA==", "dev": true, "dependencies": { - "@webassemblyjs/ast": "1.11.6", + "@webassemblyjs/ast": "1.12.1", "@xtuc/long": "4.2.2" } }, @@ -563,9 +543,9 @@ } }, "node_modules/acorn": { - "version": "8.10.0", - "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.10.0.tgz", - "integrity": "sha512-F0SAmZ8iUtS//m8DmCTA0jlh6TDKkHQyK6xc6V4KDTyZKA9dnvX9/3sRTVQrWm79glUAZbnmmNcdYwUIHWVybw==", + "version": "8.12.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.12.1.tgz", + "integrity": "sha512-tcpGyI9zbizT9JbV6oYE477V6mTlXvvi0T0G3SNIYE2apm/G5huBa1+K89VGeovbg+jycCrfhl3ADxErOuO6Jg==", "dev": true, "bin": { "acorn": "bin/acorn" @@ -574,10 +554,10 @@ "node": ">=0.4.0" } }, - "node_modules/acorn-import-assertions": { - "version": "1.9.0", - "resolved": "https://registry.npmjs.org/acorn-import-assertions/-/acorn-import-assertions-1.9.0.tgz", - "integrity": "sha512-cmMwop9x+8KFhxvKrKfPYmN6/pKTYYHBqLa0DfvVZcKMJWNyWLnaqND7dx/qn66R7ewM1UX5XMaDVP5wlVTaVA==", + "node_modules/acorn-import-attributes": { + "version": "1.9.5", + "resolved": "https://registry.npmjs.org/acorn-import-attributes/-/acorn-import-attributes-1.9.5.tgz", + "integrity": "sha512-n02Vykv5uA3eHGM/Z2dQrcD56kL8TyDb2p1+0P83PClMnC/nc+anbQRhIOWnSq4Ke/KvDPrY3C9hDtC/A3eHnQ==", "dev": true, "peerDependencies": { "acorn": "^8" @@ -1322,9 +1302,9 @@ } }, "node_modules/enhanced-resolve": { - "version": "5.15.0", - "resolved": "https://registry.npmjs.org/enhanced-resolve/-/enhanced-resolve-5.15.0.tgz", - "integrity": "sha512-LXYT42KJ7lpIKECr2mAXIaMldcNCh/7E0KBKOu4KSfkHmP+mZmSs+8V5gBAqisWBy0OO4W5Oyys0GO1Y8KtdKg==", + "version": "5.17.1", + "resolved": "https://registry.npmjs.org/enhanced-resolve/-/enhanced-resolve-5.17.1.tgz", + "integrity": "sha512-LMHl3dXhTcfv8gM4kEzIUeTQ+7fpdA0l2tUf34BddXPkz2A5xJ5L/Pchd5BL6rdccM9QGvu0sWZzK1Z1t4wwyg==", "dev": true, "dependencies": { "graceful-fs": "^4.2.4", @@ -3724,9 +3704,9 @@ } }, "node_modules/terser": { - "version": "5.20.0", - "resolved": "https://registry.npmjs.org/terser/-/terser-5.20.0.tgz", - "integrity": "sha512-e56ETryaQDyebBwJIWYB2TT6f2EZ0fL0sW/JRXNMN26zZdKi2u/E/5my5lG6jNxym6qsrVXfFRmOdV42zlAgLQ==", + "version": "5.31.6", + "resolved": "https://registry.npmjs.org/terser/-/terser-5.31.6.tgz", + "integrity": "sha512-PQ4DAriWzKj+qgehQ7LK5bQqCFNMmlhjR2PFFLuqGCpuCAauxemVBWwWOxo3UIwWQx8+Pr61Df++r76wDmkQBg==", "dev": true, "dependencies": { "@jridgewell/source-map": "^0.3.3", @@ -3742,16 +3722,16 @@ } }, "node_modules/terser-webpack-plugin": { - "version": "5.3.9", - "resolved": "https://registry.npmjs.org/terser-webpack-plugin/-/terser-webpack-plugin-5.3.9.tgz", - "integrity": "sha512-ZuXsqE07EcggTWQjXUj+Aot/OMcD0bMKGgF63f7UxYcu5/AJF53aIpK1YoP5xR9l6s/Hy2b+t1AM0bLNPRuhwA==", + "version": "5.3.10", + "resolved": "https://registry.npmjs.org/terser-webpack-plugin/-/terser-webpack-plugin-5.3.10.tgz", + "integrity": "sha512-BKFPWlPDndPs+NGGCr1U59t0XScL5317Y0UReNrHaw9/FwhPENlq6bfgs+4yPfyP51vqC1bQ4rp1EfXW5ZSH9w==", "dev": true, "dependencies": { - "@jridgewell/trace-mapping": "^0.3.17", + "@jridgewell/trace-mapping": "^0.3.20", "jest-worker": "^27.4.5", "schema-utils": "^3.1.1", "serialize-javascript": "^6.0.1", - "terser": "^5.16.8" + "terser": "^5.26.0" }, "engines": { "node": ">= 10.13.0" @@ -3776,9 +3756,9 @@ } }, "node_modules/terser-webpack-plugin/node_modules/serialize-javascript": { - "version": "6.0.1", - "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.1.tgz", - "integrity": "sha512-owoXEFjWRllis8/M1Q+Cw5k8ZH40e3zhp/ovX+Xr/vi1qj6QesbyXXViFbpNvWvPNAD62SutwEXavefrLJWj7w==", + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.2.tgz", + "integrity": "sha512-Saa1xPByTTq2gdeFZYLLo+RFE35NHZkAbqZeWNd3BpzppeVisAqpDjcp8dyf6uIvEqJRd46jemmyA4iFIeVk8g==", "dev": true, "dependencies": { "randombytes": "^2.1.0" @@ -3930,9 +3910,9 @@ } }, "node_modules/watchpack": { - "version": "2.4.0", - "resolved": "https://registry.npmjs.org/watchpack/-/watchpack-2.4.0.tgz", - "integrity": "sha512-Lcvm7MGST/4fup+ifyKi2hjyIAwcdI4HRgtvTpIUxBRhB+RFtUh8XtDOxUfctVCnhVi+QQj49i91OyvzkJl6cg==", + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/watchpack/-/watchpack-2.4.2.tgz", + "integrity": "sha512-TnbFSbcOCcDgjZ4piURLCbJ3nJhznVh9kw6F6iokjiFPl8ONxe9A6nMDVXDiNbrSfLILs6vB07F7wLBrwPYzJw==", "dev": true, "dependencies": { "glob-to-regexp": "^0.4.1", @@ -3952,34 +3932,33 @@ } }, "node_modules/webpack": { - "version": "5.88.2", - "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.88.2.tgz", - "integrity": "sha512-JmcgNZ1iKj+aiR0OvTYtWQqJwq37Pf683dY9bVORwVbUrDhLhdn/PlO2sHsFHPkj7sHNQF3JwaAkp49V+Sq1tQ==", + "version": "5.94.0", + "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.94.0.tgz", + "integrity": "sha512-KcsGn50VT+06JH/iunZJedYGUJS5FGjow8wb9c0v5n1Om8O1g4L6LjtfxwlXIATopoQu+vOXXa7gYisWxCoPyg==", "dev": true, "dependencies": { - "@types/eslint-scope": "^3.7.3", - "@types/estree": "^1.0.0", - "@webassemblyjs/ast": "^1.11.5", - "@webassemblyjs/wasm-edit": "^1.11.5", - "@webassemblyjs/wasm-parser": "^1.11.5", + "@types/estree": "^1.0.5", + "@webassemblyjs/ast": "^1.12.1", + "@webassemblyjs/wasm-edit": "^1.12.1", + "@webassemblyjs/wasm-parser": "^1.12.1", "acorn": "^8.7.1", - "acorn-import-assertions": "^1.9.0", - "browserslist": "^4.14.5", + "acorn-import-attributes": "^1.9.5", + "browserslist": "^4.21.10", "chrome-trace-event": "^1.0.2", - "enhanced-resolve": "^5.15.0", + "enhanced-resolve": "^5.17.1", "es-module-lexer": "^1.2.1", "eslint-scope": "5.1.1", "events": "^3.2.0", "glob-to-regexp": "^0.4.1", - "graceful-fs": "^4.2.9", + "graceful-fs": "^4.2.11", "json-parse-even-better-errors": "^2.3.1", "loader-runner": "^4.2.0", "mime-types": "^2.1.27", "neo-async": "^2.6.2", "schema-utils": "^3.2.0", "tapable": "^2.1.1", - "terser-webpack-plugin": "^5.3.7", - "watchpack": "^2.4.0", + "terser-webpack-plugin": "^5.3.10", + "watchpack": "^2.4.1", "webpack-sources": "^3.2.3" }, "bin": { @@ -4376,48 +4355,48 @@ "dev": true }, "@jridgewell/gen-mapping": { - "version": "0.3.3", - "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.3.tgz", - "integrity": "sha512-HLhSWOLRi875zjjMG/r+Nv0oCW8umGb0BgEhyX3dDX3egwZtB8PqLnjz3yedt8R5StBrzcg4aBpnh8UA9D1BoQ==", + "version": "0.3.5", + "resolved": "https://registry.npmjs.org/@jridgewell/gen-mapping/-/gen-mapping-0.3.5.tgz", + "integrity": "sha512-IzL8ZoEDIBRWEzlCcRhOaCupYyN5gdIK+Q6fbFdPDg6HqX6jpkItn7DFIpW9LQzXG6Df9sA7+OKnq0qlz/GaQg==", "dev": true, "requires": { - "@jridgewell/set-array": "^1.0.1", + "@jridgewell/set-array": "^1.2.1", "@jridgewell/sourcemap-codec": "^1.4.10", - "@jridgewell/trace-mapping": "^0.3.9" + "@jridgewell/trace-mapping": "^0.3.24" } }, "@jridgewell/resolve-uri": { - "version": "3.1.1", - "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.1.tgz", - "integrity": "sha512-dSYZh7HhCDtCKm4QakX0xFpsRDqjjtZf/kjI/v3T3Nwt5r8/qz/M19F9ySyOqU94SXBmeG9ttTul+YnR4LOxFA==", + "version": "3.1.2", + "resolved": "https://registry.npmjs.org/@jridgewell/resolve-uri/-/resolve-uri-3.1.2.tgz", + "integrity": "sha512-bRISgCIjP20/tbWSPWMEi54QVPRZExkuD9lJL+UIxUKtwVJA8wW1Trb1jMs1RFXo1CBTNZ/5hpC9QvmKWdopKw==", "dev": true }, "@jridgewell/set-array": { - "version": "1.1.2", - "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.1.2.tgz", - "integrity": "sha512-xnkseuNADM0gt2bs+BvhO0p78Mk762YnZdsuzFV018NoG1Sj1SCQvpSqa7XUaTam5vAGasABV9qXASMKnFMwMw==", + "version": "1.2.1", + "resolved": "https://registry.npmjs.org/@jridgewell/set-array/-/set-array-1.2.1.tgz", + "integrity": "sha512-R8gLRTZeyp03ymzP/6Lil/28tGeGEzhx1q2k703KGWRAI1VdvPIXdG70VJc2pAMw3NA6JKL5hhFu1sJX0Mnn/A==", "dev": true }, "@jridgewell/source-map": { - "version": "0.3.5", - "resolved": "https://registry.npmjs.org/@jridgewell/source-map/-/source-map-0.3.5.tgz", - "integrity": "sha512-UTYAUj/wviwdsMfzoSJspJxbkH5o1snzwX0//0ENX1u/55kkZZkcTZP6u9bwKGkv+dkk9at4m1Cpt0uY80kcpQ==", + "version": "0.3.6", + "resolved": "https://registry.npmjs.org/@jridgewell/source-map/-/source-map-0.3.6.tgz", + "integrity": "sha512-1ZJTZebgqllO79ue2bm3rIGud/bOe0pP5BjSRCRxxYkEZS8STV7zN84UBbiYu7jy+eCKSnVIUgoWWE/tt+shMQ==", "dev": true, "requires": { - "@jridgewell/gen-mapping": "^0.3.0", - "@jridgewell/trace-mapping": "^0.3.9" + "@jridgewell/gen-mapping": "^0.3.5", + "@jridgewell/trace-mapping": "^0.3.25" } }, "@jridgewell/sourcemap-codec": { - "version": "1.4.15", - "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.4.15.tgz", - "integrity": "sha512-eF2rxCRulEKXHTRiDrDy6erMYWqNw4LPdQ8UQA4huuxaQsVeRPFl2oM8oDGxMFhJUWZf9McpLtJasDDZb/Bpeg==", + "version": "1.5.0", + "resolved": "https://registry.npmjs.org/@jridgewell/sourcemap-codec/-/sourcemap-codec-1.5.0.tgz", + "integrity": "sha512-gv3ZRaISU3fjPAgNsriBRqGWQL6quFx04YMPW/zD8XMLsU32mhCCbfbO6KZFLjvYpCZ8zyDEgqsgf+PwPaM7GQ==", "dev": true }, "@jridgewell/trace-mapping": { - "version": "0.3.19", - "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.19.tgz", - "integrity": "sha512-kf37QtfW+Hwx/buWGMPcR60iF9ziHa6r/CZJIHbmcm4+0qrXiVdxegAH0F6yddEVQ7zdkjcGCgCzUu+BcbhQxw==", + "version": "0.3.25", + "resolved": "https://registry.npmjs.org/@jridgewell/trace-mapping/-/trace-mapping-0.3.25.tgz", + "integrity": "sha512-vNk6aEwybGtawWmy/PzwnGDOjCkLWSD2wqvjGGAgOAwCGWySYXfYoxt00IJkTF+8Lb57DwOb3Aa0o9CApepiYQ==", "dev": true, "requires": { "@jridgewell/resolve-uri": "^3.1.0", @@ -4514,30 +4493,10 @@ "@types/node": "*" } }, - "@types/eslint": { - "version": "8.44.2", - "resolved": "https://registry.npmjs.org/@types/eslint/-/eslint-8.44.2.tgz", - "integrity": "sha512-sdPRb9K6iL5XZOmBubg8yiFp5yS/JdUDQsq5e6h95km91MCYMuvp7mh1fjPEYUhvHepKpZOjnEaMBR4PxjWDzg==", - "dev": true, - "requires": { - "@types/estree": "*", - "@types/json-schema": "*" - } - }, - "@types/eslint-scope": { - "version": "3.7.4", - "resolved": "https://registry.npmjs.org/@types/eslint-scope/-/eslint-scope-3.7.4.tgz", - "integrity": "sha512-9K4zoImiZc3HlIp6AVUDE4CWYx22a+lhSZMYNpbjW04+YF0KWj4pJXnEMjdnFTiQibFFmElcsasJXDbdI/EPhA==", - "dev": true, - "requires": { - "@types/eslint": "*", - "@types/estree": "*" - } - }, "@types/estree": { - "version": "1.0.1", - "resolved": "https://registry.npmjs.org/@types/estree/-/estree-1.0.1.tgz", - "integrity": "sha512-LG4opVs2ANWZ1TJoKc937iMmNstM/d0ae1vNbnBvBhqCSezgVUOzcLCqbI5elV8Vy6WKwKjaqR+zO9VKirBBCA==", + "version": "1.0.5", + "resolved": "https://registry.npmjs.org/@types/estree/-/estree-1.0.5.tgz", + "integrity": "sha512-/kYRxGDLWzHOB7q+wtSUQlFrtcdUccpfy+X+9iMBpHK8QLLhx2wIPYuS5DYtR9Wa/YlZAbIovy7qVdB1Aq6Lyw==", "dev": true }, "@types/express": { @@ -4664,9 +4623,9 @@ } }, "@webassemblyjs/ast": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/ast/-/ast-1.11.6.tgz", - "integrity": "sha512-IN1xI7PwOvLPgjcf180gC1bqn3q/QaOCwYUahIOhbYUu8KA/3tw2RT/T0Gidi1l7Hhj5D/INhJxiICObqpMu4Q==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/ast/-/ast-1.12.1.tgz", + "integrity": "sha512-EKfMUOPRRUTy5UII4qJDGPpqfwjOmZ5jeGFwid9mnoqIFK+e0vqoi1qH56JpmZSzEL53jKnNzScdmftJyG5xWg==", "dev": true, "requires": { "@webassemblyjs/helper-numbers": "1.11.6", @@ -4686,9 +4645,9 @@ "dev": true }, "@webassemblyjs/helper-buffer": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-buffer/-/helper-buffer-1.11.6.tgz", - "integrity": "sha512-z3nFzdcp1mb8nEOFFk8DrYLpHvhKC3grJD2ardfKOzmbmJvEf/tPIqCY+sNcwZIY8ZD7IkB2l7/pqhUhqm7hLA==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-buffer/-/helper-buffer-1.12.1.tgz", + "integrity": "sha512-nzJwQw99DNDKr9BVCOZcLuJJUlqkJh+kVzVl6Fmq/tI5ZtEyWT1KZMyOXltXLZJmDtvLCDgwsyrkohEtopTXCw==", "dev": true }, "@webassemblyjs/helper-numbers": { @@ -4709,15 +4668,15 @@ "dev": true }, "@webassemblyjs/helper-wasm-section": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.11.6.tgz", - "integrity": "sha512-LPpZbSOwTpEC2cgn4hTydySy1Ke+XEu+ETXuoyvuyezHO3Kjdu90KK95Sh9xTbmjrCsUwvWwCOQQNta37VrS9g==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/helper-wasm-section/-/helper-wasm-section-1.12.1.tgz", + "integrity": "sha512-Jif4vfB6FJlUlSbgEMHUyk1j234GTNG9dBJ4XJdOySoj518Xj0oGsNi59cUQF4RRMS9ouBUxDDdyBVfPTypa5g==", "dev": true, "requires": { - "@webassemblyjs/ast": "1.11.6", - "@webassemblyjs/helper-buffer": "1.11.6", + "@webassemblyjs/ast": "1.12.1", + "@webassemblyjs/helper-buffer": "1.12.1", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", - "@webassemblyjs/wasm-gen": "1.11.6" + "@webassemblyjs/wasm-gen": "1.12.1" } }, "@webassemblyjs/ieee754": { @@ -4745,28 +4704,28 @@ "dev": true }, "@webassemblyjs/wasm-edit": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-edit/-/wasm-edit-1.11.6.tgz", - "integrity": "sha512-Ybn2I6fnfIGuCR+Faaz7YcvtBKxvoLV3Lebn1tM4o/IAJzmi9AWYIPWpyBfU8cC+JxAO57bk4+zdsTjJR+VTOw==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-edit/-/wasm-edit-1.12.1.tgz", + "integrity": "sha512-1DuwbVvADvS5mGnXbE+c9NfA8QRcZ6iKquqjjmR10k6o+zzsRVesil54DKexiowcFCPdr/Q0qaMgB01+SQ1u6g==", "dev": true, "requires": { - "@webassemblyjs/ast": "1.11.6", - "@webassemblyjs/helper-buffer": "1.11.6", + "@webassemblyjs/ast": "1.12.1", + "@webassemblyjs/helper-buffer": "1.12.1", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", - "@webassemblyjs/helper-wasm-section": "1.11.6", - "@webassemblyjs/wasm-gen": "1.11.6", - "@webassemblyjs/wasm-opt": "1.11.6", - "@webassemblyjs/wasm-parser": "1.11.6", - "@webassemblyjs/wast-printer": "1.11.6" + "@webassemblyjs/helper-wasm-section": "1.12.1", + "@webassemblyjs/wasm-gen": "1.12.1", + "@webassemblyjs/wasm-opt": "1.12.1", + "@webassemblyjs/wasm-parser": "1.12.1", + "@webassemblyjs/wast-printer": "1.12.1" } }, "@webassemblyjs/wasm-gen": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-gen/-/wasm-gen-1.11.6.tgz", - "integrity": "sha512-3XOqkZP/y6B4F0PBAXvI1/bky7GryoogUtfwExeP/v7Nzwo1QLcq5oQmpKlftZLbT+ERUOAZVQjuNVak6UXjPA==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-gen/-/wasm-gen-1.12.1.tgz", + "integrity": "sha512-TDq4Ojh9fcohAw6OIMXqiIcTq5KUXTGRkVxbSo1hQnSy6lAM5GSdfwWeSxpAo0YzgsgF182E/U0mDNhuA0tW7w==", "dev": true, "requires": { - "@webassemblyjs/ast": "1.11.6", + "@webassemblyjs/ast": "1.12.1", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", "@webassemblyjs/ieee754": "1.11.6", "@webassemblyjs/leb128": "1.11.6", @@ -4774,24 +4733,24 @@ } }, "@webassemblyjs/wasm-opt": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-opt/-/wasm-opt-1.11.6.tgz", - "integrity": "sha512-cOrKuLRE7PCe6AsOVl7WasYf3wbSo4CeOk6PkrjS7g57MFfVUF9u6ysQBBODX0LdgSvQqRiGz3CXvIDKcPNy4g==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-opt/-/wasm-opt-1.12.1.tgz", + "integrity": "sha512-Jg99j/2gG2iaz3hijw857AVYekZe2SAskcqlWIZXjji5WStnOpVoat3gQfT/Q5tb2djnCjBtMocY/Su1GfxPBg==", "dev": true, "requires": { - "@webassemblyjs/ast": "1.11.6", - "@webassemblyjs/helper-buffer": "1.11.6", - "@webassemblyjs/wasm-gen": "1.11.6", - "@webassemblyjs/wasm-parser": "1.11.6" + "@webassemblyjs/ast": "1.12.1", + "@webassemblyjs/helper-buffer": "1.12.1", + "@webassemblyjs/wasm-gen": "1.12.1", + "@webassemblyjs/wasm-parser": "1.12.1" } }, "@webassemblyjs/wasm-parser": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-parser/-/wasm-parser-1.11.6.tgz", - "integrity": "sha512-6ZwPeGzMJM3Dqp3hCsLgESxBGtT/OeCvCZ4TA1JUPYgmhAx38tTPR9JaKy0S5H3evQpO/h2uWs2j6Yc/fjkpTQ==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wasm-parser/-/wasm-parser-1.12.1.tgz", + "integrity": "sha512-xikIi7c2FHXysxXe3COrVUPSheuBtpcfhbpFj4gmu7KRLYOzANztwUU0IbsqvMqzuNK2+glRGWCEqZo1WCLyAQ==", "dev": true, "requires": { - "@webassemblyjs/ast": "1.11.6", + "@webassemblyjs/ast": "1.12.1", "@webassemblyjs/helper-api-error": "1.11.6", "@webassemblyjs/helper-wasm-bytecode": "1.11.6", "@webassemblyjs/ieee754": "1.11.6", @@ -4800,12 +4759,12 @@ } }, "@webassemblyjs/wast-printer": { - "version": "1.11.6", - "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-printer/-/wast-printer-1.11.6.tgz", - "integrity": "sha512-JM7AhRcE+yW2GWYaKeHL5vt4xqee5N2WcezptmgyhNS+ScggqcT1OtXykhAb13Sn5Yas0j2uv9tHgrjwvzAP4A==", + "version": "1.12.1", + "resolved": "https://registry.npmjs.org/@webassemblyjs/wast-printer/-/wast-printer-1.12.1.tgz", + "integrity": "sha512-+X4WAlOisVWQMikjbcvY2e0rwPsKQ9F688lksZhBcPycBBuii3O7m8FACbDMWDojpAqvjIncrG8J0XHKyQfVeA==", "dev": true, "requires": { - "@webassemblyjs/ast": "1.11.6", + "@webassemblyjs/ast": "1.12.1", "@xtuc/long": "4.2.2" } }, @@ -4853,15 +4812,15 @@ } }, "acorn": { - "version": "8.10.0", - "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.10.0.tgz", - "integrity": "sha512-F0SAmZ8iUtS//m8DmCTA0jlh6TDKkHQyK6xc6V4KDTyZKA9dnvX9/3sRTVQrWm79glUAZbnmmNcdYwUIHWVybw==", + "version": "8.12.1", + "resolved": "https://registry.npmjs.org/acorn/-/acorn-8.12.1.tgz", + "integrity": "sha512-tcpGyI9zbizT9JbV6oYE477V6mTlXvvi0T0G3SNIYE2apm/G5huBa1+K89VGeovbg+jycCrfhl3ADxErOuO6Jg==", "dev": true }, - "acorn-import-assertions": { - "version": "1.9.0", - "resolved": "https://registry.npmjs.org/acorn-import-assertions/-/acorn-import-assertions-1.9.0.tgz", - "integrity": "sha512-cmMwop9x+8KFhxvKrKfPYmN6/pKTYYHBqLa0DfvVZcKMJWNyWLnaqND7dx/qn66R7ewM1UX5XMaDVP5wlVTaVA==", + "acorn-import-attributes": { + "version": "1.9.5", + "resolved": "https://registry.npmjs.org/acorn-import-attributes/-/acorn-import-attributes-1.9.5.tgz", + "integrity": "sha512-n02Vykv5uA3eHGM/Z2dQrcD56kL8TyDb2p1+0P83PClMnC/nc+anbQRhIOWnSq4Ke/KvDPrY3C9hDtC/A3eHnQ==", "dev": true, "requires": {} }, @@ -5415,9 +5374,9 @@ "dev": true }, "enhanced-resolve": { - "version": "5.15.0", - "resolved": "https://registry.npmjs.org/enhanced-resolve/-/enhanced-resolve-5.15.0.tgz", - "integrity": "sha512-LXYT42KJ7lpIKECr2mAXIaMldcNCh/7E0KBKOu4KSfkHmP+mZmSs+8V5gBAqisWBy0OO4W5Oyys0GO1Y8KtdKg==", + "version": "5.17.1", + "resolved": "https://registry.npmjs.org/enhanced-resolve/-/enhanced-resolve-5.17.1.tgz", + "integrity": "sha512-LMHl3dXhTcfv8gM4kEzIUeTQ+7fpdA0l2tUf34BddXPkz2A5xJ5L/Pchd5BL6rdccM9QGvu0sWZzK1Z1t4wwyg==", "dev": true, "requires": { "graceful-fs": "^4.2.4", @@ -7199,9 +7158,9 @@ } }, "terser": { - "version": "5.20.0", - "resolved": "https://registry.npmjs.org/terser/-/terser-5.20.0.tgz", - "integrity": "sha512-e56ETryaQDyebBwJIWYB2TT6f2EZ0fL0sW/JRXNMN26zZdKi2u/E/5my5lG6jNxym6qsrVXfFRmOdV42zlAgLQ==", + "version": "5.31.6", + "resolved": "https://registry.npmjs.org/terser/-/terser-5.31.6.tgz", + "integrity": "sha512-PQ4DAriWzKj+qgehQ7LK5bQqCFNMmlhjR2PFFLuqGCpuCAauxemVBWwWOxo3UIwWQx8+Pr61Df++r76wDmkQBg==", "dev": true, "requires": { "@jridgewell/source-map": "^0.3.3", @@ -7211,22 +7170,22 @@ } }, "terser-webpack-plugin": { - "version": "5.3.9", - "resolved": "https://registry.npmjs.org/terser-webpack-plugin/-/terser-webpack-plugin-5.3.9.tgz", - "integrity": "sha512-ZuXsqE07EcggTWQjXUj+Aot/OMcD0bMKGgF63f7UxYcu5/AJF53aIpK1YoP5xR9l6s/Hy2b+t1AM0bLNPRuhwA==", + "version": "5.3.10", + "resolved": "https://registry.npmjs.org/terser-webpack-plugin/-/terser-webpack-plugin-5.3.10.tgz", + "integrity": "sha512-BKFPWlPDndPs+NGGCr1U59t0XScL5317Y0UReNrHaw9/FwhPENlq6bfgs+4yPfyP51vqC1bQ4rp1EfXW5ZSH9w==", "dev": true, "requires": { - "@jridgewell/trace-mapping": "^0.3.17", + "@jridgewell/trace-mapping": "^0.3.20", "jest-worker": "^27.4.5", "schema-utils": "^3.1.1", "serialize-javascript": "^6.0.1", - "terser": "^5.16.8" + "terser": "^5.26.0" }, "dependencies": { "serialize-javascript": { - "version": "6.0.1", - "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.1.tgz", - "integrity": "sha512-owoXEFjWRllis8/M1Q+Cw5k8ZH40e3zhp/ovX+Xr/vi1qj6QesbyXXViFbpNvWvPNAD62SutwEXavefrLJWj7w==", + "version": "6.0.2", + "resolved": "https://registry.npmjs.org/serialize-javascript/-/serialize-javascript-6.0.2.tgz", + "integrity": "sha512-Saa1xPByTTq2gdeFZYLLo+RFE35NHZkAbqZeWNd3BpzppeVisAqpDjcp8dyf6uIvEqJRd46jemmyA4iFIeVk8g==", "dev": true, "requires": { "randombytes": "^2.1.0" @@ -7339,9 +7298,9 @@ "dev": true }, "watchpack": { - "version": "2.4.0", - "resolved": "https://registry.npmjs.org/watchpack/-/watchpack-2.4.0.tgz", - "integrity": "sha512-Lcvm7MGST/4fup+ifyKi2hjyIAwcdI4HRgtvTpIUxBRhB+RFtUh8XtDOxUfctVCnhVi+QQj49i91OyvzkJl6cg==", + "version": "2.4.2", + "resolved": "https://registry.npmjs.org/watchpack/-/watchpack-2.4.2.tgz", + "integrity": "sha512-TnbFSbcOCcDgjZ4piURLCbJ3nJhznVh9kw6F6iokjiFPl8ONxe9A6nMDVXDiNbrSfLILs6vB07F7wLBrwPYzJw==", "dev": true, "requires": { "glob-to-regexp": "^0.4.1", @@ -7358,34 +7317,33 @@ } }, "webpack": { - "version": "5.88.2", - "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.88.2.tgz", - "integrity": "sha512-JmcgNZ1iKj+aiR0OvTYtWQqJwq37Pf683dY9bVORwVbUrDhLhdn/PlO2sHsFHPkj7sHNQF3JwaAkp49V+Sq1tQ==", + "version": "5.94.0", + "resolved": "https://registry.npmjs.org/webpack/-/webpack-5.94.0.tgz", + "integrity": "sha512-KcsGn50VT+06JH/iunZJedYGUJS5FGjow8wb9c0v5n1Om8O1g4L6LjtfxwlXIATopoQu+vOXXa7gYisWxCoPyg==", "dev": true, "requires": { - "@types/eslint-scope": "^3.7.3", - "@types/estree": "^1.0.0", - "@webassemblyjs/ast": "^1.11.5", - "@webassemblyjs/wasm-edit": "^1.11.5", - "@webassemblyjs/wasm-parser": "^1.11.5", + "@types/estree": "^1.0.5", + "@webassemblyjs/ast": "^1.12.1", + "@webassemblyjs/wasm-edit": "^1.12.1", + "@webassemblyjs/wasm-parser": "^1.12.1", "acorn": "^8.7.1", - "acorn-import-assertions": "^1.9.0", - "browserslist": "^4.14.5", + "acorn-import-attributes": "^1.9.5", + "browserslist": "^4.21.10", "chrome-trace-event": "^1.0.2", - "enhanced-resolve": "^5.15.0", + "enhanced-resolve": "^5.17.1", "es-module-lexer": "^1.2.1", "eslint-scope": "5.1.1", "events": "^3.2.0", "glob-to-regexp": "^0.4.1", - "graceful-fs": "^4.2.9", + "graceful-fs": "^4.2.11", "json-parse-even-better-errors": "^2.3.1", "loader-runner": "^4.2.0", "mime-types": "^2.1.27", "neo-async": "^2.6.2", "schema-utils": "^3.2.0", "tapable": "^2.1.1", - "terser-webpack-plugin": "^5.3.7", - "watchpack": "^2.4.0", + "terser-webpack-plugin": "^5.3.10", + "watchpack": "^2.4.1", "webpack-sources": "^3.2.3" }, "dependencies": { diff --git a/datafusion/wasmtest/datafusion-wasm-app/package.json b/datafusion/wasmtest/datafusion-wasm-app/package.json index cd32070fa0bc..0860473276ea 100644 --- a/datafusion/wasmtest/datafusion-wasm-app/package.json +++ b/datafusion/wasmtest/datafusion-wasm-app/package.json @@ -27,7 +27,7 @@ "datafusion-wasmtest": "../pkg" }, "devDependencies": { - "webpack": "5.88.2", + "webpack": "5.94.0", "webpack-cli": "5.1.4", "webpack-dev-server": "4.15.1", "copy-webpack-plugin": "6.4.1" From 5272007f364b8d43deab19f530e6d7a6ffe7b678 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Sun, 1 Sep 2024 14:59:32 +0200 Subject: [PATCH 41/75] Avoid redundant pass-by-value in physical optimizer (#12261) --- .../enforce_distribution.rs | 6 ++--- .../physical_optimizer/projection_pushdown.rs | 24 ++++++++--------- .../src/physical_optimizer/sort_pushdown.rs | 26 ++++++++++--------- .../physical_optimizer/topk_aggregation.rs | 14 +++++----- 4 files changed, 35 insertions(+), 35 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index ba6f7d0439c2..095590fe03f6 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -309,7 +309,7 @@ fn adjust_input_keys_ordering( return reorder_partitioned_join_keys( requirements, on, - vec![], + &[], &join_constructor, ) .map(Transformed::yes); @@ -373,7 +373,7 @@ fn adjust_input_keys_ordering( return reorder_partitioned_join_keys( requirements, on, - sort_options.clone(), + sort_options, &join_constructor, ) .map(Transformed::yes); @@ -421,7 +421,7 @@ fn adjust_input_keys_ordering( fn reorder_partitioned_join_keys( mut join_plan: PlanWithKeyRequirements, on: &[(PhysicalExprRef, PhysicalExprRef)], - sort_options: Vec, + sort_options: &[SortOptions], join_constructor: &F, ) -> Result where diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index b3f3f90154d0..e09332e8c36a 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -631,7 +631,7 @@ fn try_pushdown_through_hash_join( if !join_allows_pushdown( &projection_as_columns, - hash_join.schema(), + &hash_join.schema(), far_right_left_col_ind, far_left_right_col_ind, ) { @@ -662,7 +662,7 @@ fn try_pushdown_through_hash_join( }; let (new_left, new_right) = new_join_children( - projection_as_columns, + &projection_as_columns, far_right_left_col_ind, far_left_right_col_ind, hash_join.left(), @@ -700,7 +700,7 @@ fn try_swapping_with_cross_join( if !join_allows_pushdown( &projection_as_columns, - cross_join.schema(), + &cross_join.schema(), far_right_left_col_ind, far_left_right_col_ind, ) { @@ -708,7 +708,7 @@ fn try_swapping_with_cross_join( } let (new_left, new_right) = new_join_children( - projection_as_columns, + &projection_as_columns, far_right_left_col_ind, far_left_right_col_ind, cross_join.left(), @@ -740,7 +740,7 @@ fn try_swapping_with_nested_loop_join( if !join_allows_pushdown( &projection_as_columns, - nl_join.schema(), + &nl_join.schema(), far_right_left_col_ind, far_left_right_col_ind, ) { @@ -762,7 +762,7 @@ fn try_swapping_with_nested_loop_join( }; let (new_left, new_right) = new_join_children( - projection_as_columns, + &projection_as_columns, far_right_left_col_ind, far_left_right_col_ind, nl_join.left(), @@ -796,7 +796,7 @@ fn try_swapping_with_sort_merge_join( if !join_allows_pushdown( &projection_as_columns, - sm_join.schema(), + &sm_join.schema(), far_right_left_col_ind, far_left_right_col_ind, ) { @@ -813,7 +813,7 @@ fn try_swapping_with_sort_merge_join( }; let (new_left, new_right) = new_join_children( - projection_as_columns, + &projection_as_columns, far_right_left_col_ind, far_left_right_col_ind, sm_join.children()[0], @@ -850,7 +850,7 @@ fn try_swapping_with_sym_hash_join( if !join_allows_pushdown( &projection_as_columns, - sym_join.schema(), + &sym_join.schema(), far_right_left_col_ind, far_left_right_col_ind, ) { @@ -881,7 +881,7 @@ fn try_swapping_with_sym_hash_join( }; let (new_left, new_right) = new_join_children( - projection_as_columns, + &projection_as_columns, far_right_left_col_ind, far_left_right_col_ind, sym_join.left(), @@ -1243,7 +1243,7 @@ fn new_indices_for_join_filter( /// - Left or right table is not lost after the projection. fn join_allows_pushdown( projection_as_columns: &[(Column, String)], - join_schema: SchemaRef, + join_schema: &SchemaRef, far_right_left_col_ind: i32, far_left_right_col_ind: i32, ) -> bool { @@ -1260,7 +1260,7 @@ fn join_allows_pushdown( /// this function constructs the new [`ProjectionExec`]s that will come on top /// of the original children of the join. fn new_join_children( - projection_as_columns: Vec<(Column, String)>, + projection_as_columns: &[(Column, String)], far_right_left_col_ind: i32, far_left_right_col_ind: i32, left_child: &Arc, diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 9ab6802d18f1..41059ef7b1ef 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -165,7 +165,7 @@ fn pushdown_requirement_to_children( if is_window(plan) { let required_input_ordering = plan.required_input_ordering(); let request_child = required_input_ordering[0].as_deref().unwrap_or(&[]); - let child_plan = plan.children().swap_remove(0).clone(); + 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()); @@ -225,7 +225,7 @@ fn pushdown_requirement_to_children( Some(JoinSide::Left) => try_pushdown_requirements_to_join( smj, parent_required, - parent_required_expr, + &parent_required_expr, JoinSide::Left, ), Some(JoinSide::Right) => { @@ -238,7 +238,7 @@ fn pushdown_requirement_to_children( try_pushdown_requirements_to_join( smj, parent_required, - new_right_required_expr, + &new_right_required_expr, JoinSide::Right, ) } @@ -321,7 +321,7 @@ fn pushdown_would_violate_requirements( fn determine_children_requirement( parent_required: LexRequirementRef, request_child: LexRequirementRef, - child_plan: Arc, + child_plan: &Arc, ) -> RequirementsCompatibility { if child_plan .equivalence_properties() @@ -344,7 +344,7 @@ fn determine_children_requirement( fn try_pushdown_requirements_to_join( smj: &SortMergeJoinExec, parent_required: LexRequirementRef, - sort_expr: Vec, + sort_expr: &[PhysicalSortExpr], push_side: JoinSide, ) -> Result>>> { let left_eq_properties = smj.left().equivalence_properties(); @@ -356,25 +356,27 @@ fn try_pushdown_requirements_to_join( let right_ordering = smj.right().output_ordering().unwrap_or(&[]); let (new_left_ordering, new_right_ordering) = match push_side { JoinSide::Left => { - let left_eq_properties = - left_eq_properties.clone().with_reorder(sort_expr.clone()); + let left_eq_properties = left_eq_properties + .clone() + .with_reorder(Vec::from(sort_expr)); if left_eq_properties .ordering_satisfy_requirement(&left_requirement.unwrap_or_default()) { // After re-ordering requirement is still satisfied - (sort_expr.as_slice(), right_ordering) + (sort_expr, right_ordering) } else { return Ok(None); } } JoinSide::Right => { - let right_eq_properties = - right_eq_properties.clone().with_reorder(sort_expr.clone()); + let right_eq_properties = right_eq_properties + .clone() + .with_reorder(Vec::from(sort_expr)); if right_eq_properties .ordering_satisfy_requirement(&right_requirement.unwrap_or_default()) { // After re-ordering requirement is still satisfied - (left_ordering, sort_expr.as_slice()) + (left_ordering, sort_expr) } else { return Ok(None); } @@ -397,7 +399,7 @@ fn try_pushdown_requirements_to_join( let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); Ok(should_pushdown.then(|| { let mut required_input_ordering = smj.required_input_ordering(); - let new_req = Some(PhysicalSortRequirement::from_sort_exprs(&sort_expr)); + let new_req = Some(PhysicalSortRequirement::from_sort_exprs(sort_expr)); match push_side { JoinSide::Left => { required_input_ordering[0] = new_req; diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/core/src/physical_optimizer/topk_aggregation.rs index 82cf44ad7796..b2c8f640fda0 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/topk_aggregation.rs @@ -84,7 +84,7 @@ impl TopKAggregation { Some(Arc::new(new_aggr)) } - fn transform_sort(plan: Arc) -> Option> { + fn transform_sort(plan: &Arc) -> Option> { let sort = plan.as_any().downcast_ref::()?; let children = sort.children(); @@ -142,13 +142,11 @@ impl PhysicalOptimizerRule for TopKAggregation { ) -> Result> { if config.optimizer.enable_topk_aggregation { plan.transform_down(|plan| { - Ok( - if let Some(plan) = TopKAggregation::transform_sort(plan.clone()) { - Transformed::yes(plan) - } else { - Transformed::no(plan) - }, - ) + Ok(if let Some(plan) = TopKAggregation::transform_sort(&plan) { + Transformed::yes(plan) + } else { + Transformed::no(plan) + }) }) .data() } else { From ff7d76d8fe5527101557327cf0047f1ef72ee893 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Sun, 1 Sep 2024 15:00:27 +0200 Subject: [PATCH 42/75] Remove FileScanConfig::repartition_file_groups function (#12242) Deprecated since 33.0.0. --- .../physical_plan/file_scan_config.rs | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 3ea467539adc..9f674185694d 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -22,9 +22,7 @@ use std::{ borrow::Cow, collections::HashMap, fmt::Debug, marker::PhantomData, sync::Arc, vec, }; -use super::{ - get_projected_output_ordering, statistics::MinMaxStatistics, FileGroupPartitioner, -}; +use super::{get_projected_output_ordering, statistics::MinMaxStatistics}; use crate::datasource::{listing::PartitionedFile, object_store::ObjectStoreUrl}; use crate::{error::Result, scalar::ScalarValue}; @@ -296,19 +294,6 @@ impl FileScanConfig { }) } - #[allow(missing_docs)] - #[deprecated(since = "33.0.0", note = "Use SessionContext::new_with_config")] - pub fn repartition_file_groups( - file_groups: Vec>, - target_partitions: usize, - repartition_file_min_size: usize, - ) -> Option>> { - FileGroupPartitioner::new() - .with_target_partitions(target_partitions) - .with_repartition_file_min_size(repartition_file_min_size) - .repartition_file_groups(&file_groups) - } - /// Attempts to do a bin-packing on files into file groups, such that any two files /// in a file group are ordered and non-overlapping with respect to their statistics. /// It will produce the smallest number of file groups possible. From 780cccb5201f9c37eba647419c63a2a55625adfc Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 1 Sep 2024 09:16:14 -0400 Subject: [PATCH 43/75] Minor: Improve ExecutionMode documentation (#12214) * Minor: Improve ExecutionMode documentation * Update datafusion/physical-plan/src/execution_plan.rs Co-authored-by: Mehmet Ozan Kabak * Update datafusion/physical-plan/src/execution_plan.rs Co-authored-by: Mehmet Ozan Kabak --------- Co-authored-by: Mehmet Ozan Kabak --- .../physical-plan/src/execution_plan.rs | 45 +++++++++++++------ 1 file changed, 32 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index c1c66f6d3923..53ae59f7072f 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -482,24 +482,43 @@ impl ExecutionPlanProperties for &dyn ExecutionPlan { } } -/// Describes the execution mode of an operator's resulting stream with respect -/// to its size and behavior. There are three possible execution modes: `Bounded`, -/// `Unbounded` and `PipelineBreaking`. +/// Describes the execution mode of the result of calling +/// [`ExecutionPlan::execute`] with respect to its size and behavior. +/// +/// The mode of the execution plan is determined by the mode of its input +/// execution plans and the details of the operator itself. For example, a +/// `FilterExec` operator will have the same execution mode as its input, but a +/// `SortExec` operator may have a different execution mode than its input, +/// depending on how the input stream is sorted. +/// +/// There are three possible execution modes: `Bounded`, `Unbounded` and +/// `PipelineBreaking`. #[derive(Clone, Copy, PartialEq, Debug)] pub enum ExecutionMode { - /// Represents the mode where generated stream is bounded, e.g. finite. + /// The stream is bounded / finite. + /// + /// In this case the stream will eventually return `None` to indicate that + /// there are no more records to process. Bounded, - /// Represents the mode where generated stream is unbounded, e.g. infinite. - /// Even though the operator generates an unbounded stream of results, it - /// works with bounded memory and execution can still continue successfully. + /// The stream is unbounded / infinite. /// - /// The stream that results from calling `execute` on an `ExecutionPlan` that is `Unbounded` - /// will never be done (return `None`), except in case of error. + /// In this case, the stream will never be done (never return `None`), + /// except in case of error. + /// + /// This mode is often used in "Steaming" use cases where data is + /// incrementally processed as it arrives. + /// + /// Note that even though the operator generates an unbounded stream of + /// results, it can execute with bounded memory and incrementally produces + /// output. Unbounded, - /// Represents the mode where some of the operator's input stream(s) are - /// unbounded; however, the operator cannot generate streaming results from - /// these streaming inputs. In this case, the execution mode will be pipeline - /// breaking, e.g. the operator requires unbounded memory to generate results. + /// Some of the operator's input stream(s) are unbounded, but the operator + /// cannot generate streaming results from these streaming inputs. + /// + /// In this case, the execution mode will be pipeline breaking, e.g. the + /// operator requires unbounded memory to generate results. This + /// information is used by the planner when performing sanity checks + /// on plans processings unbounded data sources. PipelineBreaking, } From d86b7f968d55a25326d56d80f0cdd17d6a81c748 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 2 Sep 2024 03:31:19 +0200 Subject: [PATCH 44/75] Avoid unnecessary move when setting SessionConfig (#12260) --- datafusion/common/src/config.rs | 2 +- datafusion/core/src/dataframe/mod.rs | 8 +++++--- datafusion/core/src/dataframe/parquet.rs | 2 +- datafusion/core/src/datasource/listing/table.rs | 4 ++-- datafusion/core/src/execution/context/parquet.rs | 2 +- datafusion/execution/src/config.rs | 6 +++--- datafusion/physical-plan/src/aggregates/mod.rs | 8 ++++---- 7 files changed, 17 insertions(+), 15 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 37d26c6f00c4..4a0c61910ba4 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -781,7 +781,7 @@ impl ConfigOptions { /// /// Only the built-in configurations will be extracted from the hash map /// and other key value pairs will be ignored. - pub fn from_string_hash_map(settings: HashMap) -> Result { + pub fn from_string_hash_map(settings: &HashMap) -> Result { struct Visitor(Vec); impl Visit for Visitor { diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 5dbeb535a546..b8c0bd9d744c 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -3277,7 +3277,7 @@ mod tests { #[tokio::test] async fn with_column_renamed_case_sensitive() -> Result<()> { let config = - SessionConfig::from_string_hash_map(std::collections::HashMap::from([( + SessionConfig::from_string_hash_map(&std::collections::HashMap::from([( "datafusion.sql_parser.enable_ident_normalization".to_owned(), "false".to_owned(), )]))?; @@ -3713,8 +3713,10 @@ mod tests { // Test issue: https://github.com/apache/datafusion/issues/12065 #[tokio::test] async fn filtered_aggr_with_param_values() -> Result<()> { - let cfg = SessionConfig::new() - .set("datafusion.sql_parser.dialect", "PostgreSQL".into()); + let cfg = SessionConfig::new().set( + "datafusion.sql_parser.dialect", + &ScalarValue::from("PostgreSQL"), + ); let ctx = SessionContext::new_with_config(cfg); register_aggregate_csv(&ctx, "table1").await?; diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index 2a23f045f3b2..66974e37f453 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -190,7 +190,7 @@ mod tests { // This test verifies writing a parquet file with small rg size // relative to datafusion.execution.batch_size does not panic let ctx = SessionContext::new_with_config(SessionConfig::from_string_hash_map( - HashMap::from_iter( + &HashMap::from_iter( [("datafusion.execution.batch_size", "10")] .iter() .map(|(s1, s2)| (s1.to_string(), s2.to_string())), diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 1f5fa738b253..35286612a887 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1781,7 +1781,7 @@ mod tests { // Create the initial context, schema, and batch. let session_ctx = match session_config_map { Some(cfg) => { - let config = SessionConfig::from_string_hash_map(cfg)?; + let config = SessionConfig::from_string_hash_map(&cfg)?; SessionContext::new_with_config(config) } None => SessionContext::new(), @@ -1979,7 +1979,7 @@ mod tests { // Create the initial context let session_ctx = match session_config_map { Some(cfg) => { - let config = SessionConfig::from_string_hash_map(cfg)?; + let config = SessionConfig::from_string_hash_map(&cfg)?; SessionContext::new_with_config(config) } None => SessionContext::new(), diff --git a/datafusion/core/src/execution/context/parquet.rs b/datafusion/core/src/execution/context/parquet.rs index fef20df6e69d..1d83c968c1a8 100644 --- a/datafusion/core/src/execution/context/parquet.rs +++ b/datafusion/core/src/execution/context/parquet.rs @@ -106,7 +106,7 @@ mod tests { #[tokio::test] async fn read_with_glob_path_issue_2465() -> Result<()> { let config = - SessionConfig::from_string_hash_map(std::collections::HashMap::from([( + SessionConfig::from_string_hash_map(&std::collections::HashMap::from([( "datafusion.execution.listing_table_ignore_subdirectory".to_owned(), "false".to_owned(), )]))?; diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index 7f872db049a5..c2b1f9e74128 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -48,7 +48,7 @@ use datafusion_common::{ /// use datafusion_common::ScalarValue; /// /// let config = SessionConfig::new() -/// .set("datafusion.execution.batch_size", ScalarValue::UInt64(Some(1234))) +/// .set("datafusion.execution.batch_size", &ScalarValue::UInt64(Some(1234))) /// .set_bool("datafusion.execution.parquet.pushdown_filters", true); /// /// assert_eq!(config.batch_size(), 1234); @@ -123,7 +123,7 @@ impl SessionConfig { } /// Create new ConfigOptions struct, taking values from a string hash map. - pub fn from_string_hash_map(settings: HashMap) -> Result { + pub fn from_string_hash_map(settings: &HashMap) -> Result { Ok(ConfigOptions::from_string_hash_map(settings)?.into()) } @@ -157,7 +157,7 @@ impl SessionConfig { } /// Set a configuration option - pub fn set(self, key: &str, value: ScalarValue) -> Self { + pub fn set(self, key: &str, value: &ScalarValue) -> Self { self.set_str(key, &value.to_string()) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2f974f1ef4e0..161ffb6c6541 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2373,11 +2373,11 @@ mod tests { let mut session_config = SessionConfig::default(); session_config = session_config.set( "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", - ScalarValue::Int64(Some(2)), + &ScalarValue::Int64(Some(2)), ); session_config = session_config.set( "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", - ScalarValue::Float64(Some(0.1)), + &ScalarValue::Float64(Some(0.1)), ); let ctx = TaskContext::default().with_session_config(session_config); @@ -2462,11 +2462,11 @@ mod tests { let mut session_config = SessionConfig::default(); session_config = session_config.set( "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", - ScalarValue::Int64(Some(5)), + &ScalarValue::Int64(Some(5)), ); session_config = session_config.set( "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", - ScalarValue::Float64(Some(0.1)), + &ScalarValue::Float64(Some(0.1)), ); let ctx = TaskContext::default().with_session_config(session_config); From dd3208943d728d845497d6a12ce4c0eacc061dcd Mon Sep 17 00:00:00 2001 From: JC <1950050+jc4x4@users.noreply.github.com> Date: Mon, 2 Sep 2024 09:41:21 +0800 Subject: [PATCH 45/75] Remove unused `AggregateOptions` struct and `scalar_update_factor` config setting (#12241) * Remove scalar_update_factor from AggregateOptions Keeping the AggregateOptions struct in case we need it later. https://github.com/apache/datafusion/issues/11998 * Remove AggregateOption; update configs.md * Run prettier * Fix cargo doc --- datafusion/common/src/config.rs | 24 --- datafusion/execution/src/config.rs | 14 -- .../test_files/information_schema.slt | 2 - docs/source/user-guide/configs.md | 173 +++++++++--------- 4 files changed, 86 insertions(+), 127 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 4a0c61910ba4..7c247103f6e7 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -261,9 +261,6 @@ config_namespace! { /// Parquet options pub parquet: ParquetOptions, default = Default::default() - /// Aggregate options - pub aggregate: AggregateOptions, default = Default::default() - /// Fan-out during initial physical planning. /// /// This is mostly use to plan `UNION` children in parallel. @@ -493,27 +490,6 @@ config_namespace! { } } -config_namespace! { - /// Options related to aggregate execution - /// - /// See also: [`SessionConfig`] - /// - /// [`SessionConfig`]: https://docs.rs/datafusion/latest/datafusion/prelude/struct.SessionConfig.html - pub struct AggregateOptions { - /// Specifies the threshold for using `ScalarValue`s to update - /// accumulators during high-cardinality aggregations for each input batch. - /// - /// The aggregation is considered high-cardinality if the number of affected groups - /// is greater than or equal to `batch_size / scalar_update_factor`. In such cases, - /// `ScalarValue`s are utilized for updating accumulators, rather than the default - /// batch-slice approach. This can lead to performance improvements. - /// - /// By adjusting the `scalar_update_factor`, you can balance the trade-off between - /// more efficient accumulator updates and the number of groups affected. - pub scalar_update_factor: usize, default = 10 - } -} - config_namespace! { /// Options related to query optimization /// diff --git a/datafusion/execution/src/config.rs b/datafusion/execution/src/config.rs index c2b1f9e74128..cede75d21ca4 100644 --- a/datafusion/execution/src/config.rs +++ b/datafusion/execution/src/config.rs @@ -78,7 +78,6 @@ use datafusion_common::{ /// | --------- | ------------- | /// | `datafusion.catalog` | [CatalogOptions][datafusion_common::config::CatalogOptions] | /// | `datafusion.execution` | [ExecutionOptions][datafusion_common::config::ExecutionOptions] | -/// | `datafusion.execution.aggregate` | [AggregateOptions][datafusion_common::config::AggregateOptions] | /// | `datafusion.execution.parquet` | [ParquetOptions][datafusion_common::config::ParquetOptions] | /// | `datafusion.optimizer` | [OptimizerOptions][datafusion_common::config::OptimizerOptions] | /// | `datafusion.sql_parser` | [SqlParserOptions][datafusion_common::config::SqlParserOptions] | @@ -383,19 +382,6 @@ impl SessionConfig { self.options.execution.batch_size } - /// Get the currently configured scalar_update_factor for aggregate - pub fn agg_scalar_update_factor(&self) -> usize { - self.options.execution.aggregate.scalar_update_factor - } - - /// Customize scalar_update_factor for aggregate - pub fn with_agg_scalar_update_factor(mut self, n: usize) -> Self { - // scalar update factor must be greater than zero - assert!(n > 0); - self.options.execution.aggregate.scalar_update_factor = n; - self - } - /// Enables or disables the coalescence of small batches into larger batches pub fn with_coalesce_batches(mut self, enabled: bool) -> Self { self.options.execution.coalesce_batches = enabled; diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index efd017a90bc4..f797a7a6539d 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -169,7 +169,6 @@ datafusion.catalog.has_header true datafusion.catalog.information_schema true datafusion.catalog.location NULL datafusion.catalog.newlines_in_values false -datafusion.execution.aggregate.scalar_update_factor 10 datafusion.execution.batch_size 8192 datafusion.execution.coalesce_batches true datafusion.execution.collect_statistics false @@ -259,7 +258,6 @@ datafusion.catalog.has_header true Default value for `format.has_header` for `CR datafusion.catalog.information_schema true Should DataFusion provide access to `information_schema` virtual tables for displaying schema information datafusion.catalog.location NULL Location scanned to load tables for `default` schema datafusion.catalog.newlines_in_values false Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. -datafusion.execution.aggregate.scalar_update_factor 10 Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. datafusion.execution.batch_size 8192 Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption datafusion.execution.coalesce_batches true When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting datafusion.execution.collect_statistics false Should DataFusion collect statistics after listing files diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 4255307781b6..8514fb1fbd93 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,90 +35,89 @@ Values are parsed according to the [same rules used in casts from Utf8](https:// If the value in the environment variable cannot be cast to the type of the configuration option, the default value will be used instead and a warning emitted. Environment variables are read during `SessionConfig` initialisation so they must be set beforehand and will not affect running sessions. -| key | default | description | -| ----------------------------------------------------------------------- | ------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | -| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | -| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | -| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | -| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | -| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | -| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | -| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | NULL | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | -| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | -| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | -| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | -| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | -| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | -| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | -| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.max_statistics_size | 4096 | (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | -| datafusion.execution.parquet.created_by | datafusion version 41.0.0 | (writing) Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | -| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | -| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_on_read | true | (writing) Use any available bloom filters when reading parquet files | -| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | -| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | -| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | -| datafusion.execution.aggregate.scalar_update_factor | 10 | Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. | -| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | -| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | -| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | -| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | -| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | -| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | -| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | -| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | -| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | -| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | -| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | -| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | -| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | -| datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for this decision. Setting this flag to `true` will likely produce better plans. if the source of statistics is accurate. We plan to make this the default in the future. | -| datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | -| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | -| datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | -| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | -| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | -| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | -| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | -| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | -| datafusion.sql_parser.enable_options_value_normalization | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | -| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, and Ansi. | -| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | +| key | default | description | +| ----------------------------------------------------------------------- | ------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | +| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | +| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | +| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | +| datafusion.execution.collect_statistics | false | Should DataFusion collect statistics after listing files | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | +| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | +| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | +| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | NULL | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | +| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | +| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | +| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | +| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | +| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | +| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | +| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.max_statistics_size | 4096 | (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | +| datafusion.execution.parquet.created_by | datafusion version 41.0.0 | (writing) Sets "created by" property | +| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | +| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | +| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_on_read | true | (writing) Use any available bloom filters when reading parquet files | +| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | +| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | +| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | +| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | +| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | +| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | +| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | +| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | +| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | +| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | +| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | +| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | +| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | +| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | +| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | +| datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for this decision. Setting this flag to `true` will likely produce better plans. if the source of statistics is accurate. We plan to make this the default in the future. | +| datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | +| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | +| datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | +| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | +| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | +| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | +| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | +| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | +| datafusion.sql_parser.enable_options_value_normalization | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | +| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, and Ansi. | +| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | From 53de592381b8585f7d9de82f2ac1eb58b1524e44 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Mon, 2 Sep 2024 16:48:24 +0800 Subject: [PATCH 46/75] Make group expressions nullable more accurate (#12256) * Make group expressions nullable more accurate * Add test --- .../physical-plan/src/aggregates/mod.rs | 69 ++++++++++++++++--- 1 file changed, 61 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 161ffb6c6541..e66a57fd2ee5 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -164,9 +164,17 @@ impl PhysicalGroupBy { } } - /// Returns true if this GROUP BY contains NULL expressions - pub fn contains_null(&self) -> bool { - self.groups.iter().flatten().any(|is_null| *is_null) + /// Calculate GROUP BY expressions nullable + pub fn exprs_nullable(&self) -> Vec { + let mut exprs_nullable = vec![false; self.expr.len()]; + for group in self.groups.iter() { + group.iter().enumerate().for_each(|(index, is_null)| { + if *is_null { + exprs_nullable[index] = true; + } + }) + } + exprs_nullable } /// Returns the group expressions @@ -278,7 +286,7 @@ pub struct AggregateExec { } impl AggregateExec { - /// Function used in `ConvertFirstLast` optimizer rule, + /// Function used in `OptimizeAggregateOrder` optimizer rule, /// where we need parts of the new value, others cloned from the old one /// Rewrites aggregate exec with new aggregate expressions. pub fn with_new_aggr_exprs( @@ -319,7 +327,7 @@ impl AggregateExec { &input.schema(), &group_by.expr, &aggr_expr, - group_by.contains_null(), + group_by.exprs_nullable(), mode, )?; @@ -793,18 +801,18 @@ fn create_schema( input_schema: &Schema, group_expr: &[(Arc, String)], aggr_expr: &[Arc], - contains_null_expr: bool, + group_expr_nullable: Vec, mode: AggregateMode, ) -> Result { let mut fields = Vec::with_capacity(group_expr.len() + aggr_expr.len()); - for (expr, name) in group_expr { + for (index, (expr, name)) in group_expr.iter().enumerate() { fields.push(Field::new( name, expr.data_type(input_schema)?, // In cases where we have multiple grouping sets, we will use NULL expressions in // order to align the grouping sets. So the field must be nullable even if the underlying // schema field is not. - contains_null_expr || expr.nullable(input_schema)?, + group_expr_nullable[index] || expr.nullable(input_schema)?, )) } @@ -2489,4 +2497,49 @@ mod tests { Ok(()) } + + #[test] + fn group_exprs_nullable() -> Result<()> { + let input_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Float32, false), + Field::new("b", DataType::Float32, false), + ])); + + let aggr_expr = + vec![ + AggregateExprBuilder::new(count_udaf(), vec![col("a", &input_schema)?]) + .schema(Arc::clone(&input_schema)) + .alias("COUNT(a)") + .build()?, + ]; + + let grouping_set = PhysicalGroupBy { + expr: vec![ + (col("a", &input_schema)?, "a".to_string()), + (col("b", &input_schema)?, "b".to_string()), + ], + null_expr: vec![ + (lit(ScalarValue::Float32(None)), "a".to_string()), + (lit(ScalarValue::Float32(None)), "b".to_string()), + ], + groups: vec![ + vec![false, true], // (a, NULL) + vec![false, false], // (a,b) + ], + }; + let aggr_schema = create_schema( + &input_schema, + &grouping_set.expr, + &aggr_expr, + grouping_set.exprs_nullable(), + AggregateMode::Final, + )?; + let expected_schema = Schema::new(vec![ + Field::new("a", DataType::Float32, false), + Field::new("b", DataType::Float32, true), + Field::new("COUNT(a)", DataType::Int64, false), + ]); + assert_eq!(aggr_schema, expected_schema); + Ok(()) + } } From 447cb02a3a0a26cf696e7dffddb91b85822c2e02 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 2 Sep 2024 11:17:15 +0200 Subject: [PATCH 47/75] Avoid redundant pass-by-value in optimizer (#12262) --- .../core/src/execution/session_state.rs | 4 +-- datafusion/expr/src/utils.rs | 2 +- .../src/analyzer/expand_wildcard_rule.rs | 6 ++-- .../src/analyzer/function_rewrite.rs | 2 +- datafusion/optimizer/src/analyzer/subquery.rs | 3 +- .../optimizer/src/analyzer/type_coercion.rs | 32 +++++++++---------- .../optimizer/src/common_subexpr_eliminate.rs | 6 ++-- datafusion/optimizer/src/decorrelate.rs | 14 ++++---- .../src/decorrelate_predicate_subquery.rs | 2 +- .../optimizer/src/eliminate_cross_join.rs | 2 +- datafusion/optimizer/src/join_key_set.rs | 4 +-- .../optimizer/src/optimize_projections/mod.rs | 4 +-- datafusion/optimizer/src/optimizer.rs | 6 ++-- .../optimizer/src/scalar_subquery_to_join.rs | 8 ++--- .../simplify_expressions/expr_simplifier.rs | 20 ++++++------ .../simplify_expressions/simplify_exprs.rs | 2 +- .../src/simplify_expressions/utils.rs | 4 +-- .../src/unwrap_cast_in_comparison.rs | 20 ++++++------ datafusion/optimizer/src/utils.rs | 6 ++-- 19 files changed, 73 insertions(+), 74 deletions(-) diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 88a90e1e1d09..90f175b59385 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -1740,8 +1740,8 @@ impl OptimizerConfig for SessionState { self.execution_props.query_execution_start_time } - fn alias_generator(&self) -> Arc { - self.execution_props.alias_generator.clone() + fn alias_generator(&self) -> &Arc { + &self.execution_props.alias_generator } fn options(&self) -> &ConfigOptions { diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 9b64bca8121e..1d8eb9445eda 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -1305,7 +1305,7 @@ pub fn only_or_err(slice: &[T]) -> Result<&T> { } /// merge inputs schema into a single schema. -pub fn merge_schema(inputs: Vec<&LogicalPlan>) -> DFSchema { +pub fn merge_schema(inputs: &[&LogicalPlan]) -> DFSchema { if inputs.len() == 1 { inputs[0].schema().as_ref().clone() } else { diff --git a/datafusion/optimizer/src/analyzer/expand_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/expand_wildcard_rule.rs index dd422f7aab95..b2de541a34c7 100644 --- a/datafusion/optimizer/src/analyzer/expand_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/expand_wildcard_rule.rs @@ -84,7 +84,7 @@ fn expand_exprlist(input: &LogicalPlan, expr: Vec) -> Result> { // If there is a REPLACE statement, replace that column with the given // replace expression. Column name remains the same. let replaced = if let Some(replace) = options.replace { - replace_columns(expanded, replace)? + replace_columns(expanded, &replace)? } else { expanded }; @@ -95,7 +95,7 @@ fn expand_exprlist(input: &LogicalPlan, expr: Vec) -> Result> { // If there is a REPLACE statement, replace that column with the given // replace expression. Column name remains the same. let replaced = if let Some(replace) = options.replace { - replace_columns(expanded, replace)? + replace_columns(expanded, &replace)? } else { expanded }; @@ -139,7 +139,7 @@ fn expand_exprlist(input: &LogicalPlan, expr: Vec) -> Result> { /// Multiple REPLACEs are also possible with comma separations. fn replace_columns( mut exprs: Vec, - replace: PlannedReplaceSelectItem, + replace: &PlannedReplaceSelectItem, ) -> Result> { for expr in exprs.iter_mut() { if let Expr::Column(Column { name, .. }) = expr { diff --git a/datafusion/optimizer/src/analyzer/function_rewrite.rs b/datafusion/optimizer/src/analyzer/function_rewrite.rs index 098c934bf7e1..94f5657b899b 100644 --- a/datafusion/optimizer/src/analyzer/function_rewrite.rs +++ b/datafusion/optimizer/src/analyzer/function_rewrite.rs @@ -48,7 +48,7 @@ impl ApplyFunctionRewrites { ) -> Result> { // get schema representing all available input fields. This is used for data type // resolution only, so order does not matter here - let mut schema = merge_schema(plan.inputs()); + let mut schema = merge_schema(&plan.inputs()); if let LogicalPlan::TableScan(ts) = &plan { let source_schema = DFSchema::try_from_qualified_schema( diff --git a/datafusion/optimizer/src/analyzer/subquery.rs b/datafusion/optimizer/src/analyzer/subquery.rs index 78be48a189f7..996dc238853e 100644 --- a/datafusion/optimizer/src/analyzer/subquery.rs +++ b/datafusion/optimizer/src/analyzer/subquery.rs @@ -16,7 +16,6 @@ // under the License. use std::ops::Deref; -use std::sync::Arc; use crate::analyzer::check_plan; use crate::utils::collect_subquery_cols; @@ -246,7 +245,7 @@ fn check_aggregation_in_scalar_subquery( if !agg.group_expr.is_empty() { let correlated_exprs = get_correlated_expressions(inner_plan)?; let inner_subquery_cols = - collect_subquery_cols(&correlated_exprs, Arc::clone(agg.input.schema()))?; + collect_subquery_cols(&correlated_exprs, agg.input.schema())?; let mut group_columns = agg .group_expr .iter() diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 61ff4b4fd5a8..284650c3d64e 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -91,7 +91,7 @@ fn analyze_internal( ) -> Result> { // get schema representing all available input fields. This is used for data type // resolution only, so order does not matter here - let mut schema = merge_schema(plan.inputs()); + let mut schema = merge_schema(&plan.inputs()); if let LogicalPlan::TableScan(ts) = &plan { let source_schema = DFSchema::try_from_qualified_schema( @@ -544,12 +544,12 @@ fn coerce_scalar(target_type: &DataType, value: &ScalarValue) -> Result Result { - coerce_scalar(target_type, &value).or_else(|err| { + coerce_scalar(target_type, value).or_else(|err| { // If type coercion fails, check if the largest type in family works: if let Some(largest_type) = get_widest_type_in_family(target_type) { - coerce_scalar(largest_type, &value).map_or_else( + coerce_scalar(largest_type, value).map_or_else( |_| exec_err!("Cannot cast {value:?} to {target_type:?}"), |_| ScalarValue::try_from(target_type), ) @@ -578,11 +578,11 @@ fn coerce_frame_bound( ) -> Result { match bound { WindowFrameBound::Preceding(v) => { - coerce_scalar_range_aware(target_type, v).map(WindowFrameBound::Preceding) + coerce_scalar_range_aware(target_type, &v).map(WindowFrameBound::Preceding) } WindowFrameBound::CurrentRow => Ok(WindowFrameBound::CurrentRow), WindowFrameBound::Following(v) => { - coerce_scalar_range_aware(target_type, v).map(WindowFrameBound::Following) + coerce_scalar_range_aware(target_type, &v).map(WindowFrameBound::Following) } } } @@ -1459,26 +1459,26 @@ mod test { fn cast_helper( case: Case, - case_when_type: DataType, - then_else_type: DataType, + case_when_type: &DataType, + then_else_type: &DataType, schema: &DFSchemaRef, ) -> Case { let expr = case .expr - .map(|e| cast_if_not_same_type(e, &case_when_type, schema)); + .map(|e| cast_if_not_same_type(e, case_when_type, schema)); let when_then_expr = case .when_then_expr .into_iter() .map(|(when, then)| { ( - cast_if_not_same_type(when, &case_when_type, schema), - cast_if_not_same_type(then, &then_else_type, schema), + cast_if_not_same_type(when, case_when_type, schema), + cast_if_not_same_type(then, then_else_type, schema), ) }) .collect::>(); let else_expr = case .else_expr - .map(|e| cast_if_not_same_type(e, &then_else_type, schema)); + .map(|e| cast_if_not_same_type(e, then_else_type, schema)); Case { expr, @@ -1526,8 +1526,8 @@ mod test { let then_else_common_type = DataType::Utf8; let expected = cast_helper( case.clone(), - case_when_common_type, - then_else_common_type, + &case_when_common_type, + &then_else_common_type, &schema, ); let actual = coerce_case_expression(case, &schema)?; @@ -1546,8 +1546,8 @@ mod test { let then_else_common_type = DataType::Utf8; let expected = cast_helper( case.clone(), - case_when_common_type, - then_else_common_type, + &case_when_common_type, + &then_else_common_type, &schema, ); let actual = coerce_case_expression(case, &schema)?; diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 22e9d220d324..583c6cf50de3 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -229,7 +229,7 @@ impl CommonSubexprEliminate { fn rewrite_exprs_list<'n>( &self, exprs_list: Vec>, - arrays_list: Vec>>, + arrays_list: &[Vec>], expr_stats: &ExprStats<'n>, common_exprs: &mut CommonExprs<'n>, alias_generator: &AliasGenerator, @@ -284,10 +284,10 @@ impl CommonSubexprEliminate { // Must clone as Identifiers use references to original expressions so we have // to keep the original expressions intact. exprs_list.clone(), - id_arrays_list, + &id_arrays_list, &expr_stats, &mut common_exprs, - &config.alias_generator(), + config.alias_generator().as_ref(), )?; assert!(!common_exprs.is_empty()); diff --git a/datafusion/optimizer/src/decorrelate.rs b/datafusion/optimizer/src/decorrelate.rs index 4d0770ccbbfb..7f918c03e3ac 100644 --- a/datafusion/optimizer/src/decorrelate.rs +++ b/datafusion/optimizer/src/decorrelate.rs @@ -148,7 +148,7 @@ impl TreeNodeRewriter for PullUpCorrelatedExpr { } fn f_up(&mut self, plan: LogicalPlan) -> Result> { - let subquery_schema = Arc::clone(plan.schema()); + let subquery_schema = plan.schema(); match &plan { LogicalPlan::Filter(plan_filter) => { let subquery_filter_exprs = split_conjunction(&plan_filter.predicate); @@ -231,7 +231,7 @@ impl TreeNodeRewriter for PullUpCorrelatedExpr { { proj_exprs_evaluation_result_on_empty_batch( &projection.expr, - Arc::clone(projection.input.schema()), + projection.input.schema(), expr_result_map, &mut expr_result_map_for_count_bug, )?; @@ -277,7 +277,7 @@ impl TreeNodeRewriter for PullUpCorrelatedExpr { { agg_exprs_evaluation_result_on_empty_batch( &aggregate.aggr_expr, - Arc::clone(aggregate.input.schema()), + aggregate.input.schema(), &mut expr_result_map_for_count_bug, )?; if !expr_result_map_for_count_bug.is_empty() { @@ -423,7 +423,7 @@ fn remove_duplicated_filter(filters: Vec, in_predicate: &Expr) -> Vec Result<()> { for e in agg_expr.iter() { @@ -446,7 +446,7 @@ fn agg_exprs_evaluation_result_on_empty_batch( let result_expr = result_expr.unalias(); let props = ExecutionProps::new(); - let info = SimplifyContext::new(&props).with_schema(Arc::clone(&schema)); + let info = SimplifyContext::new(&props).with_schema(Arc::clone(schema)); let simplifier = ExprSimplifier::new(info); let result_expr = simplifier.simplify(result_expr)?; if matches!(result_expr, Expr::Literal(ScalarValue::Int64(_))) { @@ -459,7 +459,7 @@ fn agg_exprs_evaluation_result_on_empty_batch( fn proj_exprs_evaluation_result_on_empty_batch( proj_expr: &[Expr], - schema: DFSchemaRef, + schema: &DFSchemaRef, input_expr_result_map_for_count_bug: &ExprResultMap, expr_result_map_for_count_bug: &mut ExprResultMap, ) -> Result<()> { @@ -483,7 +483,7 @@ fn proj_exprs_evaluation_result_on_empty_batch( if result_expr.ne(expr) { let props = ExecutionProps::new(); - let info = SimplifyContext::new(&props).with_schema(Arc::clone(&schema)); + let info = SimplifyContext::new(&props).with_schema(Arc::clone(schema)); let simplifier = ExprSimplifier::new(info); let result_expr = simplifier.simplify(result_expr)?; let expr_name = match expr { diff --git a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs index f1cae1099a4d..9b1ada382b41 100644 --- a/datafusion/optimizer/src/decorrelate_predicate_subquery.rs +++ b/datafusion/optimizer/src/decorrelate_predicate_subquery.rs @@ -249,7 +249,7 @@ impl OptimizerRule for DecorrelatePredicateSubquery { fn build_join( query_info: &SubqueryInfo, left: &LogicalPlan, - alias: Arc, + alias: &Arc, ) -> Result> { let where_in_expr_opt = &query_info.where_in_expr; let in_predicate_opt = where_in_expr_opt diff --git a/datafusion/optimizer/src/eliminate_cross_join.rs b/datafusion/optimizer/src/eliminate_cross_join.rs index 20e6641e4d62..93df0dcfd500 100644 --- a/datafusion/optimizer/src/eliminate_cross_join.rs +++ b/datafusion/optimizer/src/eliminate_cross_join.rs @@ -378,7 +378,7 @@ fn extract_possible_join_keys(expr: &Expr, join_keys: &mut JoinKeySet) { extract_possible_join_keys(left, &mut left_join_keys); extract_possible_join_keys(right, &mut right_join_keys); - join_keys.insert_intersection(left_join_keys, right_join_keys) + join_keys.insert_intersection(&left_join_keys, &right_join_keys) } _ => (), }; diff --git a/datafusion/optimizer/src/join_key_set.rs b/datafusion/optimizer/src/join_key_set.rs index cd8ed382f069..c0eec78b183d 100644 --- a/datafusion/optimizer/src/join_key_set.rs +++ b/datafusion/optimizer/src/join_key_set.rs @@ -107,7 +107,7 @@ impl JoinKeySet { } /// Inserts any join keys that are common to both `s1` and `s2` into self - pub fn insert_intersection(&mut self, s1: JoinKeySet, s2: JoinKeySet) { + pub fn insert_intersection(&mut self, s1: &JoinKeySet, s2: &JoinKeySet) { // note can't use inner.intersection as we need to consider both (l, r) // and (r, l) in equality for (left, right) in s1.inner.iter() { @@ -234,7 +234,7 @@ mod test { let mut set = JoinKeySet::new(); // put something in there already set.insert(&col("x"), &col("y")); - set.insert_intersection(set1, set2); + set.insert_intersection(&set1, &set2); assert_contents( &set, diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index 35b0d07751ff..96772d3f2864 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -120,7 +120,7 @@ fn optimize_projections( match plan { LogicalPlan::Projection(proj) => { return merge_consecutive_projections(proj)?.transform_data(|proj| { - rewrite_projection_given_requirements(proj, config, indices) + rewrite_projection_given_requirements(proj, config, &indices) }) } LogicalPlan::Aggregate(aggregate) => { @@ -754,7 +754,7 @@ fn add_projection_on_top_if_helpful( fn rewrite_projection_given_requirements( proj: Projection, config: &dyn OptimizerConfig, - indices: RequiredIndicies, + indices: &RequiredIndicies, ) -> Result> { let Projection { expr, input, .. } = proj; diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 93923a4e1e74..2fc560cceeb4 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -126,7 +126,7 @@ pub trait OptimizerConfig { fn query_execution_start_time(&self) -> DateTime; /// Return alias generator used to generate unique aliases for subqueries - fn alias_generator(&self) -> Arc; + fn alias_generator(&self) -> &Arc; fn options(&self) -> &ConfigOptions; @@ -204,8 +204,8 @@ impl OptimizerConfig for OptimizerContext { self.query_execution_start_time } - fn alias_generator(&self) -> Arc { - Arc::clone(&self.alias_generator) + fn alias_generator(&self) -> &Arc { + &self.alias_generator } fn options(&self) -> &ConfigOptions { diff --git a/datafusion/optimizer/src/scalar_subquery_to_join.rs b/datafusion/optimizer/src/scalar_subquery_to_join.rs index c79180b79256..e5c8757405cf 100644 --- a/datafusion/optimizer/src/scalar_subquery_to_join.rs +++ b/datafusion/optimizer/src/scalar_subquery_to_join.rs @@ -54,7 +54,7 @@ impl ScalarSubqueryToJoin { fn extract_subquery_exprs( &self, predicate: &Expr, - alias_gen: Arc, + alias_gen: &Arc, ) -> Result<(Vec<(Subquery, String)>, Expr)> { let mut extract = ExtractScalarSubQuery { sub_query_info: vec![], @@ -223,12 +223,12 @@ fn contains_scalar_subquery(expr: &Expr) -> bool { .expect("Inner is always Ok") } -struct ExtractScalarSubQuery { +struct ExtractScalarSubQuery<'a> { sub_query_info: Vec<(Subquery, String)>, - alias_gen: Arc, + alias_gen: &'a Arc, } -impl TreeNodeRewriter for ExtractScalarSubQuery { +impl TreeNodeRewriter for ExtractScalarSubQuery<'_> { type Node = Expr; fn f_down(&mut self, expr: Expr) -> Result> { diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index f299d4542c36..fc3921d29615 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -730,7 +730,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { op: Eq, right, }) if is_bool_lit(&left) && info.is_boolean_type(&right)? => { - Transformed::yes(match as_bool_lit(*left)? { + Transformed::yes(match as_bool_lit(&left)? { Some(true) => *right, Some(false) => Expr::Not(right), None => lit_bool_null(), @@ -744,7 +744,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { op: Eq, right, }) if is_bool_lit(&right) && info.is_boolean_type(&left)? => { - Transformed::yes(match as_bool_lit(*right)? { + Transformed::yes(match as_bool_lit(&right)? { Some(true) => *left, Some(false) => Expr::Not(left), None => lit_bool_null(), @@ -761,7 +761,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { op: NotEq, right, }) if is_bool_lit(&left) && info.is_boolean_type(&right)? => { - Transformed::yes(match as_bool_lit(*left)? { + Transformed::yes(match as_bool_lit(&left)? { Some(true) => Expr::Not(right), Some(false) => *right, None => lit_bool_null(), @@ -775,7 +775,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { op: NotEq, right, }) if is_bool_lit(&right) && info.is_boolean_type(&left)? => { - Transformed::yes(match as_bool_lit(*right)? { + Transformed::yes(match as_bool_lit(&right)? { Some(true) => Expr::Not(left), Some(false) => *left, None => lit_bool_null(), @@ -1569,7 +1569,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { { match (*left, *right) { (Expr::InList(l1), Expr::InList(l2)) => { - return inlist_intersection(l1, l2, false).map(Transformed::yes); + return inlist_intersection(l1, &l2, false).map(Transformed::yes); } // Matched previously once _ => unreachable!(), @@ -1609,7 +1609,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { { match (*left, *right) { (Expr::InList(l1), Expr::InList(l2)) => { - return inlist_except(l1, l2).map(Transformed::yes); + return inlist_except(l1, &l2).map(Transformed::yes); } // Matched previously once _ => unreachable!(), @@ -1629,7 +1629,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { { match (*left, *right) { (Expr::InList(l1), Expr::InList(l2)) => { - return inlist_except(l2, l1).map(Transformed::yes); + return inlist_except(l2, &l1).map(Transformed::yes); } // Matched previously once _ => unreachable!(), @@ -1649,7 +1649,7 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { { match (*left, *right) { (Expr::InList(l1), Expr::InList(l2)) => { - return inlist_intersection(l1, l2, true).map(Transformed::yes); + return inlist_intersection(l1, &l2, true).map(Transformed::yes); } // Matched previously once _ => unreachable!(), @@ -1759,7 +1759,7 @@ fn inlist_union(mut l1: InList, l2: InList, negated: bool) -> Result { /// Return the intersection of two inlist expressions /// maintaining the order of the elements in the two lists -fn inlist_intersection(mut l1: InList, l2: InList, negated: bool) -> Result { +fn inlist_intersection(mut l1: InList, l2: &InList, negated: bool) -> Result { let l2_items = l2.list.iter().collect::>(); // remove all items from l1 that are not in l2 @@ -1775,7 +1775,7 @@ fn inlist_intersection(mut l1: InList, l2: InList, negated: bool) -> Result Result { +fn inlist_except(mut l1: InList, l2: &InList) -> Result { let l2_items = l2.list.iter().collect::>(); // keep only items from l1 that are not in l2 diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs index 1c39313ea132..877989c02d74 100644 --- a/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs +++ b/datafusion/optimizer/src/simplify_expressions/simplify_exprs.rs @@ -80,7 +80,7 @@ impl SimplifyExpressions { execution_props: &ExecutionProps, ) -> Result> { let schema = if !plan.inputs().is_empty() { - DFSchemaRef::new(merge_schema(plan.inputs())) + DFSchemaRef::new(merge_schema(&plan.inputs())) } else if let LogicalPlan::TableScan(scan) = &plan { // When predicates are pushed into a table scan, there is no input // schema to resolve predicates against, so it must be handled specially diff --git a/datafusion/optimizer/src/simplify_expressions/utils.rs b/datafusion/optimizer/src/simplify_expressions/utils.rs index 5da727cb5990..38bfc1a93403 100644 --- a/datafusion/optimizer/src/simplify_expressions/utils.rs +++ b/datafusion/optimizer/src/simplify_expressions/utils.rs @@ -221,9 +221,9 @@ pub fn is_negative_of(not_expr: &Expr, expr: &Expr) -> bool { /// returns the contained boolean value in `expr` as /// `Expr::Literal(ScalarValue::Boolean(v))`. -pub fn as_bool_lit(expr: Expr) -> Result> { +pub fn as_bool_lit(expr: &Expr) -> Result> { match expr { - Expr::Literal(ScalarValue::Boolean(v)) => Ok(v), + Expr::Literal(ScalarValue::Boolean(v)) => Ok(*v), _ => internal_err!("Expected boolean literal, got {expr:?}"), } } diff --git a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs index b17d69437cbe..6043f0d7c8b5 100644 --- a/datafusion/optimizer/src/unwrap_cast_in_comparison.rs +++ b/datafusion/optimizer/src/unwrap_cast_in_comparison.rs @@ -99,7 +99,7 @@ impl OptimizerRule for UnwrapCastInComparison { plan: LogicalPlan, _config: &dyn OptimizerConfig, ) -> Result> { - let mut schema = merge_schema(plan.inputs()); + let mut schema = merge_schema(&plan.inputs()); if let LogicalPlan::TableScan(ts) = &plan { let source_schema = DFSchema::try_from_qualified_schema( @@ -414,32 +414,32 @@ fn try_cast_numeric_literal( DataType::UInt64 => ScalarValue::UInt64(Some(value as u64)), DataType::Timestamp(TimeUnit::Second, tz) => { let value = cast_between_timestamp( - lit_data_type, - DataType::Timestamp(TimeUnit::Second, tz.clone()), + &lit_data_type, + &DataType::Timestamp(TimeUnit::Second, tz.clone()), value, ); ScalarValue::TimestampSecond(value, tz.clone()) } DataType::Timestamp(TimeUnit::Millisecond, tz) => { let value = cast_between_timestamp( - lit_data_type, - DataType::Timestamp(TimeUnit::Millisecond, tz.clone()), + &lit_data_type, + &DataType::Timestamp(TimeUnit::Millisecond, tz.clone()), value, ); ScalarValue::TimestampMillisecond(value, tz.clone()) } DataType::Timestamp(TimeUnit::Microsecond, tz) => { let value = cast_between_timestamp( - lit_data_type, - DataType::Timestamp(TimeUnit::Microsecond, tz.clone()), + &lit_data_type, + &DataType::Timestamp(TimeUnit::Microsecond, tz.clone()), value, ); ScalarValue::TimestampMicrosecond(value, tz.clone()) } DataType::Timestamp(TimeUnit::Nanosecond, tz) => { let value = cast_between_timestamp( - lit_data_type, - DataType::Timestamp(TimeUnit::Nanosecond, tz.clone()), + &lit_data_type, + &DataType::Timestamp(TimeUnit::Nanosecond, tz.clone()), value, ); ScalarValue::TimestampNanosecond(value, tz.clone()) @@ -505,7 +505,7 @@ fn try_cast_dictionary( } /// Cast a timestamp value from one unit to another -fn cast_between_timestamp(from: DataType, to: DataType, value: i128) -> Option { +fn cast_between_timestamp(from: &DataType, to: &DataType, value: i128) -> Option { let value = value as i64; let from_scale = match from { DataType::Timestamp(TimeUnit::Second, _) => 1, diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index 45cef55bf272..9e602ad49715 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -21,7 +21,7 @@ use std::collections::{BTreeSet, HashMap, HashSet}; use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::{Column, DFSchema, DFSchemaRef, Result}; +use datafusion_common::{Column, DFSchema, Result}; use datafusion_expr::expr_rewriter::replace_col; use datafusion_expr::utils as expr_utils; use datafusion_expr::{logical_plan::LogicalPlan, Expr, Operator}; @@ -83,7 +83,7 @@ pub(crate) fn has_all_column_refs(expr: &Expr, schema_cols: &HashSet) -> pub(crate) fn collect_subquery_cols( exprs: &[Expr], - subquery_schema: DFSchemaRef, + subquery_schema: &DFSchema, ) -> Result> { exprs.iter().try_fold(BTreeSet::new(), |mut cols, expr| { let mut using_cols: Vec = vec![]; @@ -296,5 +296,5 @@ pub fn only_or_err(slice: &[T]) -> Result<&T> { note = "use `datafusion_expr::utils::merge_schema` instead" )] pub fn merge_schema(inputs: Vec<&LogicalPlan>) -> DFSchema { - expr_utils::merge_schema(inputs) + expr_utils::merge_schema(&inputs) } From 4838cfbf453f3c21d9c5a84f9577329dd78aa763 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 2 Sep 2024 11:17:44 +0200 Subject: [PATCH 48/75] Remove deprecated LogicalPlan::with_new_inputs function (#12285) Deprecated since 35.0.0. --- datafusion/expr/src/logical_plan/plan.rs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 5bd6ab10331a..19be9143fd23 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -602,12 +602,6 @@ impl LogicalPlan { } } - /// Returns a copy of this `LogicalPlan` with the new inputs - #[deprecated(since = "35.0.0", note = "please use `with_new_exprs` instead")] - pub fn with_new_inputs(&self, inputs: &[LogicalPlan]) -> Result { - self.with_new_exprs(self.expressions(), inputs.to_vec()) - } - /// Recomputes schema and type information for this LogicalPlan if needed. /// /// Some `LogicalPlan`s may need to recompute their schema if the number or From b583591f9067cfa3a03962383b34196fbd5acc04 Mon Sep 17 00:00:00 2001 From: Sergei Grebnov Date: Mon, 2 Sep 2024 15:05:29 +0300 Subject: [PATCH 49/75] Support alternate format for Date32 unparsing (TEXT/SQLite) (#12282) --- datafusion/sql/src/unparser/dialect.rs | 24 ++++++++++++++++++++ datafusion/sql/src/unparser/expr.rs | 31 ++++++++++++++++++++++++-- 2 files changed, 53 insertions(+), 2 deletions(-) diff --git a/datafusion/sql/src/unparser/dialect.rs b/datafusion/sql/src/unparser/dialect.rs index 74f154d7870f..706b7633cb90 100644 --- a/datafusion/sql/src/unparser/dialect.rs +++ b/datafusion/sql/src/unparser/dialect.rs @@ -96,6 +96,12 @@ pub trait Dialect: Send + Sync { ast::DataType::Timestamp(None, tz_info) } + + /// The SQL type to use for Arrow Date32 unparsing + /// Most dialects use Date, but some, like SQLite require TEXT + fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { + sqlparser::ast::DataType::Date + } } /// `IntervalStyle` to use for unparsing @@ -206,6 +212,10 @@ impl Dialect for SqliteDialect { fn identifier_quote_style(&self, _: &str) -> Option { Some('`') } + + fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { + sqlparser::ast::DataType::Text + } } pub struct CustomDialect { @@ -220,6 +230,7 @@ pub struct CustomDialect { int64_cast_dtype: ast::DataType, timestamp_cast_dtype: ast::DataType, timestamp_tz_cast_dtype: ast::DataType, + date32_cast_dtype: sqlparser::ast::DataType, } impl Default for CustomDialect { @@ -239,6 +250,7 @@ impl Default for CustomDialect { None, TimezoneInfo::WithTimeZone, ), + date32_cast_dtype: sqlparser::ast::DataType::Date, } } } @@ -302,6 +314,10 @@ impl Dialect for CustomDialect { self.timestamp_cast_dtype.clone() } } + + fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { + self.date32_cast_dtype.clone() + } } /// `CustomDialectBuilder` to build `CustomDialect` using builder pattern @@ -330,6 +346,7 @@ pub struct CustomDialectBuilder { int64_cast_dtype: ast::DataType, timestamp_cast_dtype: ast::DataType, timestamp_tz_cast_dtype: ast::DataType, + date32_cast_dtype: ast::DataType, } impl Default for CustomDialectBuilder { @@ -355,6 +372,7 @@ impl CustomDialectBuilder { None, TimezoneInfo::WithTimeZone, ), + date32_cast_dtype: sqlparser::ast::DataType::Date, } } @@ -371,6 +389,7 @@ impl CustomDialectBuilder { int64_cast_dtype: self.int64_cast_dtype, timestamp_cast_dtype: self.timestamp_cast_dtype, timestamp_tz_cast_dtype: self.timestamp_tz_cast_dtype, + date32_cast_dtype: self.date32_cast_dtype, } } @@ -453,4 +472,9 @@ impl CustomDialectBuilder { self.timestamp_tz_cast_dtype = timestamp_tz_cast_dtype; self } + + pub fn with_date32_cast_dtype(mut self, date32_cast_dtype: ast::DataType) -> Self { + self.date32_cast_dtype = date32_cast_dtype; + self + } } diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 549635a31aef..8ff4773e8e94 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -1386,7 +1386,7 @@ impl Unparser<'_> { DataType::Timestamp(time_unit, tz) => { Ok(self.dialect.timestamp_cast_dtype(time_unit, tz)) } - DataType::Date32 => Ok(ast::DataType::Date), + DataType::Date32 => Ok(self.dialect.date32_cast_dtype()), DataType::Date64 => Ok(self.ast_type_for_date64_in_cast()), DataType::Time32(_) => { not_impl_err!("Unsupported DataType: conversion: {data_type:?}") @@ -2242,7 +2242,7 @@ mod tests { } #[test] - fn custom_dialect_with_teimstamp_cast_dtype() -> Result<()> { + fn custom_dialect_with_timestamp_cast_dtype() -> Result<()> { let default_dialect = CustomDialectBuilder::new().build(); let mysql_dialect = CustomDialectBuilder::new() .with_timestamp_cast_dtype( @@ -2279,4 +2279,31 @@ mod tests { } Ok(()) } + + #[test] + fn custom_dialect_date32_ast_dtype() -> Result<()> { + let default_dialect = CustomDialectBuilder::default().build(); + let sqlite_custom_dialect = CustomDialectBuilder::new() + .with_date32_cast_dtype(ast::DataType::Text) + .build(); + + for (dialect, data_type, identifier) in [ + (&default_dialect, DataType::Date32, "DATE"), + (&sqlite_custom_dialect, DataType::Date32, "TEXT"), + ] { + let unparser = Unparser::new(dialect); + + let expr = Expr::Cast(Cast { + expr: Box::new(col("a")), + data_type, + }); + let ast = unparser.expr_to_sql(&expr)?; + + let actual = format!("{}", ast); + let expected = format!(r#"CAST(a AS {identifier})"#); + + assert_eq!(actual, expected); + } + Ok(()) + } } From 93fb7157b454f00319d45ce01e1ba2bdebb52b48 Mon Sep 17 00:00:00 2001 From: June <61218022+itsjunetime@users.noreply.github.com> Date: Mon, 2 Sep 2024 06:10:17 -0600 Subject: [PATCH 50/75] Extract drive-by fixes from PR 12135 for easier reviewing (#12240) * Extract drive-by fixes from PR 12135 for easier reviewing * Add a few more cfgs to silence warnings with different feature sets * fmt --- datafusion/common/src/hash_utils.rs | 2 + .../core/src/datasource/listing/helpers.rs | 16 ++-- .../core/src/datasource/listing/table.rs | 7 +- .../datasource/physical_plan/parquet/mod.rs | 13 +-- .../physical_plan/parquet/row_filter.rs | 95 +++++++------------ datafusion/core/src/datasource/statistics.rs | 26 +++-- .../src/execution/session_state_defaults.rs | 3 + .../core/src/physical_optimizer/pruning.rs | 4 + .../physical-expr/src/expressions/binary.rs | 20 ++-- .../physical-plan/src/execution_plan.rs | 2 +- 10 files changed, 93 insertions(+), 95 deletions(-) diff --git a/datafusion/common/src/hash_utils.rs b/datafusion/common/src/hash_utils.rs index f3d2a0a4f9ab..72cfeafd0bfe 100644 --- a/datafusion/common/src/hash_utils.rs +++ b/datafusion/common/src/hash_utils.rs @@ -245,6 +245,8 @@ fn hash_struct_array( Ok(()) } +// only adding this `cfg` b/c this function is only used with this `cfg` +#[cfg(not(feature = "force_hash_collisions"))] fn hash_map_array( array: &MapArray, random_state: &RandomState, diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index dbeaf5dfcc36..33a16237e162 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -51,12 +51,12 @@ use object_store::{ObjectMeta, ObjectStore}; /// - the table provider can filter the table partition values with this expression /// - the expression can be marked as `TableProviderFilterPushDown::Exact` once this filtering /// was performed -pub fn expr_applicable_for_cols(col_names: &[String], expr: &Expr) -> bool { +pub fn expr_applicable_for_cols(col_names: &[&str], expr: &Expr) -> bool { let mut is_applicable = true; expr.apply(|expr| { match expr { Expr::Column(Column { ref name, .. }) => { - is_applicable &= col_names.contains(name); + is_applicable &= col_names.contains(&name.as_str()); if is_applicable { Ok(TreeNodeRecursion::Jump) } else { @@ -745,27 +745,27 @@ mod tests { #[test] fn test_expr_applicable_for_cols() { assert!(expr_applicable_for_cols( - &[String::from("c1")], + &["c1"], &Expr::eq(col("c1"), lit("value")) )); assert!(!expr_applicable_for_cols( - &[String::from("c1")], + &["c1"], &Expr::eq(col("c2"), lit("value")) )); assert!(!expr_applicable_for_cols( - &[String::from("c1")], + &["c1"], &Expr::eq(col("c1"), col("c2")) )); assert!(expr_applicable_for_cols( - &[String::from("c1"), String::from("c2")], + &["c1", "c2"], &Expr::eq(col("c1"), col("c2")) )); assert!(expr_applicable_for_cols( - &[String::from("c1"), String::from("c2")], + &["c1", "c2"], &(Expr::eq(col("c1"), col("c2").alias("c2_alias"))).not() )); assert!(expr_applicable_for_cols( - &[String::from("c1"), String::from("c2")], + &["c1", "c2"], &(case(col("c1")) .when(lit("v1"), lit(true)) .otherwise(lit(false)) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 35286612a887..9246226d431e 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -826,7 +826,7 @@ impl TableProvider for ListingTable { &self, filters: &[&Expr], ) -> Result> { - let support: Vec<_> = filters + Ok(filters .iter() .map(|filter| { if expr_applicable_for_cols( @@ -834,7 +834,7 @@ impl TableProvider for ListingTable { .options .table_partition_cols .iter() - .map(|x| x.0.clone()) + .map(|x| x.0.as_str()) .collect::>(), filter, ) { @@ -846,8 +846,7 @@ impl TableProvider for ListingTable { TableProviderFilterPushDown::Inexact } }) - .collect(); - Ok(support) + .collect()) } fn get_table_definition(&self) -> Option<&str> { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 85d6f8db2373..b2f86db742f2 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -685,10 +685,12 @@ impl ExecutionPlan for ParquetExec { partition_index: usize, ctx: Arc, ) -> Result { - let projection = match self.base_config.file_column_projection_indices() { - Some(proj) => proj, - None => (0..self.base_config.file_schema.fields().len()).collect(), - }; + let projection = self + .base_config + .file_column_projection_indices() + .unwrap_or_else(|| { + (0..self.base_config.file_schema.fields().len()).collect() + }); let parquet_file_reader_factory = self .parquet_file_reader_factory @@ -698,8 +700,7 @@ impl ExecutionPlan for ParquetExec { ctx.runtime_env() .object_store(&self.base_config.object_store_url) .map(|store| { - Arc::new(DefaultParquetFileReaderFactory::new(store)) - as Arc + Arc::new(DefaultParquetFileReaderFactory::new(store)) as _ }) })?; 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 23fdadc2cdee..59d23fd68c31 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs @@ -59,6 +59,7 @@ //! the unsorted predicates. Within each partition, predicates are //! still be sorted by size. +use std::cmp::Ordering; use std::collections::BTreeSet; use std::sync::Arc; @@ -129,7 +130,7 @@ impl DatafusionArrowPredicate { // on the order they appear in the file let projection = match candidate.projection.len() { 0 | 1 => vec![], - _ => remap_projection(&candidate.projection), + 2.. => remap_projection(&candidate.projection), }; Ok(Self { @@ -151,32 +152,31 @@ impl ArrowPredicate for DatafusionArrowPredicate { &self.projection_mask } - fn evaluate(&mut self, batch: RecordBatch) -> ArrowResult { - let batch = match self.projection.is_empty() { - true => batch, - false => batch.project(&self.projection)?, + fn evaluate(&mut self, mut batch: RecordBatch) -> ArrowResult { + if !self.projection.is_empty() { + batch = batch.project(&self.projection)?; }; let batch = self.schema_mapping.map_partial_batch(batch)?; // scoped timer updates on drop let mut timer = self.time.timer(); - match self - .physical_expr + + self.physical_expr .evaluate(&batch) .and_then(|v| v.into_array(batch.num_rows())) - { - Ok(array) => { + .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); timer.stop(); Ok(bool_arr) - } - Err(e) => Err(ArrowError::ComputeError(format!( - "Error evaluating filter predicate: {e:?}" - ))), - } + }) + .map_err(|e| { + ArrowError::ComputeError(format!( + "Error evaluating filter predicate: {e:?}" + )) + }) } } @@ -453,62 +453,33 @@ pub fn build_row_filter( // no candidates if candidates.is_empty() { - Ok(None) - } else if reorder_predicates { - // attempt to reorder the predicates by size and whether they are sorted - candidates.sort_by_key(|c| c.required_bytes); - - let (indexed_candidates, other_candidates): (Vec<_>, Vec<_>) = - candidates.into_iter().partition(|c| c.can_use_index); - - let mut filters: Vec> = vec![]; - - for candidate in indexed_candidates { - let filter = DatafusionArrowPredicate::try_new( - candidate, - file_schema, - metadata, - rows_filtered.clone(), - time.clone(), - Arc::clone(&schema_mapping), - )?; - - filters.push(Box::new(filter)); - } - - for candidate in other_candidates { - let filter = DatafusionArrowPredicate::try_new( - candidate, - file_schema, - metadata, - rows_filtered.clone(), - time.clone(), - Arc::clone(&schema_mapping), - )?; + return Ok(None); + } - filters.push(Box::new(filter)); - } + if reorder_predicates { + candidates.sort_unstable_by(|c1, c2| { + match c1.can_use_index.cmp(&c2.can_use_index) { + Ordering::Equal => c1.required_bytes.cmp(&c2.required_bytes), + ord => ord, + } + }); + } - Ok(Some(RowFilter::new(filters))) - } else { - // otherwise evaluate the predicates in the order the appeared in the - // original expressions - let mut filters: Vec> = vec![]; - for candidate in candidates { - let filter = DatafusionArrowPredicate::try_new( + candidates + .into_iter() + .map(|candidate| { + DatafusionArrowPredicate::try_new( candidate, file_schema, metadata, rows_filtered.clone(), time.clone(), Arc::clone(&schema_mapping), - )?; - - filters.push(Box::new(filter)); - } - - Ok(Some(RowFilter::new(filters))) - } + ) + .map(|pred| Box::new(pred) as _) + }) + .collect::, _>>() + .map(|filters| Some(RowFilter::new(filters))) } #[cfg(test)] diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 6f89657defd3..201bbfd5c007 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -18,16 +18,21 @@ use std::mem; use std::sync::Arc; -use arrow_schema::DataType; use futures::{Stream, StreamExt}; use datafusion_common::stats::Precision; use datafusion_common::ScalarValue; -use crate::arrow::datatypes::{Schema, SchemaRef}; +use crate::arrow::datatypes::SchemaRef; use crate::error::Result; -use crate::functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}; -use crate::physical_plan::{Accumulator, ColumnStatistics, Statistics}; +use crate::physical_plan::{ColumnStatistics, Statistics}; + +#[cfg(feature = "parquet")] +use crate::{ + arrow::datatypes::Schema, + functions_aggregate::min_max::{MaxAccumulator, MinAccumulator}, + physical_plan::Accumulator, +}; use super::listing::PartitionedFile; @@ -144,6 +149,8 @@ pub async fn get_statistics_with_limit( Ok((result_files, statistics)) } +// only adding this cfg b/c this is the only feature it's used with currently +#[cfg(feature = "parquet")] pub(crate) fn create_max_min_accs( schema: &Schema, ) -> (Vec>, Vec>) { @@ -175,6 +182,8 @@ fn add_row_stats( } } +// only adding this cfg b/c this is the only feature it's used with currently +#[cfg(feature = "parquet")] pub(crate) fn get_col_stats( schema: &Schema, null_counts: Vec>, @@ -205,8 +214,13 @@ pub(crate) fn get_col_stats( // (aka non Dictionary) output. We need to adjust the output data type to reflect this. // The reason min/max aggregate produces unpacked output because there is only one // min/max value per group; there is no needs to keep them Dictionary encode -fn min_max_aggregate_data_type(input_type: &DataType) -> &DataType { - if let DataType::Dictionary(_, value_type) = input_type { +// +// only adding this cfg b/c this is the only feature it's used with currently +#[cfg(feature = "parquet")] +fn min_max_aggregate_data_type( + input_type: &arrow_schema::DataType, +) -> &arrow_schema::DataType { + if let arrow_schema::DataType::Dictionary(_, value_type) = input_type { value_type.as_ref() } else { input_type diff --git a/datafusion/core/src/execution/session_state_defaults.rs b/datafusion/core/src/execution/session_state_defaults.rs index bc7e194caeae..b5370efa0a97 100644 --- a/datafusion/core/src/execution/session_state_defaults.rs +++ b/datafusion/core/src/execution/session_state_defaults.rs @@ -100,7 +100,9 @@ impl SessionStateDefaults { /// returns the list of default [`ScalarUDF']'s pub fn default_scalar_functions() -> Vec> { + #[cfg_attr(not(feature = "nested_expressions"), allow(unused_mut))] let mut functions: Vec> = functions::all_default_functions(); + #[cfg(feature = "nested_expressions")] functions.append(&mut functions_nested::all_default_nested_functions()); @@ -144,6 +146,7 @@ impl SessionStateDefaults { } /// registers all the builtin array functions + #[cfg_attr(not(feature = "nested_expressions"), allow(unused_variables))] pub fn register_array_functions(state: &mut SessionState) { // register crate of array expressions (if enabled) #[cfg(feature = "nested_expressions")] diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index a16abc607ee6..9bc2bb1d1db9 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -615,6 +615,8 @@ impl PruningPredicate { is_always_true(&self.predicate_expr) && self.literal_guarantees.is_empty() } + // this is only used by `parquet` feature right now + #[allow(dead_code)] pub(crate) fn required_columns(&self) -> &RequiredColumns { &self.required_columns } @@ -746,6 +748,8 @@ impl RequiredColumns { /// * `a > 5 OR a < 10` returns `Some(a)` /// * `a > 5 OR b < 10` returns `None` /// * `true` returns None + #[allow(dead_code)] + // this fn is only used by `parquet` feature right now, thus the `allow(dead_code)` pub(crate) fn single_column(&self) -> Option<&phys_expr::Column> { if self.columns.windows(2).all(|w| { // check if all columns are the same (ignoring statistics and field) diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 2680a7930ff1..08c133d7193a 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -33,6 +33,7 @@ use arrow::compute::kernels::comparison::{ use arrow::compute::kernels::concat_elements::concat_elements_utf8; use arrow::compute::{cast, ilike, like, nilike, nlike}; use arrow::datatypes::*; +use arrow_schema::ArrowError; use datafusion_common::cast::as_boolean_array; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_expr::interval_arithmetic::{apply_operator, Interval}; @@ -133,12 +134,15 @@ impl std::fmt::Display for BinaryExpr { } /// Invoke a boolean kernel on a pair of arrays -macro_rules! boolean_op { - ($LEFT:expr, $RIGHT:expr, $OP:ident) => {{ - let ll = as_boolean_array($LEFT).expect("boolean_op failed to downcast array"); - let rr = as_boolean_array($RIGHT).expect("boolean_op failed to downcast array"); - Ok(Arc::new($OP(&ll, &rr)?)) - }}; +#[inline] +fn boolean_op( + left: &dyn Array, + right: &dyn Array, + op: impl FnOnce(&BooleanArray, &BooleanArray) -> Result, +) -> Result, ArrowError> { + let ll = as_boolean_array(left).expect("boolean_op failed to downcast left array"); + let rr = as_boolean_array(right).expect("boolean_op failed to downcast right array"); + op(ll, rr).map(|t| Arc::new(t) as _) } macro_rules! binary_string_array_flag_op { @@ -596,7 +600,7 @@ impl BinaryExpr { | NotLikeMatch | NotILikeMatch => unreachable!(), And => { if left_data_type == &DataType::Boolean { - boolean_op!(&left, &right, and_kleene) + Ok(boolean_op(&left, &right, and_kleene)?) } else { internal_err!( "Cannot evaluate binary expression {:?} with types {:?} and {:?}", @@ -608,7 +612,7 @@ impl BinaryExpr { } Or => { if left_data_type == &DataType::Boolean { - boolean_op!(&left, &right, or_kleene) + Ok(boolean_op(&left, &right, or_kleene)?) } else { internal_err!( "Cannot evaluate binary expression {:?} with types {:?} and {:?}", diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 53ae59f7072f..f584542fafcf 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -718,7 +718,7 @@ pub fn execute_stream( match plan.output_partitioning().partition_count() { 0 => Ok(Box::pin(EmptyRecordBatchStream::new(plan.schema()))), 1 => plan.execute(0, context), - _ => { + 2.. => { // merge into a single partition let plan = CoalescePartitionsExec::new(Arc::clone(&plan)); // CoalescePartitionsExec must produce a single partition From 244d8a1bc2213163d110b40cb959649511086df6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Mon, 2 Sep 2024 20:48:49 +0800 Subject: [PATCH 51/75] Move `CombinePartialFinalAggregate` rule into physical-optimizer crate (#12167) --- .../combine_partial_final_agg.rs | 435 ------------------ datafusion/core/src/physical_optimizer/mod.rs | 1 - .../combine_partial_final_agg.rs | 290 ++++++++++++ .../core/tests/physical_optimizer/mod.rs | 1 + .../src/combine_partial_final_agg.rs | 164 +++++++ datafusion/physical-optimizer/src/lib.rs | 1 + 6 files changed, 456 insertions(+), 436 deletions(-) delete mode 100644 datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs create mode 100644 datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs create mode 100644 datafusion/physical-optimizer/src/combine_partial_final_agg.rs diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs deleted file mode 100644 index 1a12fc7de888..000000000000 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ /dev/null @@ -1,435 +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. - -//! CombinePartialFinalAggregate optimizer rule checks the adjacent Partial and Final AggregateExecs -//! and try to combine them if necessary - -use std::sync::Arc; - -use crate::error::Result; -use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; -use crate::physical_plan::ExecutionPlan; - -use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_physical_expr::aggregate::AggregateFunctionExpr; -use datafusion_physical_expr::{physical_exprs_equal, PhysicalExpr}; -use datafusion_physical_optimizer::PhysicalOptimizerRule; - -/// CombinePartialFinalAggregate optimizer rule combines the adjacent Partial and Final AggregateExecs -/// into a Single AggregateExec if their grouping exprs and aggregate exprs equal. -/// -/// This rule should be applied after the EnforceDistribution and EnforceSorting rules -/// -#[derive(Default)] -pub struct CombinePartialFinalAggregate {} - -impl CombinePartialFinalAggregate { - #[allow(missing_docs)] - pub fn new() -> Self { - Self {} - } -} - -impl PhysicalOptimizerRule for CombinePartialFinalAggregate { - fn optimize( - &self, - plan: Arc, - _config: &ConfigOptions, - ) -> Result> { - plan.transform_down(|plan| { - // Check if the plan is AggregateExec - let Some(agg_exec) = plan.as_any().downcast_ref::() else { - return Ok(Transformed::no(plan)); - }; - - if !matches!( - agg_exec.mode(), - AggregateMode::Final | AggregateMode::FinalPartitioned - ) { - return Ok(Transformed::no(plan)); - } - - // Check if the input is AggregateExec - let Some(input_agg_exec) = - agg_exec.input().as_any().downcast_ref::() - else { - return Ok(Transformed::no(plan)); - }; - - let transformed = if matches!(input_agg_exec.mode(), AggregateMode::Partial) - && can_combine( - ( - agg_exec.group_expr(), - agg_exec.aggr_expr(), - agg_exec.filter_expr(), - ), - ( - input_agg_exec.group_expr(), - input_agg_exec.aggr_expr(), - input_agg_exec.filter_expr(), - ), - ) { - let mode = if agg_exec.mode() == &AggregateMode::Final { - AggregateMode::Single - } else { - AggregateMode::SinglePartitioned - }; - AggregateExec::try_new( - mode, - input_agg_exec.group_expr().clone(), - input_agg_exec.aggr_expr().to_vec(), - input_agg_exec.filter_expr().to_vec(), - input_agg_exec.input().clone(), - input_agg_exec.input_schema(), - ) - .map(|combined_agg| combined_agg.with_limit(agg_exec.limit())) - .ok() - .map(Arc::new) - } else { - None - }; - Ok(if let Some(transformed) = transformed { - Transformed::yes(transformed) - } else { - Transformed::no(plan) - }) - }) - .data() - } - - fn name(&self) -> &str { - "CombinePartialFinalAggregate" - } - - fn schema_check(&self) -> bool { - true - } -} - -type GroupExprsRef<'a> = ( - &'a PhysicalGroupBy, - &'a [Arc], - &'a [Option>], -); - -fn can_combine(final_agg: GroupExprsRef, partial_agg: GroupExprsRef) -> bool { - let (final_group_by, final_aggr_expr, final_filter_expr) = final_agg; - let (input_group_by, input_aggr_expr, input_filter_expr) = partial_agg; - - // Compare output expressions of the partial, and input expressions of the final operator. - physical_exprs_equal( - &input_group_by.output_exprs(), - &final_group_by.input_exprs(), - ) && input_group_by.groups() == final_group_by.groups() - && input_group_by.null_expr().len() == final_group_by.null_expr().len() - && input_group_by - .null_expr() - .iter() - .zip(final_group_by.null_expr().iter()) - .all(|((lhs_expr, lhs_str), (rhs_expr, rhs_str))| { - lhs_expr.eq(rhs_expr) && lhs_str == rhs_str - }) - && final_aggr_expr.len() == input_aggr_expr.len() - && final_aggr_expr - .iter() - .zip(input_aggr_expr.iter()) - .all(|(final_expr, partial_expr)| final_expr.eq(partial_expr)) - && final_filter_expr.len() == input_filter_expr.len() - && final_filter_expr.iter().zip(input_filter_expr.iter()).all( - |(final_expr, partial_expr)| match (final_expr, partial_expr) { - (Some(l), Some(r)) => l.eq(r), - (None, None) => true, - _ => false, - }, - ) -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::datasource::listing::PartitionedFile; - use crate::datasource::object_store::ObjectStoreUrl; - use crate::datasource::physical_plan::{FileScanConfig, ParquetExec}; - use crate::physical_plan::expressions::lit; - use crate::physical_plan::repartition::RepartitionExec; - use crate::physical_plan::{displayable, Partitioning}; - - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use datafusion_functions_aggregate::count::count_udaf; - use datafusion_functions_aggregate::sum::sum_udaf; - use datafusion_physical_expr::aggregate::AggregateExprBuilder; - use datafusion_physical_expr::expressions::col; - - /// Runs the CombinePartialFinalAggregate optimizer and asserts the plan against the expected - macro_rules! assert_optimized { - ($EXPECTED_LINES: expr, $PLAN: expr) => { - let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); - - // run optimizer - let optimizer = CombinePartialFinalAggregate {}; - let config = ConfigOptions::new(); - let optimized = optimizer.optimize($PLAN, &config)?; - // Now format correctly - let plan = displayable(optimized.as_ref()).indent(true).to_string(); - let actual_lines = trim_plan_display(&plan); - - assert_eq!( - &expected_lines, &actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines - ); - }; - } - - fn trim_plan_display(plan: &str) -> Vec<&str> { - plan.split('\n') - .map(|s| s.trim()) - .filter(|s| !s.is_empty()) - .collect() - } - - fn schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Int64, true), - Field::new("c", DataType::Int64, true), - ])) - } - - fn parquet_exec(schema: &SchemaRef) -> Arc { - ParquetExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)), - ) - .build_arc() - } - - fn partial_aggregate_exec( - input: Arc, - group_by: PhysicalGroupBy, - aggr_expr: Vec>, - ) -> Arc { - let schema = input.schema(); - let n_aggr = aggr_expr.len(); - Arc::new( - AggregateExec::try_new( - AggregateMode::Partial, - group_by, - aggr_expr, - vec![None; n_aggr], - input, - schema, - ) - .unwrap(), - ) - } - - fn final_aggregate_exec( - input: Arc, - group_by: PhysicalGroupBy, - aggr_expr: Vec>, - ) -> Arc { - let schema = input.schema(); - let n_aggr = aggr_expr.len(); - Arc::new( - AggregateExec::try_new( - AggregateMode::Final, - group_by, - aggr_expr, - vec![None; n_aggr], - input, - schema, - ) - .unwrap(), - ) - } - - fn repartition_exec(input: Arc) -> Arc { - Arc::new( - RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap(), - ) - } - - // Return appropriate expr depending if COUNT is for col or table (*) - fn count_expr( - expr: Arc, - name: &str, - schema: &Schema, - ) -> Arc { - AggregateExprBuilder::new(count_udaf(), vec![expr]) - .schema(Arc::new(schema.clone())) - .alias(name) - .build() - .unwrap() - } - - #[test] - fn aggregations_not_combined() -> Result<()> { - let schema = schema(); - - let aggr_expr = vec![count_expr(lit(1i8), "COUNT(1)", &schema)]; - - let plan = final_aggregate_exec( - repartition_exec(partial_aggregate_exec( - parquet_exec(&schema), - PhysicalGroupBy::default(), - aggr_expr.clone(), - )), - PhysicalGroupBy::default(), - aggr_expr, - ); - // should not combine the Partial/Final AggregateExecs - let expected = &[ - "AggregateExec: mode=Final, gby=[], aggr=[COUNT(1)]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "AggregateExec: mode=Partial, gby=[], aggr=[COUNT(1)]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", - ]; - assert_optimized!(expected, plan); - - let aggr_expr1 = vec![count_expr(lit(1i8), "COUNT(1)", &schema)]; - let aggr_expr2 = vec![count_expr(lit(1i8), "COUNT(2)", &schema)]; - - let plan = final_aggregate_exec( - partial_aggregate_exec( - parquet_exec(&schema), - PhysicalGroupBy::default(), - aggr_expr1, - ), - PhysicalGroupBy::default(), - aggr_expr2, - ); - // should not combine the Partial/Final AggregateExecs - let expected = &[ - "AggregateExec: mode=Final, gby=[], aggr=[COUNT(2)]", - "AggregateExec: mode=Partial, gby=[], aggr=[COUNT(1)]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", - ]; - - assert_optimized!(expected, plan); - - Ok(()) - } - - #[test] - fn aggregations_combined() -> Result<()> { - let schema = schema(); - let aggr_expr = vec![count_expr(lit(1i8), "COUNT(1)", &schema)]; - - let plan = final_aggregate_exec( - partial_aggregate_exec( - parquet_exec(&schema), - PhysicalGroupBy::default(), - aggr_expr.clone(), - ), - PhysicalGroupBy::default(), - aggr_expr, - ); - // should combine the Partial/Final AggregateExecs to the Single AggregateExec - let expected = &[ - "AggregateExec: mode=Single, gby=[], aggr=[COUNT(1)]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", - ]; - - assert_optimized!(expected, plan); - Ok(()) - } - - #[test] - fn aggregations_with_group_combined() -> Result<()> { - let schema = schema(); - let aggr_expr = - vec![ - AggregateExprBuilder::new(sum_udaf(), vec![col("b", &schema)?]) - .schema(Arc::clone(&schema)) - .alias("Sum(b)") - .build() - .unwrap(), - ]; - let groups: Vec<(Arc, String)> = - vec![(col("c", &schema)?, "c".to_string())]; - - let partial_group_by = PhysicalGroupBy::new_single(groups); - let partial_agg = partial_aggregate_exec( - parquet_exec(&schema), - partial_group_by, - aggr_expr.clone(), - ); - - let groups: Vec<(Arc, String)> = - vec![(col("c", &partial_agg.schema())?, "c".to_string())]; - let final_group_by = PhysicalGroupBy::new_single(groups); - - let plan = final_aggregate_exec(partial_agg, final_group_by, aggr_expr); - // should combine the Partial/Final AggregateExecs to the Single AggregateExec - let expected = &[ - "AggregateExec: mode=Single, gby=[c@2 as c], aggr=[Sum(b)]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", - ]; - - assert_optimized!(expected, plan); - Ok(()) - } - - #[test] - fn aggregations_with_limit_combined() -> Result<()> { - let schema = schema(); - let aggr_expr = vec![]; - - let groups: Vec<(Arc, String)> = - vec![(col("c", &schema)?, "c".to_string())]; - - let partial_group_by = PhysicalGroupBy::new_single(groups); - let partial_agg = partial_aggregate_exec( - parquet_exec(&schema), - partial_group_by, - aggr_expr.clone(), - ); - - let groups: Vec<(Arc, String)> = - vec![(col("c", &partial_agg.schema())?, "c".to_string())]; - let final_group_by = PhysicalGroupBy::new_single(groups); - - let schema = partial_agg.schema(); - let final_agg = Arc::new( - AggregateExec::try_new( - AggregateMode::Final, - final_group_by, - aggr_expr, - vec![], - partial_agg, - schema, - ) - .unwrap() - .with_limit(Some(5)), - ); - let plan: Arc = final_agg; - // should combine the Partial/Final AggregateExecs to a Single AggregateExec - // with the final limit preserved - let expected = &[ - "AggregateExec: mode=Single, gby=[c@2 as c], aggr=[], lim=[5]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", - ]; - - assert_optimized!(expected, plan); - Ok(()) - } -} diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index c32c77043f15..46d86ead183b 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -22,7 +22,6 @@ //! //! [`ExecutionPlan`]: crate::physical_plan::ExecutionPlan pub mod coalesce_batches; -pub mod combine_partial_final_agg; pub mod enforce_distribution; pub mod enforce_sorting; pub mod join_selection; diff --git a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs new file mode 100644 index 000000000000..5152afa6c050 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs @@ -0,0 +1,290 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::physical_optimizer::combine_partial_final_agg::CombinePartialFinalAggregate; +use datafusion_common::config::ConfigOptions; +use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion_functions_aggregate::count::count_udaf; +use datafusion_functions_aggregate::sum::sum_udaf; +use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; +use datafusion_physical_expr::expressions::{col, lit}; +use datafusion_physical_expr::Partitioning; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; +use datafusion_physical_plan::displayable; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::ExecutionPlan; + +/// Runs the CombinePartialFinalAggregate optimizer and asserts the plan against the expected +macro_rules! assert_optimized { + ($EXPECTED_LINES: expr, $PLAN: expr) => { + let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); + + // run optimizer + let optimizer = CombinePartialFinalAggregate {}; + let config = ConfigOptions::new(); + let optimized = optimizer.optimize($PLAN, &config)?; + // Now format correctly + let plan = displayable(optimized.as_ref()).indent(true).to_string(); + let actual_lines = trim_plan_display(&plan); + + assert_eq!( + &expected_lines, &actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + }; +} + +fn trim_plan_display(plan: &str) -> Vec<&str> { + plan.split('\n') + .map(|s| s.trim()) + .filter(|s| !s.is_empty()) + .collect() +} + +fn schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + Field::new("c", DataType::Int64, true), + ])) +} + +fn parquet_exec(schema: &SchemaRef) -> Arc { + ParquetExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) + .with_file(PartitionedFile::new("x".to_string(), 100)), + ) + .build_arc() +} + +fn partial_aggregate_exec( + input: Arc, + group_by: PhysicalGroupBy, + aggr_expr: Vec>, +) -> Arc { + let schema = input.schema(); + let n_aggr = aggr_expr.len(); + Arc::new( + AggregateExec::try_new( + AggregateMode::Partial, + group_by, + aggr_expr, + vec![None; n_aggr], + input, + schema, + ) + .unwrap(), + ) +} + +fn final_aggregate_exec( + input: Arc, + group_by: PhysicalGroupBy, + aggr_expr: Vec>, +) -> Arc { + let schema = input.schema(); + let n_aggr = aggr_expr.len(); + Arc::new( + AggregateExec::try_new( + AggregateMode::Final, + group_by, + aggr_expr, + vec![None; n_aggr], + input, + schema, + ) + .unwrap(), + ) +} + +fn repartition_exec(input: Arc) -> Arc { + Arc::new(RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap()) +} + +// Return appropriate expr depending if COUNT is for col or table (*) +fn count_expr( + expr: Arc, + name: &str, + schema: &Schema, +) -> Arc { + AggregateExprBuilder::new(count_udaf(), vec![expr]) + .schema(Arc::new(schema.clone())) + .alias(name) + .build() + .unwrap() +} + +#[test] +fn aggregations_not_combined() -> datafusion_common::Result<()> { + let schema = schema(); + + let aggr_expr = vec![count_expr(lit(1i8), "COUNT(1)", &schema)]; + + let plan = final_aggregate_exec( + repartition_exec(partial_aggregate_exec( + parquet_exec(&schema), + PhysicalGroupBy::default(), + aggr_expr.clone(), + )), + PhysicalGroupBy::default(), + aggr_expr, + ); + // should not combine the Partial/Final AggregateExecs + let expected = &[ + "AggregateExec: mode=Final, gby=[], aggr=[COUNT(1)]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "AggregateExec: mode=Partial, gby=[], aggr=[COUNT(1)]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + ]; + assert_optimized!(expected, plan); + + let aggr_expr1 = vec![count_expr(lit(1i8), "COUNT(1)", &schema)]; + let aggr_expr2 = vec![count_expr(lit(1i8), "COUNT(2)", &schema)]; + + let plan = final_aggregate_exec( + partial_aggregate_exec( + parquet_exec(&schema), + PhysicalGroupBy::default(), + aggr_expr1, + ), + PhysicalGroupBy::default(), + aggr_expr2, + ); + // should not combine the Partial/Final AggregateExecs + let expected = &[ + "AggregateExec: mode=Final, gby=[], aggr=[COUNT(2)]", + "AggregateExec: mode=Partial, gby=[], aggr=[COUNT(1)]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + ]; + + assert_optimized!(expected, plan); + + Ok(()) +} + +#[test] +fn aggregations_combined() -> datafusion_common::Result<()> { + let schema = schema(); + let aggr_expr = vec![count_expr(lit(1i8), "COUNT(1)", &schema)]; + + let plan = final_aggregate_exec( + partial_aggregate_exec( + parquet_exec(&schema), + PhysicalGroupBy::default(), + aggr_expr.clone(), + ), + PhysicalGroupBy::default(), + aggr_expr, + ); + // should combine the Partial/Final AggregateExecs to the Single AggregateExec + let expected = &[ + "AggregateExec: mode=Single, gby=[], aggr=[COUNT(1)]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + ]; + + assert_optimized!(expected, plan); + Ok(()) +} + +#[test] +fn aggregations_with_group_combined() -> datafusion_common::Result<()> { + let schema = schema(); + let aggr_expr = vec![ + AggregateExprBuilder::new(sum_udaf(), vec![col("b", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("Sum(b)") + .build() + .unwrap(), + ]; + let groups: Vec<(Arc, String)> = + vec![(col("c", &schema)?, "c".to_string())]; + + let partial_group_by = PhysicalGroupBy::new_single(groups); + let partial_agg = partial_aggregate_exec( + parquet_exec(&schema), + partial_group_by, + aggr_expr.clone(), + ); + + let groups: Vec<(Arc, String)> = + vec![(col("c", &partial_agg.schema())?, "c".to_string())]; + let final_group_by = PhysicalGroupBy::new_single(groups); + + let plan = final_aggregate_exec(partial_agg, final_group_by, aggr_expr); + // should combine the Partial/Final AggregateExecs to the Single AggregateExec + let expected = &[ + "AggregateExec: mode=Single, gby=[c@2 as c], aggr=[Sum(b)]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + ]; + + assert_optimized!(expected, plan); + Ok(()) +} + +#[test] +fn aggregations_with_limit_combined() -> datafusion_common::Result<()> { + let schema = schema(); + let aggr_expr = vec![]; + + let groups: Vec<(Arc, String)> = + vec![(col("c", &schema)?, "c".to_string())]; + + let partial_group_by = PhysicalGroupBy::new_single(groups); + let partial_agg = partial_aggregate_exec( + parquet_exec(&schema), + partial_group_by, + aggr_expr.clone(), + ); + + let groups: Vec<(Arc, String)> = + vec![(col("c", &partial_agg.schema())?, "c".to_string())]; + let final_group_by = PhysicalGroupBy::new_single(groups); + + let schema = partial_agg.schema(); + let final_agg = Arc::new( + AggregateExec::try_new( + AggregateMode::Final, + final_group_by, + aggr_expr, + vec![], + partial_agg, + schema, + ) + .unwrap() + .with_limit(Some(5)), + ); + let plan: Arc = final_agg; + // should combine the Partial/Final AggregateExecs to a Single AggregateExec + // with the final limit preserved + let expected = &[ + "AggregateExec: mode=Single, gby=[c@2 as c], aggr=[], lim=[5]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", + ]; + + assert_optimized!(expected, plan); + Ok(()) +} diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 149103cf3482..4ec981bf2a74 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -16,6 +16,7 @@ // under the License. mod aggregate_statistics; +mod combine_partial_final_agg; mod limit_pushdown; mod limited_distinct_aggregation; mod test_util; diff --git a/datafusion/physical-optimizer/src/combine_partial_final_agg.rs b/datafusion/physical-optimizer/src/combine_partial_final_agg.rs new file mode 100644 index 000000000000..12ff13f8f6ae --- /dev/null +++ b/datafusion/physical-optimizer/src/combine_partial_final_agg.rs @@ -0,0 +1,164 @@ +// 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. + +//! CombinePartialFinalAggregate optimizer rule checks the adjacent Partial and Final AggregateExecs +//! and try to combine them if necessary + +use std::sync::Arc; + +use datafusion_common::error::Result; +use datafusion_physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; +use datafusion_physical_plan::ExecutionPlan; + +use crate::PhysicalOptimizerRule; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_physical_expr::aggregate::AggregateFunctionExpr; +use datafusion_physical_expr::{physical_exprs_equal, PhysicalExpr}; + +/// CombinePartialFinalAggregate optimizer rule combines the adjacent Partial and Final AggregateExecs +/// into a Single AggregateExec if their grouping exprs and aggregate exprs equal. +/// +/// This rule should be applied after the EnforceDistribution and EnforceSorting rules +/// +#[derive(Default)] +pub struct CombinePartialFinalAggregate {} + +impl CombinePartialFinalAggregate { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +impl PhysicalOptimizerRule for CombinePartialFinalAggregate { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + plan.transform_down(|plan| { + // Check if the plan is AggregateExec + let Some(agg_exec) = plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(plan)); + }; + + if !matches!( + agg_exec.mode(), + AggregateMode::Final | AggregateMode::FinalPartitioned + ) { + return Ok(Transformed::no(plan)); + } + + // Check if the input is AggregateExec + let Some(input_agg_exec) = + agg_exec.input().as_any().downcast_ref::() + else { + return Ok(Transformed::no(plan)); + }; + + let transformed = if matches!(input_agg_exec.mode(), AggregateMode::Partial) + && can_combine( + ( + agg_exec.group_expr(), + agg_exec.aggr_expr(), + agg_exec.filter_expr(), + ), + ( + input_agg_exec.group_expr(), + input_agg_exec.aggr_expr(), + input_agg_exec.filter_expr(), + ), + ) { + let mode = if agg_exec.mode() == &AggregateMode::Final { + AggregateMode::Single + } else { + AggregateMode::SinglePartitioned + }; + AggregateExec::try_new( + mode, + input_agg_exec.group_expr().clone(), + input_agg_exec.aggr_expr().to_vec(), + input_agg_exec.filter_expr().to_vec(), + Arc::clone(input_agg_exec.input()), + input_agg_exec.input_schema(), + ) + .map(|combined_agg| combined_agg.with_limit(agg_exec.limit())) + .ok() + .map(Arc::new) + } else { + None + }; + Ok(if let Some(transformed) = transformed { + Transformed::yes(transformed) + } else { + Transformed::no(plan) + }) + }) + .data() + } + + fn name(&self) -> &str { + "CombinePartialFinalAggregate" + } + + fn schema_check(&self) -> bool { + true + } +} + +type GroupExprsRef<'a> = ( + &'a PhysicalGroupBy, + &'a [Arc], + &'a [Option>], +); + +fn can_combine(final_agg: GroupExprsRef, partial_agg: GroupExprsRef) -> bool { + let (final_group_by, final_aggr_expr, final_filter_expr) = final_agg; + let (input_group_by, input_aggr_expr, input_filter_expr) = partial_agg; + + // Compare output expressions of the partial, and input expressions of the final operator. + physical_exprs_equal( + &input_group_by.output_exprs(), + &final_group_by.input_exprs(), + ) && input_group_by.groups() == final_group_by.groups() + && input_group_by.null_expr().len() == final_group_by.null_expr().len() + && input_group_by + .null_expr() + .iter() + .zip(final_group_by.null_expr().iter()) + .all(|((lhs_expr, lhs_str), (rhs_expr, rhs_str))| { + lhs_expr.eq(rhs_expr) && lhs_str == rhs_str + }) + && final_aggr_expr.len() == input_aggr_expr.len() + && final_aggr_expr + .iter() + .zip(input_aggr_expr.iter()) + .all(|(final_expr, partial_expr)| final_expr.eq(partial_expr)) + && final_filter_expr.len() == input_filter_expr.len() + && final_filter_expr.iter().zip(input_filter_expr.iter()).all( + |(final_expr, partial_expr)| match (final_expr, partial_expr) { + (Some(l), Some(r)) => l.eq(r), + (None, None) => true, + _ => false, + }, + ) +} + +// See tests in datafusion/core/tests/physical_optimizer diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index caebdcc927ae..41dfdb84a9f4 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -18,6 +18,7 @@ #![deny(clippy::clone_on_ref_ptr)] pub mod aggregate_statistics; +pub mod combine_partial_final_agg; pub mod limit_pushdown; pub mod limited_distinct_aggregation; mod optimizer; From 77e0e3b8b4df21eede9f1e3d1f8ee7709681a2d4 Mon Sep 17 00:00:00 2001 From: jcsherin Date: Mon, 2 Sep 2024 18:24:14 +0530 Subject: [PATCH 52/75] Fixes missing `nth_value` UDAF expr function (#12279) * Makes `nth_value` expression API public * Updates type of `order_by` parameter --- datafusion/functions-aggregate/src/lib.rs | 1 + .../functions-aggregate/src/nth_value.rs | 28 ++++++++++++++----- .../tests/cases/roundtrip_logical_plan.rs | 13 +++++++++ 3 files changed, 35 insertions(+), 7 deletions(-) diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index b54cd181a0cb..ca0276d326a4 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -113,6 +113,7 @@ pub mod expr_fn { pub use super::median::median; pub use super::min_max::max; pub use super::min_max::min; + pub use super::nth_value::nth_value; pub use super::regr::regr_avgx; pub use super::regr::regr_avgy; pub use super::regr::regr_count; diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index 7425bdfa18e7..bbfe56914c91 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -30,19 +30,33 @@ use datafusion_common::{exec_err, internal_err, not_impl_err, Result, ScalarValu use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, ReversedUDAF, Signature, Volatility, + lit, Accumulator, AggregateUDFImpl, ExprFunctionExt, ReversedUDAF, Signature, + SortExpr, Volatility, }; use datafusion_functions_aggregate_common::merge_arrays::merge_ordered_arrays; use datafusion_functions_aggregate_common::utils::ordering_fields; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; -make_udaf_expr_and_func!( - NthValueAgg, - nth_value, - "Returns the nth value in a group of values.", - nth_value_udaf -); +create_func!(NthValueAgg, nth_value_udaf); + +/// Returns the nth value in a group of values. +pub fn nth_value( + expr: datafusion_expr::Expr, + n: i64, + order_by: Vec, +) -> datafusion_expr::Expr { + let args = vec![expr, lit(n)]; + if !order_by.is_empty() { + nth_value_udaf() + .call(args) + .order_by(order_by) + .build() + .unwrap() + } else { + nth_value_udaf().call(args) + } +} /// Expression for a `NTH_VALUE(... ORDER BY ..., ...)` aggregation. In a multi /// partition setting, partial aggregations are computed for every partition, diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index e174d1b50713..994ed8ad2352 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -71,6 +71,7 @@ use datafusion_expr::{ use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::expr_fn::{ approx_distinct, array_agg, avg, bit_and, bit_or, bit_xor, bool_and, bool_or, corr, + nth_value, }; use datafusion_functions_aggregate::string_agg::string_agg; use datafusion_proto::bytes::{ @@ -903,6 +904,18 @@ async fn roundtrip_expr_api() -> Result<()> { vec![lit(10), lit(20), lit(30)], ), row_number(), + nth_value(col("b"), 1, vec![]), + nth_value( + col("b"), + 1, + vec![col("a").sort(false, false), col("b").sort(true, false)], + ), + nth_value(col("b"), -1, vec![]), + nth_value( + col("b"), + -1, + vec![col("a").sort(false, false), col("b").sort(true, false)], + ), ]; // ensure expressions created with the expr api can be round tripped From ac74cd3163e43563807a8c6e8e72bb058cb6f459 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 2 Sep 2024 09:02:42 -0400 Subject: [PATCH 53/75] Minor: Add `RuntimeEnvBuilder::build_arc() (#12213) --- datafusion/core/src/execution/context/mod.rs | 6 +-- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 10 ++--- datafusion/core/tests/memory_limit/mod.rs | 11 +++-- .../core/tests/parquet/file_statistics.rs | 11 +++-- datafusion/execution/src/runtime_env.rs | 5 +++ datafusion/execution/src/task.rs | 4 +- .../physical-plan/src/aggregates/mod.rs | 18 ++++----- .../physical-plan/src/joins/cross_join.rs | 8 ++-- .../physical-plan/src/joins/hash_join.rs | 16 +++----- .../src/joins/nested_loop_join.rs | 8 ++-- .../src/joins/sort_merge_join.rs | 40 ++++++++----------- .../physical-plan/src/repartition/mod.rs | 8 ++-- datafusion/physical-plan/src/sorts/sort.rs | 22 +++++----- datafusion/wasmtest/src/lib.rs | 10 ++--- 14 files changed, 75 insertions(+), 102 deletions(-) diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index c67424c0fa53..06dc797ae27a 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -219,13 +219,13 @@ where /// // configure a memory limit of 1GB with 20% slop /// let runtime_env = RuntimeEnvBuilder::new() /// .with_memory_limit(1024 * 1024 * 1024, 0.80) -/// .build() +/// .build_arc() /// .unwrap(); /// /// // Create a SessionState using the config and runtime_env /// let state = SessionStateBuilder::new() /// .with_config(config) -/// .with_runtime_env(Arc::new(runtime_env)) +/// .with_runtime_env(runtime_env) /// // include support for built in functions and configurations /// .with_default_features() /// .build(); @@ -1758,7 +1758,7 @@ mod tests { let path = path.join("tests/tpch-csv"); let url = format!("file://{}", path.display()); - let runtime = Arc::new(RuntimeEnvBuilder::new().build()?); + let runtime = RuntimeEnvBuilder::new().build_arc()?; let cfg = SessionConfig::new() .set_str("datafusion.catalog.location", url.as_str()) .set_str("datafusion.catalog.format", "CSV") diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index 1980589491a5..fae4731569b6 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -136,12 +136,10 @@ impl SortTest { .sort_spill_reservation_bytes, ); - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_pool(Arc::new(GreedyMemoryPool::new(pool_size))) - .build() - .unwrap(), - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_pool(Arc::new(GreedyMemoryPool::new(pool_size))) + .build_arc() + .unwrap(); SessionContext::new_with_config_rt(session_config, runtime) } else { SessionContext::new_with_config(session_config) diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index 592c25dedc50..69ef6058a2f6 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -509,21 +509,20 @@ impl TestCase { let table = scenario.table(); - let rt_config = RuntimeEnvBuilder::new() + let mut builder = RuntimeEnvBuilder::new() // disk manager setting controls the spilling .with_disk_manager(disk_manager_config) .with_memory_limit(memory_limit, MEMORY_FRACTION); - let runtime = if let Some(pool) = memory_pool { - rt_config.with_memory_pool(pool).build().unwrap() - } else { - rt_config.build().unwrap() + if let Some(pool) = memory_pool { + builder = builder.with_memory_pool(pool); }; + let runtime = builder.build_arc().unwrap(); // Configure execution let builder = SessionStateBuilder::new() .with_config(config) - .with_runtime_env(Arc::new(runtime)) + .with_runtime_env(runtime) .with_default_features(); let builder = match scenario.rules() { Some(rules) => builder.with_physical_optimizer_rules(rules), diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index bd251f1a6669..cd62c3bf426f 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -197,12 +197,11 @@ fn get_cache_runtime_state() -> ( .with_files_statistics_cache(Some(file_static_cache.clone())) .with_list_files_cache(Some(list_file_cache.clone())); - let rt = Arc::new( - RuntimeEnvBuilder::new() - .with_cache_manager(cache_config) - .build() - .expect("could not build runtime environment"), - ); + let rt = RuntimeEnvBuilder::new() + .with_cache_manager(cache_config) + .build_arc() + .expect("could not build runtime environment"); + let state = SessionContext::new_with_config_rt(SessionConfig::default(), rt).state(); (file_static_cache, list_file_cache, state) diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index e7b48be95cff..574d387ae697 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -246,4 +246,9 @@ impl RuntimeEnvBuilder { object_store_registry: self.object_store_registry, }) } + + /// Convenience method to create a new `Arc` + pub fn build_arc(self) -> Result> { + self.build().map(Arc::new) + } } diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index 35689b8e08df..57fcac0ee5ab 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -58,7 +58,7 @@ pub struct TaskContext { impl Default for TaskContext { fn default() -> Self { let runtime = RuntimeEnvBuilder::new() - .build() + .build_arc() .expect("default runtime created successfully"); // Create a default task context, mostly useful for testing @@ -69,7 +69,7 @@ impl Default for TaskContext { scalar_functions: HashMap::new(), aggregate_functions: HashMap::new(), window_functions: HashMap::new(), - runtime: Arc::new(runtime), + runtime, } } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index e66a57fd2ee5..764227e5e717 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1331,12 +1331,10 @@ mod tests { fn new_spill_ctx(batch_size: usize, max_memory: usize) -> Arc { let session_config = SessionConfig::new().with_batch_size(batch_size); - let runtime = Arc::new( - RuntimeEnvBuilder::default() - .with_memory_pool(Arc::new(FairSpillPool::new(max_memory))) - .build() - .unwrap(), - ); + let runtime = RuntimeEnvBuilder::default() + .with_memory_pool(Arc::new(FairSpillPool::new(max_memory))) + .build_arc() + .unwrap(); let task_ctx = TaskContext::default() .with_session_config(session_config) .with_runtime(runtime); @@ -1815,11 +1813,9 @@ mod tests { let input: Arc = Arc::new(TestYieldingExec::new(true)); let input_schema = input.schema(); - let runtime = Arc::new( - RuntimeEnvBuilder::default() - .with_memory_limit(1, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::default() + .with_memory_limit(1, 1.0) + .build_arc()?; let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index b99d4f17c42a..11153556f253 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -673,11 +673,9 @@ mod tests { #[tokio::test] async fn test_overallocation() -> Result<()> { - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build_arc()?; let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index f20d00e1a298..38827108e815 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -3798,11 +3798,9 @@ mod tests { ]; for join_type in join_types { - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build_arc()?; let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); @@ -3874,11 +3872,9 @@ mod tests { ]; for join_type in join_types { - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build_arc()?; let session_config = SessionConfig::default().with_batch_size(50); let task_ctx = TaskContext::default() .with_session_config(session_config) diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 3cd373544157..dadd20714ead 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -1019,11 +1019,9 @@ mod tests { ]; for join_type in join_types { - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .build_arc()?; let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 09fe5d9ebc54..2118c1a5266f 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -2900,12 +2900,10 @@ mod tests { ]; // Disable DiskManager to prevent spilling - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .with_disk_manager(DiskManagerConfig::Disabled) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .with_disk_manager(DiskManagerConfig::Disabled) + .build_arc()?; let session_config = SessionConfig::default().with_batch_size(50); for join_type in join_types { @@ -2987,12 +2985,10 @@ mod tests { ]; // Disable DiskManager to prevent spilling - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .with_disk_manager(DiskManagerConfig::Disabled) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .with_disk_manager(DiskManagerConfig::Disabled) + .build_arc()?; let session_config = SessionConfig::default().with_batch_size(50); for join_type in join_types { @@ -3052,12 +3048,10 @@ mod tests { ]; // Enable DiskManager to allow spilling - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(100, 1.0) - .with_disk_manager(DiskManagerConfig::NewOs) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(100, 1.0) + .with_disk_manager(DiskManagerConfig::NewOs) + .build_arc()?; for batch_size in [1, 50] { let session_config = SessionConfig::default().with_batch_size(batch_size); @@ -3162,12 +3156,10 @@ mod tests { ]; // Enable DiskManager to allow spilling - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(500, 1.0) - .with_disk_manager(DiskManagerConfig::NewOs) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(500, 1.0) + .with_disk_manager(DiskManagerConfig::NewOs) + .build_arc()?; for batch_size in [1, 50] { let session_config = SessionConfig::default().with_batch_size(batch_size); diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 650006a9d02d..47e5192c237e 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1506,11 +1506,9 @@ mod tests { let partitioning = Partitioning::RoundRobinBatch(4); // setup up context - let runtime = Arc::new( - RuntimeEnvBuilder::default() - .with_memory_limit(1, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::default() + .with_memory_limit(1, 1.0) + .build_arc()?; let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index e0041194016c..fa9628abdfbb 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1148,11 +1148,9 @@ mod tests { .options() .execution .sort_spill_reservation_bytes; - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit(sort_spill_reservation_bytes + 12288, 1.0) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(sort_spill_reservation_bytes + 12288, 1.0) + .build_arc()?; let task_ctx = Arc::new( TaskContext::default() .with_session_config(session_config) @@ -1226,14 +1224,12 @@ mod tests { .execution .sort_spill_reservation_bytes; - let runtime = Arc::new( - RuntimeEnvBuilder::new() - .with_memory_limit( - sort_spill_reservation_bytes + avg_batch_size * (partitions - 1), - 1.0, - ) - .build()?, - ); + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit( + sort_spill_reservation_bytes + avg_batch_size * (partitions - 1), + 1.0, + ) + .build_arc()?; let task_ctx = Arc::new( TaskContext::default() .with_runtime(runtime) diff --git a/datafusion/wasmtest/src/lib.rs b/datafusion/wasmtest/src/lib.rs index 50325d262d1d..0f24449cbed3 100644 --- a/datafusion/wasmtest/src/lib.rs +++ b/datafusion/wasmtest/src/lib.rs @@ -98,12 +98,10 @@ mod test { let sql = "SELECT 2 + 2;"; // Execute SQL (using datafusion) - let rt = Arc::new( - RuntimeEnvBuilder::new() - .with_disk_manager(DiskManagerConfig::Disabled) - .build() - .unwrap(), - ); + let rt = RuntimeEnvBuilder::new() + .with_disk_manager(DiskManagerConfig::Disabled) + .build_arc() + .unwrap(); let session_config = SessionConfig::new().with_target_partitions(1); let session_context = Arc::new(SessionContext::new_with_config_rt(session_config, rt)); From 8db30e25d6fe65a9779d237cf48aea9aee297502 Mon Sep 17 00:00:00 2001 From: Jay Zhan Date: Tue, 3 Sep 2024 07:57:40 +0800 Subject: [PATCH 54/75] Introduce `Signature::Coercible` (#12275) * introduce signature float Signed-off-by: jayzhan211 * fix test Signed-off-by: jayzhan211 * change float to coercible Signed-off-by: jayzhan211 * fmt Signed-off-by: jayzhan211 * rm test Signed-off-by: jayzhan211 * add comment Signed-off-by: jayzhan211 * typo Signed-off-by: jayzhan211 --------- Signed-off-by: jayzhan211 --- datafusion/core/src/dataframe/mod.rs | 6 ++-- datafusion/expr-common/src/signature.rs | 15 ++++++++- .../src/type_coercion/aggregates.rs | 6 ++-- .../expr/src/type_coercion/functions.rs | 33 ++++++++++++++++++- datafusion/functions-aggregate/src/stddev.rs | 11 +++---- .../functions-aggregate/src/variance.rs | 11 +++---- 6 files changed, 64 insertions(+), 18 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index b8c0bd9d744c..2138bd1294b4 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2428,7 +2428,8 @@ mod tests { let df: Vec = df.select(aggr_expr)?.collect().await?; assert_batches_sorted_eq!( - ["+-------------+----------+-----------------+---------------+--------+-----+------+----+------+", + [ + "+-------------+----------+-----------------+---------------+--------+-----+------+----+------+", "| first_value | last_val | approx_distinct | approx_median | median | max | min | c2 | c3 |", "+-------------+----------+-----------------+---------------+--------+-----+------+----+------+", "| | | | | | | | 1 | -85 |", @@ -2452,7 +2453,8 @@ mod tests { "| -85 | 45 | 8 | -34 | 45 | 83 | -85 | 3 | -72 |", "| -85 | 65 | 17 | -17 | 65 | 83 | -101 | 5 | -101 |", "| -85 | 83 | 5 | -25 | 83 | 83 | -85 | 2 | -48 |", - "+-------------+----------+-----------------+---------------+--------+-----+------+----+------+"], + "+-------------+----------+-----------------+---------------+--------+-----+------+----+------+", + ], &df ); diff --git a/datafusion/expr-common/src/signature.rs b/datafusion/expr-common/src/signature.rs index 4dcfa423e371..2043757a49fb 100644 --- a/datafusion/expr-common/src/signature.rs +++ b/datafusion/expr-common/src/signature.rs @@ -105,6 +105,11 @@ pub enum TypeSignature { Uniform(usize, Vec), /// Exact number of arguments of an exact type Exact(Vec), + /// The number of arguments that can be coerced to in order + /// For example, `Coercible(vec![DataType::Float64])` accepts + /// arguments like `vec![DataType::Int32]` or `vec![DataType::Float32]` + /// since i32 and f32 can be casted to f64 + Coercible(Vec), /// Fixed number of arguments of arbitrary types /// If a function takes 0 argument, its `TypeSignature` should be `Any(0)` Any(usize), @@ -188,7 +193,7 @@ impl TypeSignature { TypeSignature::Numeric(num) => { vec![format!("Numeric({})", num)] } - TypeSignature::Exact(types) => { + TypeSignature::Exact(types) | TypeSignature::Coercible(types) => { vec![Self::join_types(types, ", ")] } TypeSignature::Any(arg_count) => { @@ -300,6 +305,14 @@ impl Signature { volatility, } } + /// Target coerce types in order + pub fn coercible(target_types: Vec, volatility: Volatility) -> Self { + Self { + type_signature: TypeSignature::Coercible(target_types), + volatility, + } + } + /// A specified number of arguments of any type pub fn any(arg_count: usize, volatility: Volatility) -> Self { Signature { diff --git a/datafusion/expr-common/src/type_coercion/aggregates.rs b/datafusion/expr-common/src/type_coercion/aggregates.rs index 40ee596eee05..2add9e7c1867 100644 --- a/datafusion/expr-common/src/type_coercion/aggregates.rs +++ b/datafusion/expr-common/src/type_coercion/aggregates.rs @@ -128,9 +128,11 @@ pub fn check_arg_count( ); } } - TypeSignature::UserDefined | TypeSignature::Numeric(_) => { + TypeSignature::UserDefined + | TypeSignature::Numeric(_) + | TypeSignature::Coercible(_) => { // User-defined signature is validated in `coerce_types` - // Numreic signature is validated in `get_valid_types` + // Numeric and Coercible signature is validated in `get_valid_types` } _ => { return internal_err!( diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index b0b14a1a4e6e..d30d202df050 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -175,7 +175,14 @@ fn try_coerce_types( let mut valid_types = valid_types; // Well-supported signature that returns exact valid types. - if !valid_types.is_empty() && matches!(type_signature, TypeSignature::UserDefined) { + if !valid_types.is_empty() + && matches!( + type_signature, + TypeSignature::UserDefined + | TypeSignature::Numeric(_) + | TypeSignature::Coercible(_) + ) + { // exact valid types assert_eq!(valid_types.len(), 1); let valid_types = valid_types.swap_remove(0); @@ -397,6 +404,30 @@ fn get_valid_types( vec![vec![valid_type; *number]] } + TypeSignature::Coercible(target_types) => { + if target_types.is_empty() { + return plan_err!( + "The signature expected at least one argument but received {}", + current_types.len() + ); + } + if target_types.len() != current_types.len() { + return plan_err!( + "The signature expected {} arguments but received {}", + target_types.len(), + current_types.len() + ); + } + + for (data_type, target_type) in current_types.iter().zip(target_types.iter()) + { + if !can_cast_types(data_type, target_type) { + return plan_err!("{data_type} is not coercible to {target_type}"); + } + } + + vec![target_types.to_owned()] + } TypeSignature::Uniform(number, valid_types) => valid_types .iter() .map(|valid_type| (0..*number).map(|_| valid_type.clone()).collect()) diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index 3534fb5b4d26..a25ab5e31991 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -68,7 +68,10 @@ impl Stddev { /// Create a new STDDEV aggregate function pub fn new() -> Self { Self { - signature: Signature::numeric(1, Volatility::Immutable), + signature: Signature::coercible( + vec![DataType::Float64], + Volatility::Immutable, + ), alias: vec!["stddev_samp".to_string()], } } @@ -88,11 +91,7 @@ impl AggregateUDFImpl for Stddev { &self.signature } - fn return_type(&self, arg_types: &[DataType]) -> Result { - if !arg_types[0].is_numeric() { - return plan_err!("Stddev requires numeric input types"); - } - + fn return_type(&self, _arg_types: &[DataType]) -> Result { Ok(DataType::Float64) } diff --git a/datafusion/functions-aggregate/src/variance.rs b/datafusion/functions-aggregate/src/variance.rs index f5f2d06e3837..367a8669ab7d 100644 --- a/datafusion/functions-aggregate/src/variance.rs +++ b/datafusion/functions-aggregate/src/variance.rs @@ -79,7 +79,10 @@ impl VarianceSample { pub fn new() -> Self { Self { aliases: vec![String::from("var_sample"), String::from("var_samp")], - signature: Signature::numeric(1, Volatility::Immutable), + signature: Signature::coercible( + vec![DataType::Float64], + Volatility::Immutable, + ), } } } @@ -97,11 +100,7 @@ impl AggregateUDFImpl for VarianceSample { &self.signature } - fn return_type(&self, arg_types: &[DataType]) -> Result { - if !arg_types[0].is_numeric() { - return plan_err!("Variance requires numeric input types"); - } - + fn return_type(&self, _arg_types: &[DataType]) -> Result { Ok(DataType::Float64) } From 4e1b6de5549cc7ed399766dd22cafd450701db16 Mon Sep 17 00:00:00 2001 From: iamthinh Date: Tue, 3 Sep 2024 00:35:56 -0700 Subject: [PATCH 55/75] fix partition-by panic (#12297) --- datafusion/physical-plan/src/coalesce/mod.rs | 15 ++++++++++++-- .../physical-plan/src/repartition/mod.rs | 12 ++++++++--- .../sqllogictest/test_files/repartition.slt | 20 +++++++++++++++++++ 3 files changed, 42 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/coalesce/mod.rs b/datafusion/physical-plan/src/coalesce/mod.rs index 5befa5ecda99..ce5a1e53abfd 100644 --- a/datafusion/physical-plan/src/coalesce/mod.rs +++ b/datafusion/physical-plan/src/coalesce/mod.rs @@ -18,7 +18,7 @@ use arrow::compute::concat_batches; use arrow_array::builder::StringViewBuilder; use arrow_array::cast::AsArray; -use arrow_array::{Array, ArrayRef, RecordBatch}; +use arrow_array::{Array, ArrayRef, RecordBatch, RecordBatchOptions}; use arrow_schema::SchemaRef; use std::sync::Arc; @@ -265,7 +265,9 @@ fn gc_string_view_batch(batch: &RecordBatch) -> RecordBatch { } }) .collect(); - RecordBatch::try_new(batch.schema(), new_columns) + let mut options = RecordBatchOptions::new(); + options = options.with_row_count(Some(batch.num_rows())); + RecordBatch::try_new_with_options(batch.schema(), new_columns, &options) .expect("Failed to re-create the gc'ed record batch") } @@ -501,6 +503,15 @@ mod tests { assert_eq!(array.data_buffers().len(), gc_array.data_buffers().len()); // no compaction } + #[test] + fn test_gc_string_view_test_batch_empty() { + let schema = Schema::empty(); + let batch = RecordBatch::new_empty(schema.into()); + let output_batch = gc_string_view_batch(&batch); + assert_eq!(batch.num_columns(), output_batch.num_columns()); + assert_eq!(batch.num_rows(), output_batch.num_rows()); + } + #[test] fn test_gc_string_view_batch_large_no_compact() { // view with large strings (has buffers) but full --> no need to compact diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 47e5192c237e..093803e3c8b3 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -41,7 +41,7 @@ use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Stat use arrow::array::ArrayRef; use arrow::datatypes::{SchemaRef, UInt64Type}; use arrow::record_batch::RecordBatch; -use arrow_array::PrimitiveArray; +use arrow_array::{PrimitiveArray, RecordBatchOptions}; use datafusion_common::utils::transpose; use datafusion_common::{arrow_datafusion_err, not_impl_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; @@ -309,8 +309,14 @@ impl BatchPartitioner { }) .collect::>>()?; - let batch = - RecordBatch::try_new(batch.schema(), columns).unwrap(); + let mut options = RecordBatchOptions::new(); + options = options.with_row_count(Some(indices.len())); + let batch = RecordBatch::try_new_with_options( + batch.schema(), + columns, + &options, + ) + .unwrap(); Ok((partition, batch)) }); diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index e3c204a4f4ce..2d59ad2b5b0e 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -127,3 +127,23 @@ physical_plan 04)------FilterExec: c3@2 > 0 05)--------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 06)----------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true + +# Start repratition on empty column test. +# See https://github.com/apache/datafusion/issues/12057 + +statement ok +CREATE TABLE t1(v1 int); + +statement ok +INSERT INTO t1 values(42); + +query I +SELECT sum(1) OVER (PARTITION BY false=false) +FROM t1 WHERE ((false > (v1 = v1)) IS DISTINCT FROM true); +---- +1 + +statement ok +DROP TABLE t1; + +# End repartition on empty columns test \ No newline at end of file From 4a227c524f9130eb9b911af17a25c23b25aa41fa Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 3 Sep 2024 09:51:19 +0200 Subject: [PATCH 56/75] Remove unsafe Send impl from PriorityMap (#12289) It's not necessary to use unsafe Send impl. It's enough to require the referenced trait objects as Send. --- .../physical-plan/src/aggregates/topk/hash_table.rs | 5 ++++- datafusion/physical-plan/src/aggregates/topk/heap.rs | 6 +++++- .../physical-plan/src/aggregates/topk/priority_map.rs | 9 ++------- 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/topk/hash_table.rs b/datafusion/physical-plan/src/aggregates/topk/hash_table.rs index 2b02fff1f573..232b87de3231 100644 --- a/datafusion/physical-plan/src/aggregates/topk/hash_table.rs +++ b/datafusion/physical-plan/src/aggregates/topk/hash_table.rs @@ -367,7 +367,10 @@ has_integer!(u8, u16, u32, u64); has_integer!(IntervalDayTime, IntervalMonthDayNano); hash_float!(f16, f32, f64); -pub fn new_hash_table(limit: usize, kt: DataType) -> Result> { +pub fn new_hash_table( + limit: usize, + kt: DataType, +) -> Result> { macro_rules! downcast_helper { ($kt:ty, $d:ident) => { return Ok(Box::new(PrimitiveHashTable::<$kt>::new(limit))) diff --git a/datafusion/physical-plan/src/aggregates/topk/heap.rs b/datafusion/physical-plan/src/aggregates/topk/heap.rs index 81eadbc018b3..e694422e443d 100644 --- a/datafusion/physical-plan/src/aggregates/topk/heap.rs +++ b/datafusion/physical-plan/src/aggregates/topk/heap.rs @@ -459,7 +459,11 @@ compare_integer!(u8, u16, u32, u64); compare_integer!(IntervalDayTime, IntervalMonthDayNano); compare_float!(f16, f32, f64); -pub fn new_heap(limit: usize, desc: bool, vt: DataType) -> Result> { +pub fn new_heap( + limit: usize, + desc: bool, + vt: DataType, +) -> Result> { macro_rules! downcast_helper { ($vt:ty, $d:ident) => { return Ok(Box::new(PrimitiveHeap::<$vt>::new(limit, desc, vt))) diff --git a/datafusion/physical-plan/src/aggregates/topk/priority_map.rs b/datafusion/physical-plan/src/aggregates/topk/priority_map.rs index 668018b9c24c..ed41d22e935b 100644 --- a/datafusion/physical-plan/src/aggregates/topk/priority_map.rs +++ b/datafusion/physical-plan/src/aggregates/topk/priority_map.rs @@ -25,17 +25,12 @@ use datafusion_common::Result; /// A `Map` / `PriorityQueue` combo that evicts the worst values after reaching `capacity` pub struct PriorityMap { - map: Box, - heap: Box, + map: Box, + heap: Box, capacity: usize, mapper: Vec<(usize, usize)>, } -// JUSTIFICATION -// Benefit: ~15% speedup + required to index into RawTable from binary heap -// Soundness: it is only accessed by one thread at a time, and indexes are kept up to date -unsafe impl Send for PriorityMap {} - impl PriorityMap { pub fn new( key_type: DataType, From 6bceeae6c622485c8cfaae34eb60152f5c7cbe01 Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Tue, 3 Sep 2024 20:03:24 +0300 Subject: [PATCH 57/75] test: check record count and types in parquet window test (#12277) * test: check record count and types in paqruet window test * Review suggestions Co-authored-by: Andrew Lamb --------- Co-authored-by: Andrew Lamb --- .../sqllogictest/test_files/parquet.slt | 32 ++++++++----------- 1 file changed, 14 insertions(+), 18 deletions(-) diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 34d4ed6ff284..9a7b085312bb 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -251,25 +251,21 @@ SELECT COUNT(*) FROM timestamp_with_tz; ---- 131072 -# Perform the query: -query IPT -SELECT - count, - LAG(timestamp, 1) OVER (ORDER BY timestamp), - arrow_typeof(LAG(timestamp, 1) OVER (ORDER BY timestamp)) -FROM timestamp_with_tz -LIMIT 10; +# Ensure that output timestamp columns preserve the timezone from the input +# and output record count will match input file record count +query TPI +SELECT arrow_typeof(lag_timestamp), + MIN(lag_timestamp), + COUNT(1) +FROM ( + SELECT + count, + LAG(timestamp, 1) OVER (ORDER BY timestamp) AS lag_timestamp + FROM timestamp_with_tz +) t +GROUP BY 1 ---- -0 NULL Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -4 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -14 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) -0 2014-08-27T14:00:00Z Timestamp(Millisecond, Some("UTC")) +Timestamp(Millisecond, Some("UTC")) 2014-08-27T14:00:00Z 131072 # Test config listing_table_ignore_subdirectory: From bf6c82fc18a8243380b8302f9b266344b6955656 Mon Sep 17 00:00:00 2001 From: Alexander Rafferty Date: Wed, 4 Sep 2024 06:12:04 +1000 Subject: [PATCH 58/75] Optimize `struct` and `named_struct` functions (#11688) * Remove unnecessary heap allocations in implementation of `named_struct_expr` caused by zipping then unzipping fields and values. * Change implementation of `array_struct` to reduce number of allocations * Remove tests already covered by `struct.slt` --- datafusion/functions/src/core/named_struct.rs | 21 +++--- datafusion/functions/src/core/struct.rs | 68 ++++--------------- 2 files changed, 21 insertions(+), 68 deletions(-) diff --git a/datafusion/functions/src/core/named_struct.rs b/datafusion/functions/src/core/named_struct.rs index f71b1b00f0fe..85c332745355 100644 --- a/datafusion/functions/src/core/named_struct.rs +++ b/datafusion/functions/src/core/named_struct.rs @@ -70,20 +70,17 @@ fn named_struct_expr(args: &[ColumnarValue]) -> Result { } } - let arrays = ColumnarValue::values_to_arrays(&values)?; - - let fields = names + let fields: Fields = names .into_iter() - .zip(arrays) - .map(|(name, value)| { - ( - Arc::new(Field::new(name, value.data_type().clone(), true)), - value, - ) - }) - .collect::>(); + .zip(&values) + .map(|(name, value)| Arc::new(Field::new(name, value.data_type().clone(), true))) + .collect::>() + .into(); + + let arrays = ColumnarValue::values_to_arrays(&values)?; - Ok(ColumnarValue::Array(Arc::new(StructArray::from(fields)))) + let struct_array = StructArray::new(fields, arrays, None); + Ok(ColumnarValue::Array(Arc::new(struct_array))) } #[derive(Debug)] diff --git a/datafusion/functions/src/core/struct.rs b/datafusion/functions/src/core/struct.rs index c3dee8b1ccb4..bdddbb81beab 100644 --- a/datafusion/functions/src/core/struct.rs +++ b/datafusion/functions/src/core/struct.rs @@ -29,23 +29,23 @@ fn array_struct(args: &[ArrayRef]) -> Result { return exec_err!("struct requires at least one argument"); } - let vec: Vec<_> = args + let fields = args .iter() .enumerate() .map(|(i, arg)| { let field_name = format!("c{i}"); - Ok(( - Arc::new(Field::new( - field_name.as_str(), - arg.data_type().clone(), - true, - )), - Arc::clone(arg), - )) + Ok(Arc::new(Field::new( + field_name.as_str(), + arg.data_type().clone(), + true, + ))) }) - .collect::>>()?; + .collect::>>()? + .into(); - Ok(Arc::new(StructArray::from(vec))) + let arrays = args.to_vec(); + + Ok(Arc::new(StructArray::new(fields, arrays, None))) } /// put values in a struct array. @@ -53,6 +53,7 @@ fn struct_expr(args: &[ColumnarValue]) -> Result { let arrays = ColumnarValue::values_to_arrays(args)?; Ok(ColumnarValue::Array(array_struct(arrays.as_slice())?)) } + #[derive(Debug)] pub struct StructFunc { signature: Signature, @@ -97,48 +98,3 @@ impl ScalarUDFImpl for StructFunc { struct_expr(args) } } - -#[cfg(test)] -mod tests { - use super::*; - use arrow::array::Int64Array; - use datafusion_common::cast::as_struct_array; - use datafusion_common::ScalarValue; - - #[test] - fn test_struct() { - // struct(1, 2, 3) = {"c0": 1, "c1": 2, "c2": 3} - let args = [ - ColumnarValue::Scalar(ScalarValue::Int64(Some(1))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(2))), - ColumnarValue::Scalar(ScalarValue::Int64(Some(3))), - ]; - let struc = struct_expr(&args) - .expect("failed to initialize function struct") - .into_array(1) - .expect("Failed to convert to array"); - let result = - as_struct_array(&struc).expect("failed to initialize function struct"); - assert_eq!( - &Int64Array::from(vec![1]), - Arc::clone(result.column_by_name("c0").unwrap()) - .as_any() - .downcast_ref::() - .unwrap() - ); - assert_eq!( - &Int64Array::from(vec![2]), - Arc::clone(result.column_by_name("c1").unwrap()) - .as_any() - .downcast_ref::() - .unwrap() - ); - assert_eq!( - &Int64Array::from(vec![3]), - Arc::clone(result.column_by_name("c2").unwrap()) - .as_any() - .downcast_ref::() - .unwrap() - ); - } -} From e4a94243b502da2ad07a358b4401052651952eea Mon Sep 17 00:00:00 2001 From: WeblWabl Date: Tue, 3 Sep 2024 15:12:47 -0500 Subject: [PATCH 59/75] Update the CONCAT scalar function to support Utf8View (#12224) * wip * feat: Update the CONCAT scalar function to support Utf8View * fmt * fmt and add default return type for concat * fix clippy lint Signed-off-by: Devan * fmt Signed-off-by: Devan * add more tests for sqllogic Signed-off-by: Devan * make sure no casting with LargeUtf8 * fixing utf8large * fix large utf8 Signed-off-by: Devan * fix large utf8 Signed-off-by: Devan * add test Signed-off-by: Devan * fmt Signed-off-by: Devan * make it so Utf8View just returns Utf8 Signed-off-by: Devan * wip -- trying to build a stringview with columnar refs Signed-off-by: Devan * built stringview builder but it does allocate a new String each iter :( Signed-off-by: Devan * add some testing Signed-off-by: Devan * clippy Signed-off-by: Devan --------- Signed-off-by: Devan --- datafusion/functions/src/string/common.rs | 195 +++++++++++++++++- datafusion/functions/src/string/concat.rs | 184 ++++++++++++++--- .../sqllogictest/test_files/string_view.slt | 71 ++++++- 3 files changed, 416 insertions(+), 34 deletions(-) diff --git a/datafusion/functions/src/string/common.rs b/datafusion/functions/src/string/common.rs index 9738cb812f90..6ebcc4ee6cd3 100644 --- a/datafusion/functions/src/string/common.rs +++ b/datafusion/functions/src/string/common.rs @@ -22,12 +22,11 @@ use std::sync::Arc; use arrow::array::{ new_null_array, Array, ArrayAccessor, ArrayDataBuilder, ArrayIter, ArrayRef, - GenericStringArray, GenericStringBuilder, OffsetSizeTrait, StringArray, - StringBuilder, StringViewArray, + GenericStringArray, GenericStringBuilder, LargeStringArray, OffsetSizeTrait, + StringArray, StringBuilder, StringViewArray, StringViewBuilder, }; use arrow::buffer::{Buffer, MutableBuffer, NullBuffer}; use arrow::datatypes::DataType; - use datafusion_common::cast::{as_generic_string_array, as_string_view_array}; use datafusion_common::Result; use datafusion_common::{exec_err, ScalarValue}; @@ -249,26 +248,41 @@ where } } +#[derive(Debug)] pub(crate) enum ColumnarValueRef<'a> { Scalar(&'a [u8]), NullableArray(&'a StringArray), NonNullableArray(&'a StringArray), + NullableLargeStringArray(&'a LargeStringArray), + NonNullableLargeStringArray(&'a LargeStringArray), + NullableStringViewArray(&'a StringViewArray), + NonNullableStringViewArray(&'a StringViewArray), } impl<'a> ColumnarValueRef<'a> { #[inline] pub fn is_valid(&self, i: usize) -> bool { match &self { - Self::Scalar(_) | Self::NonNullableArray(_) => true, + Self::Scalar(_) + | Self::NonNullableArray(_) + | Self::NonNullableLargeStringArray(_) + | Self::NonNullableStringViewArray(_) => true, Self::NullableArray(array) => array.is_valid(i), + Self::NullableStringViewArray(array) => array.is_valid(i), + Self::NullableLargeStringArray(array) => array.is_valid(i), } } #[inline] pub fn nulls(&self) -> Option { match &self { - Self::Scalar(_) | Self::NonNullableArray(_) => None, + Self::Scalar(_) + | Self::NonNullableArray(_) + | Self::NonNullableStringViewArray(_) + | Self::NonNullableLargeStringArray(_) => None, Self::NullableArray(array) => array.nulls().cloned(), + Self::NullableStringViewArray(array) => array.nulls().cloned(), + Self::NullableLargeStringArray(array) => array.nulls().cloned(), } } } @@ -387,10 +401,30 @@ impl StringArrayBuilder { .extend_from_slice(array.value(i).as_bytes()); } } + ColumnarValueRef::NullableLargeStringArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + } + ColumnarValueRef::NullableStringViewArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + } ColumnarValueRef::NonNullableArray(array) => { self.value_buffer .extend_from_slice(array.value(i).as_bytes()); } + ColumnarValueRef::NonNullableLargeStringArray(array) => { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + ColumnarValueRef::NonNullableStringViewArray(array) => { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } } } @@ -416,6 +450,157 @@ impl StringArrayBuilder { } } +pub(crate) struct StringViewArrayBuilder { + builder: StringViewBuilder, + block: String, +} + +impl StringViewArrayBuilder { + pub fn with_capacity(_item_capacity: usize, data_capacity: usize) -> Self { + let builder = StringViewBuilder::with_capacity(data_capacity); + Self { + builder, + block: String::new(), + } + } + + pub fn write( + &mut self, + column: &ColumnarValueRef, + i: usize, + ) { + match column { + ColumnarValueRef::Scalar(s) => { + self.block.push_str(std::str::from_utf8(s).unwrap()); + } + ColumnarValueRef::NullableArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.block.push_str( + std::str::from_utf8(array.value(i).as_bytes()).unwrap(), + ); + } + } + ColumnarValueRef::NullableLargeStringArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.block.push_str( + std::str::from_utf8(array.value(i).as_bytes()).unwrap(), + ); + } + } + ColumnarValueRef::NullableStringViewArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.block.push_str( + std::str::from_utf8(array.value(i).as_bytes()).unwrap(), + ); + } + } + ColumnarValueRef::NonNullableArray(array) => { + self.block + .push_str(std::str::from_utf8(array.value(i).as_bytes()).unwrap()); + } + ColumnarValueRef::NonNullableLargeStringArray(array) => { + self.block + .push_str(std::str::from_utf8(array.value(i).as_bytes()).unwrap()); + } + ColumnarValueRef::NonNullableStringViewArray(array) => { + self.block + .push_str(std::str::from_utf8(array.value(i).as_bytes()).unwrap()); + } + } + } + + pub fn append_offset(&mut self) { + self.builder.append_value(&self.block); + self.block = String::new(); + } + + pub fn finish(mut self) -> StringViewArray { + self.builder.finish() + } +} + +pub(crate) struct LargeStringArrayBuilder { + offsets_buffer: MutableBuffer, + value_buffer: MutableBuffer, +} + +impl LargeStringArrayBuilder { + pub fn with_capacity(item_capacity: usize, data_capacity: usize) -> Self { + let mut offsets_buffer = MutableBuffer::with_capacity( + (item_capacity + 1) * std::mem::size_of::(), + ); + // SAFETY: the first offset value is definitely not going to exceed the bounds. + unsafe { offsets_buffer.push_unchecked(0_i64) }; + Self { + offsets_buffer, + value_buffer: MutableBuffer::with_capacity(data_capacity), + } + } + + pub fn write( + &mut self, + column: &ColumnarValueRef, + i: usize, + ) { + match column { + ColumnarValueRef::Scalar(s) => { + self.value_buffer.extend_from_slice(s); + } + ColumnarValueRef::NullableArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + } + ColumnarValueRef::NullableLargeStringArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + } + ColumnarValueRef::NullableStringViewArray(array) => { + if !CHECK_VALID || array.is_valid(i) { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + } + ColumnarValueRef::NonNullableArray(array) => { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + ColumnarValueRef::NonNullableLargeStringArray(array) => { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + ColumnarValueRef::NonNullableStringViewArray(array) => { + self.value_buffer + .extend_from_slice(array.value(i).as_bytes()); + } + } + } + + pub fn append_offset(&mut self) { + let next_offset: i64 = self + .value_buffer + .len() + .try_into() + .expect("byte array offset overflow"); + unsafe { self.offsets_buffer.push_unchecked(next_offset) }; + } + + pub fn finish(self, null_buffer: Option) -> LargeStringArray { + let array_builder = ArrayDataBuilder::new(DataType::LargeUtf8) + .len(self.offsets_buffer.len() / std::mem::size_of::() - 1) + .add_buffer(self.offsets_buffer.into()) + .add_buffer(self.value_buffer.into()) + .nulls(null_buffer); + // SAFETY: all data that was appended was valid Large UTF8 and the values + // and offsets were created correctly + let array_data = unsafe { array_builder.build_unchecked() }; + LargeStringArray::from(array_data) + } +} + fn case_conversion_array<'a, O, F>(array: &'a ArrayRef, op: F) -> Result where O: OffsetSizeTrait, diff --git a/datafusion/functions/src/string/concat.rs b/datafusion/functions/src/string/concat.rs index 6d15e2206721..00fe69b0bd33 100644 --- a/datafusion/functions/src/string/concat.rs +++ b/datafusion/functions/src/string/concat.rs @@ -15,14 +15,13 @@ // specific language governing permissions and limitations // under the License. +use arrow::array::{as_largestring_array, Array}; +use arrow::datatypes::DataType; use std::any::Any; use std::sync::Arc; -use arrow::datatypes::DataType; -use arrow::datatypes::DataType::Utf8; - -use datafusion_common::cast::as_string_array; -use datafusion_common::{internal_err, Result, ScalarValue}; +use datafusion_common::cast::{as_string_array, as_string_view_array}; +use datafusion_common::{internal_err, plan_err, Result, ScalarValue}; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; use datafusion_expr::{lit, ColumnarValue, Expr, Volatility}; @@ -46,7 +45,10 @@ impl ConcatFunc { pub fn new() -> Self { use DataType::*; Self { - signature: Signature::variadic(vec![Utf8], Volatility::Immutable), + signature: Signature::variadic( + vec![Utf8, Utf8View, LargeUtf8], + Volatility::Immutable, + ), } } } @@ -64,13 +66,36 @@ impl ScalarUDFImpl for ConcatFunc { &self.signature } - fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(Utf8) + fn return_type(&self, arg_types: &[DataType]) -> Result { + use DataType::*; + let mut dt = &Utf8; + arg_types.iter().for_each(|data_type| { + if data_type == &Utf8View { + dt = data_type; + } + if data_type == &LargeUtf8 && dt != &Utf8View { + dt = data_type; + } + }); + + Ok(dt.to_owned()) } /// Concatenates the text representations of all the arguments. NULL arguments are ignored. /// concat('abcde', 2, NULL, 22) = 'abcde222' fn invoke(&self, args: &[ColumnarValue]) -> Result { + let mut return_datatype = DataType::Utf8; + args.iter().for_each(|col| { + if col.data_type() == DataType::Utf8View { + return_datatype = col.data_type(); + } + if col.data_type() == DataType::LargeUtf8 + && return_datatype != DataType::Utf8View + { + return_datatype = col.data_type(); + } + }); + let array_len = args .iter() .filter_map(|x| match x { @@ -87,7 +112,21 @@ impl ScalarUDFImpl for ConcatFunc { result.push_str(v); } } - return Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some(result)))); + + return match return_datatype { + DataType::Utf8View => { + Ok(ColumnarValue::Scalar(ScalarValue::Utf8View(Some(result)))) + } + DataType::Utf8 => { + Ok(ColumnarValue::Scalar(ScalarValue::Utf8(Some(result)))) + } + DataType::LargeUtf8 => { + Ok(ColumnarValue::Scalar(ScalarValue::LargeUtf8(Some(result)))) + } + other => { + plan_err!("Concat function does not support datatype of {other}") + } + }; } // Array @@ -103,28 +142,95 @@ impl ScalarUDFImpl for ConcatFunc { columns.push(ColumnarValueRef::Scalar(s.as_bytes())); } } + ColumnarValue::Scalar(ScalarValue::Utf8View(maybe_value)) => { + if let Some(s) = maybe_value { + data_size += s.len() * len; + columns.push(ColumnarValueRef::Scalar(s.as_bytes())); + } + } ColumnarValue::Array(array) => { - let string_array = as_string_array(array)?; - data_size += string_array.values().len(); - let column = if array.is_nullable() { - ColumnarValueRef::NullableArray(string_array) - } else { - ColumnarValueRef::NonNullableArray(string_array) + match array.data_type() { + DataType::Utf8 => { + let string_array = as_string_array(array)?; + + data_size += string_array.values().len(); + let column = if array.is_nullable() { + ColumnarValueRef::NullableArray(string_array) + } else { + ColumnarValueRef::NonNullableArray(string_array) + }; + columns.push(column); + }, + DataType::LargeUtf8 => { + let string_array = as_largestring_array(array); + + data_size += string_array.values().len(); + let column = if array.is_nullable() { + ColumnarValueRef::NullableLargeStringArray(string_array) + } else { + ColumnarValueRef::NonNullableLargeStringArray(string_array) + }; + columns.push(column); + }, + DataType::Utf8View => { + let string_array = as_string_view_array(array)?; + + data_size += string_array.len(); + let column = if array.is_nullable() { + ColumnarValueRef::NullableStringViewArray(string_array) + } else { + ColumnarValueRef::NonNullableStringViewArray(string_array) + }; + columns.push(column); + }, + other => { + return plan_err!("Input was {other} which is not a supported datatype for concat function") + } }; - columns.push(column); } _ => unreachable!(), } } - let mut builder = StringArrayBuilder::with_capacity(len, data_size); - for i in 0..len { - columns - .iter() - .for_each(|column| builder.write::(column, i)); - builder.append_offset(); + match return_datatype { + DataType::Utf8 => { + let mut builder = StringArrayBuilder::with_capacity(len, data_size); + for i in 0..len { + columns + .iter() + .for_each(|column| builder.write::(column, i)); + builder.append_offset(); + } + + let string_array = builder.finish(None); + Ok(ColumnarValue::Array(Arc::new(string_array))) + } + DataType::Utf8View => { + let mut builder = StringViewArrayBuilder::with_capacity(len, data_size); + for i in 0..len { + columns + .iter() + .for_each(|column| builder.write::(column, i)); + builder.append_offset(); + } + + let string_array = builder.finish(); + Ok(ColumnarValue::Array(Arc::new(string_array))) + } + DataType::LargeUtf8 => { + let mut builder = LargeStringArrayBuilder::with_capacity(len, data_size); + for i in 0..len { + columns + .iter() + .for_each(|column| builder.write::(column, i)); + builder.append_offset(); + } + + let string_array = builder.finish(None); + Ok(ColumnarValue::Array(Arc::new(string_array))) + } + _ => unreachable!(), } - Ok(ColumnarValue::Array(Arc::new(builder.finish(None)))) } /// Simplify the `concat` function by @@ -151,11 +257,11 @@ pub fn simplify_concat(args: Vec) -> Result { for arg in args.clone() { match arg { // filter out `null` args - Expr::Literal(ScalarValue::Utf8(None) | ScalarValue::LargeUtf8(None)) => {} + Expr::Literal(ScalarValue::Utf8(None) | ScalarValue::LargeUtf8(None) | ScalarValue::Utf8View(None)) => {} // All literals have been converted to Utf8 or LargeUtf8 in type_coercion. // Concatenate it with the `contiguous_scalar`. Expr::Literal( - ScalarValue::Utf8(Some(v)) | ScalarValue::LargeUtf8(Some(v)), + ScalarValue::Utf8(Some(v)) | ScalarValue::LargeUtf8(Some(v)) | ScalarValue::Utf8View(Some(v)), ) => contiguous_scalar += &v, Expr::Literal(x) => { return internal_err!( @@ -195,8 +301,9 @@ pub fn simplify_concat(args: Vec) -> Result { mod tests { use super::*; use crate::utils::test::test_function; - use arrow::array::Array; + use arrow::array::{Array, LargeStringArray, StringViewArray}; use arrow::array::{ArrayRef, StringArray}; + use DataType::*; #[test] fn test_functions() -> Result<()> { @@ -232,6 +339,31 @@ mod tests { Utf8, StringArray ); + test_function!( + ConcatFunc::new(), + &[ + ColumnarValue::Scalar(ScalarValue::from("aa")), + ColumnarValue::Scalar(ScalarValue::Utf8View(None)), + ColumnarValue::Scalar(ScalarValue::LargeUtf8(None)), + ColumnarValue::Scalar(ScalarValue::from("cc")), + ], + Ok(Some("aacc")), + &str, + Utf8View, + StringViewArray + ); + test_function!( + ConcatFunc::new(), + &[ + ColumnarValue::Scalar(ScalarValue::from("aa")), + ColumnarValue::Scalar(ScalarValue::LargeUtf8(None)), + ColumnarValue::Scalar(ScalarValue::from("cc")), + ], + Ok(Some("aacc")), + &str, + LargeUtf8, + LargeStringArray + ); Ok(()) } diff --git a/datafusion/sqllogictest/test_files/string_view.slt b/datafusion/sqllogictest/test_files/string_view.slt index 83c75b8df38c..eb625e530b66 100644 --- a/datafusion/sqllogictest/test_files/string_view.slt +++ b/datafusion/sqllogictest/test_files/string_view.slt @@ -768,17 +768,26 @@ logical_plan 01)Projection: character_length(test.column1_utf8view) AS l 02)--TableScan: test projection=[column1_utf8view] -## Ensure no casts for CONCAT -## TODO https://github.com/apache/datafusion/issues/11836 +## Ensure no casts for CONCAT Utf8View query TT EXPLAIN SELECT concat(column1_utf8view, column2_utf8view) as c FROM test; ---- logical_plan -01)Projection: concat(CAST(test.column1_utf8view AS Utf8), CAST(test.column2_utf8view AS Utf8)) AS c +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 @@ -863,6 +872,61 @@ 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 + ## Ensure no casts for LPAD query TT EXPLAIN SELECT @@ -1307,3 +1371,4 @@ select column2|| ' ' ||column3 from temp; ---- rust fast datafusion cool + From 169fc1ef01e83393e9fde52519e820838592b96c Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Tue, 3 Sep 2024 22:57:45 +0200 Subject: [PATCH 60/75] chore: Fix warnings produced by shellcheck on bench.sh (#12303) --- benchmarks/bench.sh | 38 +++++++++++++++++++------------------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 3b854f6dc147..c02b08576eaa 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -77,7 +77,7 @@ parquet: Benchmark of parquet reader's filtering speed sort: Benchmark of sorting speed clickbench_1: ClickBench queries against a single parquet file clickbench_partitioned: ClickBench queries against a partitioned (100 files) parquet -clickbench_extended: ClickBench "inspired" queries against a single parquet (DataFusion specific) +clickbench_extended: ClickBench \"inspired\" queries against a single parquet (DataFusion specific) ********** * Supported Configuration (Environment Variables) @@ -106,7 +106,7 @@ while [[ $# -gt 0 ]]; do shift # past argument usage ;; - -*|--*) + -*) echo "Unknown option $1" exit 1 ;; @@ -175,7 +175,7 @@ main() { run) # Parse positional parameters BENCHMARK=${ARG2:-"${BENCHMARK}"} - BRANCH_NAME=$(cd ${DATAFUSION_DIR} && git rev-parse --abbrev-ref HEAD) + BRANCH_NAME=$(cd "${DATAFUSION_DIR}" && git rev-parse --abbrev-ref HEAD) BRANCH_NAME=${BRANCH_NAME//\//_} # mind blowing syntax to replace / with _ RESULTS_NAME=${RESULTS_NAME:-"${BRANCH_NAME}"} RESULTS_DIR=${RESULTS_DIR:-"$SCRIPT_DIR/results/$RESULTS_NAME"} @@ -189,7 +189,7 @@ main() { echo "DATA_DIR: ${DATA_DIR}" echo "RESULTS_DIR: ${RESULTS_DIR}" echo "CARGO_COMMAND: ${CARGO_COMMAND}" - echo "PREFER_HASH_JOIN": ${PREFER_HASH_JOIN} + echo "PREFER_HASH_JOIN: ${PREFER_HASH_JOIN}" echo "***************************" # navigate to the appropriate directory @@ -288,7 +288,7 @@ data_tpch() { echo " tbl files exist ($FILE exists)." else echo " creating tbl files with tpch_dbgen..." - docker run -v "${TPCH_DIR}":/data -it --rm ghcr.io/scalytics/tpch-docker:main -vf -s ${SCALE_FACTOR} + docker run -v "${TPCH_DIR}":/data -it --rm ghcr.io/scalytics/tpch-docker:main -vf -s "${SCALE_FACTOR}" fi # Copy expected answers into the ./data/answers directory if it does not already exist @@ -325,7 +325,7 @@ run_tpch() { RESULTS_FILE="${RESULTS_DIR}/tpch_sf${SCALE_FACTOR}.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running tpch benchmark..." - $CARGO_COMMAND --bin tpch -- benchmark datafusion --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join ${PREFER_HASH_JOIN} --format parquet -o ${RESULTS_FILE} + $CARGO_COMMAND --bin tpch -- benchmark datafusion --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --format parquet -o "${RESULTS_FILE}" } # Runs the tpch in memory @@ -341,7 +341,7 @@ run_tpch_mem() { echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running tpch_mem benchmark..." # -m means in memory - $CARGO_COMMAND --bin tpch -- benchmark datafusion --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join ${PREFER_HASH_JOIN} -m --format parquet -o ${RESULTS_FILE} + $CARGO_COMMAND --bin tpch -- benchmark datafusion --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" -m --format parquet -o "${RESULTS_FILE}" } # Runs the parquet filter benchmark @@ -349,7 +349,7 @@ run_parquet() { RESULTS_FILE="${RESULTS_DIR}/parquet.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running parquet filter benchmark..." - $CARGO_COMMAND --bin parquet -- filter --path "${DATA_DIR}" --prefer_hash_join ${PREFER_HASH_JOIN} --scale-factor 1.0 --iterations 5 -o ${RESULTS_FILE} + $CARGO_COMMAND --bin parquet -- filter --path "${DATA_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --scale-factor 1.0 --iterations 5 -o "${RESULTS_FILE}" } # Runs the sort benchmark @@ -357,7 +357,7 @@ run_sort() { RESULTS_FILE="${RESULTS_DIR}/sort.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running sort benchmark..." - $CARGO_COMMAND --bin parquet -- sort --path "${DATA_DIR}" --prefer_hash_join ${PREFER_HASH_JOIN} --scale-factor 1.0 --iterations 5 -o ${RESULTS_FILE} + $CARGO_COMMAND --bin parquet -- sort --path "${DATA_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --scale-factor 1.0 --iterations 5 -o "${RESULTS_FILE}" } @@ -369,7 +369,7 @@ data_clickbench_1() { pushd "${DATA_DIR}" > /dev/null # Avoid downloading if it already exists and is the right size - OUTPUT_SIZE=`wc -c hits.parquet 2>/dev/null | awk '{print $1}' || true` + OUTPUT_SIZE=$(wc -c hits.parquet 2>/dev/null | awk '{print $1}' || true) echo -n "Checking hits.parquet..." if test "${OUTPUT_SIZE}" = "14779976446"; then echo -n "... found ${OUTPUT_SIZE} bytes ..." @@ -393,7 +393,7 @@ data_clickbench_partitioned() { pushd "${DATA_DIR}/hits_partitioned" > /dev/null echo -n "Checking hits_partitioned..." - OUTPUT_SIZE=`wc -c * 2>/dev/null | tail -n 1 | awk '{print $1}' || true` + OUTPUT_SIZE=$(wc -c -- * 2>/dev/null | tail -n 1 | awk '{print $1}' || true) if test "${OUTPUT_SIZE}" = "14737666736"; then echo -n "... found ${OUTPUT_SIZE} bytes ..." else @@ -411,7 +411,7 @@ run_clickbench_1() { RESULTS_FILE="${RESULTS_DIR}/clickbench_1.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running clickbench (1 file) benchmark..." - $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits.parquet" --queries-path "${SCRIPT_DIR}/queries/clickbench/queries.sql" -o ${RESULTS_FILE} + $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits.parquet" --queries-path "${SCRIPT_DIR}/queries/clickbench/queries.sql" -o "${RESULTS_FILE}" } # Runs the clickbench benchmark with the partitioned parquet files @@ -419,7 +419,7 @@ run_clickbench_partitioned() { RESULTS_FILE="${RESULTS_DIR}/clickbench_partitioned.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running clickbench (partitioned, 100 files) benchmark..." - $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits_partitioned" --queries-path "${SCRIPT_DIR}/queries/clickbench/queries.sql" -o ${RESULTS_FILE} + $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits_partitioned" --queries-path "${SCRIPT_DIR}/queries/clickbench/queries.sql" -o "${RESULTS_FILE}" } # Runs the clickbench "extended" benchmark with a single large parquet file @@ -427,7 +427,7 @@ run_clickbench_extended() { RESULTS_FILE="${RESULTS_DIR}/clickbench_extended.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running clickbench (1 file) extended benchmark..." - $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits.parquet" --queries-path "${SCRIPT_DIR}/queries/clickbench/extended.sql" -o ${RESULTS_FILE} + $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits.parquet" --queries-path "${SCRIPT_DIR}/queries/clickbench/extended.sql" -o "${RESULTS_FILE}" } compare_benchmarks() { @@ -447,12 +447,12 @@ compare_benchmarks() { fi echo "Comparing ${BRANCH1} and ${BRANCH2}" - for bench in `ls ${BASE_RESULTS_DIR}/${BRANCH1}` ; do - RESULTS_FILE1="${BASE_RESULTS_DIR}/${BRANCH1}/${bench}" - RESULTS_FILE2="${BASE_RESULTS_DIR}/${BRANCH2}/${bench}" + for RESULTS_FILE1 in "${BASE_RESULTS_DIR}/${BRANCH1}"/*.json ; do + BENCH=$(basename "${RESULTS_FILE1}") + RESULTS_FILE2="${BASE_RESULTS_DIR}/${BRANCH2}/${BENCH}" if test -f "${RESULTS_FILE2}" ; then echo "--------------------" - echo "Benchmark ${bench}" + echo "Benchmark ${BENCH}" echo "--------------------" PATH=$VIRTUAL_ENV/bin:$PATH python3 "${SCRIPT_DIR}"/compare.py "${RESULTS_FILE1}" "${RESULTS_FILE2}" else @@ -463,7 +463,7 @@ compare_benchmarks() { } setup_venv() { - python3 -m venv $VIRTUAL_ENV + python3 -m venv "$VIRTUAL_ENV" PATH=$VIRTUAL_ENV/bin:$PATH python3 -m pip install -r requirements.txt } From 7ca98104fe0238e9a3991b291d98dc0a2210df7e Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Tue, 3 Sep 2024 23:58:40 +0300 Subject: [PATCH 61/75] test: re-enable merge join test with forced collisions (#12276) --- .../test_files/sort_merge_join.slt | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/datafusion/sqllogictest/test_files/sort_merge_join.slt b/datafusion/sqllogictest/test_files/sort_merge_join.slt index 2c28a5feadba..ebd53e9690fc 100644 --- a/datafusion/sqllogictest/test_files/sort_merge_join.slt +++ b/datafusion/sqllogictest/test_files/sort_merge_join.slt @@ -233,18 +233,16 @@ SELECT * FROM t1 FULL JOIN t2 ON t1_id = t2_id 44 d 4 44 x 3 NULL NULL NULL 55 w 3 -# FIXME(#TODO) fails with feature `force_hash_collisions` -# https://github.com/apache/datafusion/issues/11659 # equijoin_full_and_condition_from_both -# query ITIITI rowsort -# SELECT * FROM t1 FULL JOIN t2 ON t1_id = t2_id AND t2_int <= t1_int -# ---- -# 11 a 1 NULL NULL NULL -# 22 b 2 22 y 1 -# 33 c 3 NULL NULL NULL -# 44 d 4 44 x 3 -# NULL NULL NULL 11 z 3 -# NULL NULL NULL 55 w 3 +query ITIITI rowsort +SELECT * FROM t1 FULL JOIN t2 ON t1_id = t2_id AND t2_int <= t1_int +---- +11 a 1 NULL NULL NULL +22 b 2 22 y 1 +33 c 3 NULL NULL NULL +44 d 4 44 x 3 +NULL NULL NULL 11 z 3 +NULL NULL NULL 55 w 3 statement ok DROP TABLE t1; From 0cd7c25a37ae5d7fa27e5cf49a5aa258d73e9efa Mon Sep 17 00:00:00 2001 From: peasee <98815791+peasee@users.noreply.github.com> Date: Wed, 4 Sep 2024 06:59:19 +1000 Subject: [PATCH 62/75] feat: Add DateFieldExtractStyle::Strftime support for SqliteDialect unparser (#12161) * feat: Add DateFieldExtractStyle::Strftime support for SqliteDialect (#26) * feat: Add DateFieldExtractStyle::Strftime support for SqliteDialect * refactor: Refactor DateFieldExtractStyle if checks into if/match * Fix merge issue --------- Co-authored-by: Andrew Lamb --- datafusion/sql/src/unparser/dialect.rs | 5 ++ datafusion/sql/src/unparser/expr.rs | 97 ++++++++++++++++++++------ 2 files changed, 81 insertions(+), 21 deletions(-) diff --git a/datafusion/sql/src/unparser/dialect.rs b/datafusion/sql/src/unparser/dialect.rs index 706b7633cb90..2a8e61add1d0 100644 --- a/datafusion/sql/src/unparser/dialect.rs +++ b/datafusion/sql/src/unparser/dialect.rs @@ -130,6 +130,7 @@ pub enum IntervalStyle { pub enum DateFieldExtractStyle { DatePart, Extract, + Strftime, } pub struct DefaultDialect {} @@ -213,6 +214,10 @@ impl Dialect for SqliteDialect { Some('`') } + fn date_field_extract_style(&self) -> DateFieldExtractStyle { + DateFieldExtractStyle::Strftime + } + fn date32_cast_dtype(&self) -> sqlparser::ast::DataType { sqlparser::ast::DataType::Text } diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 8ff4773e8e94..fe34d87bfeae 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -524,30 +524,78 @@ impl Unparser<'_> { _func: &Arc, args: &[Expr], ) -> Option { - if func_name.to_lowercase() == "date_part" - && self.dialect.date_field_extract_style() == DateFieldExtractStyle::Extract - && args.len() == 2 - { - let date_expr = self.expr_to_sql(&args[1]).ok()?; - - if let Expr::Literal(ScalarValue::Utf8(Some(field))) = &args[0] { - let field = match field.to_lowercase().as_str() { - "year" => ast::DateTimeField::Year, - "month" => ast::DateTimeField::Month, - "day" => ast::DateTimeField::Day, - "hour" => ast::DateTimeField::Hour, - "minute" => ast::DateTimeField::Minute, - "second" => ast::DateTimeField::Second, - _ => return None, - }; + if func_name.to_lowercase() == "date_part" { + match (self.dialect.date_field_extract_style(), args.len()) { + (DateFieldExtractStyle::Extract, 2) => { + let date_expr = self.expr_to_sql(&args[1]).ok()?; + + if let Expr::Literal(ScalarValue::Utf8(Some(field))) = &args[0] { + let field = match field.to_lowercase().as_str() { + "year" => ast::DateTimeField::Year, + "month" => ast::DateTimeField::Month, + "day" => ast::DateTimeField::Day, + "hour" => ast::DateTimeField::Hour, + "minute" => ast::DateTimeField::Minute, + "second" => ast::DateTimeField::Second, + _ => return None, + }; + + return Some(ast::Expr::Extract { + field, + expr: Box::new(date_expr), + syntax: ast::ExtractSyntax::From, + }); + } + } + (DateFieldExtractStyle::Strftime, 2) => { + let column = self.expr_to_sql(&args[1]).ok()?; + + if let Expr::Literal(ScalarValue::Utf8(Some(field))) = &args[0] { + let field = match field.to_lowercase().as_str() { + "year" => "%Y", + "month" => "%m", + "day" => "%d", + "hour" => "%H", + "minute" => "%M", + "second" => "%S", + _ => return None, + }; - return Some(ast::Expr::Extract { - field, - expr: Box::new(date_expr), - syntax: ast::ExtractSyntax::From, - }); + return Some(ast::Expr::Function(ast::Function { + name: ast::ObjectName(vec![ast::Ident { + value: "strftime".to_string(), + quote_style: None, + }]), + args: ast::FunctionArguments::List( + ast::FunctionArgumentList { + duplicate_treatment: None, + args: vec![ + ast::FunctionArg::Unnamed( + ast::FunctionArgExpr::Expr(ast::Expr::Value( + ast::Value::SingleQuotedString( + field.to_string(), + ), + )), + ), + ast::FunctionArg::Unnamed( + ast::FunctionArgExpr::Expr(column), + ), + ], + clauses: vec![], + }, + ), + filter: None, + null_treatment: None, + over: None, + within_group: vec![], + parameters: ast::FunctionArguments::None, + })); + } + } + _ => {} // no overrides for DateFieldExtractStyle::DatePart, because it's already a date_part } } + None } @@ -2178,6 +2226,7 @@ mod tests { "YEAR", "EXTRACT(YEAR FROM x)", ), + (DateFieldExtractStyle::Strftime, "YEAR", "strftime('%Y', x)"), ( DateFieldExtractStyle::DatePart, "MONTH", @@ -2188,11 +2237,17 @@ mod tests { "MONTH", "EXTRACT(MONTH FROM x)", ), + ( + DateFieldExtractStyle::Strftime, + "MONTH", + "strftime('%m', x)", + ), ( DateFieldExtractStyle::DatePart, "DAY", "date_part('DAY', x)", ), + (DateFieldExtractStyle::Strftime, "DAY", "strftime('%d', x)"), (DateFieldExtractStyle::Extract, "DAY", "EXTRACT(DAY FROM x)"), ] { let dialect = CustomDialectBuilder::new() From f72622d6fcd27adac0bce5e87cccc83bad215448 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Wed, 4 Sep 2024 09:19:43 +0800 Subject: [PATCH 63/75] Fix various typos in aggregation doc (#12301) --- datafusion/expr-common/src/groups_accumulator.rs | 4 ++-- datafusion/physical-plan/src/aggregates/mod.rs | 3 ++- datafusion/physical-plan/src/aggregates/order/full.rs | 4 ++-- datafusion/physical-plan/src/aggregates/order/partial.rs | 2 +- datafusion/physical-plan/src/aggregates/row_hash.rs | 2 +- 5 files changed, 8 insertions(+), 7 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index e66b27d073d1..156e21d9ae20 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -29,7 +29,7 @@ pub enum EmitTo { /// indexes down by `n`. /// /// For example, if `n=10`, group_index `0, 1, ... 9` are emitted - /// and group indexes '`10, 11, 12, ...` become `0, 1, 2, ...`. + /// and group indexes `10, 11, 12, ...` become `0, 1, 2, ...`. First(usize), } @@ -113,7 +113,7 @@ pub trait GroupsAccumulator: Send { /// each group, and `evaluate` will produce that running sum as /// its output for all groups, in group_index order /// - /// If `emit_to`` is [`EmitTo::All`], the accumulator should + /// If `emit_to` is [`EmitTo::All`], the accumulator should /// return all groups and release / reset its internal state /// equivalent to when it was first created. /// diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 764227e5e717..375c6421b0d9 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1006,7 +1006,8 @@ pub fn get_finer_aggregate_exprs_requirement( Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) } -/// returns physical expressions for arguments to evaluate against a batch +/// Returns physical expressions for arguments to evaluate against a batch. +/// /// The expressions are different depending on `mode`: /// * Partial: AggregateFunctionExpr::expressions /// * Final: columns of `AggregateFunctionExpr::state_fields()` diff --git a/datafusion/physical-plan/src/aggregates/order/full.rs b/datafusion/physical-plan/src/aggregates/order/full.rs index e86d7677479a..d64c99ba1bee 100644 --- a/datafusion/physical-plan/src/aggregates/order/full.rs +++ b/datafusion/physical-plan/src/aggregates/order/full.rs @@ -21,7 +21,7 @@ use datafusion_expr::EmitTo; /// group keys /// /// When the group values are sorted, as soon as we see group `n+1` we -/// know we will never see any rows for group `n again and thus they +/// know we will never see any rows for group `n` again and thus they /// can be emitted. /// /// For example, given `SUM(amt) GROUP BY id` if the input is sorted @@ -63,7 +63,7 @@ enum State { /// Seen no input yet Start, - /// Data is in progress. `current is the current group for which + /// Data is in progress. `current` is the current group for which /// values are being generated. Can emit `current` - 1 InProgress { current: usize }, diff --git a/datafusion/physical-plan/src/aggregates/order/partial.rs b/datafusion/physical-plan/src/aggregates/order/partial.rs index 73a157f3aa96..2cbe3bbb784e 100644 --- a/datafusion/physical-plan/src/aggregates/order/partial.rs +++ b/datafusion/physical-plan/src/aggregates/order/partial.rs @@ -32,7 +32,7 @@ use std::sync::Arc; /// key and earlier. /// /// For example, given `SUM(amt) GROUP BY id, state` if the input is -/// sorted by `state, when a new value of `state` is seen, all groups +/// sorted by `state`, when a new value of `state` is seen, all groups /// with prior values of `state` can be emitted. /// /// The state is tracked like this: diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index d022bb007d9b..c38137994d44 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -95,7 +95,7 @@ struct SpillState { // ======================================================================== // STATES: // Fields changes during execution. Can be buffer, or state flags that - // influence the exeuction in parent `GroupedHashAggregateStream` + // influence the execution in parent `GroupedHashAggregateStream` // ======================================================================== /// If data has previously been spilled, the locations of the /// spill files (in Arrow IPC format) From 5ff5a6c924c0d3e2f2c2959d1a348be7913431c6 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Wed, 4 Sep 2024 15:19:38 +0800 Subject: [PATCH 64/75] Implement `kurtosis_pop` UDAF (#12273) * implement kurtosis_pop udaf * add tests * add empty end line * fix MSRV check * fix the null input and enhance tests * refactor the aggregation * address the review comments * add the doc for kurtois_pop * fix the doc style * use coercible signature * remove unused cast --- .../functions-aggregate/src/kurtosis_pop.rs | 190 ++++++++++++++++++ datafusion/functions-aggregate/src/lib.rs | 2 + .../tests/cases/roundtrip_logical_plan.rs | 2 + .../sqllogictest/test_files/aggregate.slt | 61 ++++++ .../user-guide/sql/aggregate_functions.md | 14 ++ 5 files changed, 269 insertions(+) create mode 100644 datafusion/functions-aggregate/src/kurtosis_pop.rs diff --git a/datafusion/functions-aggregate/src/kurtosis_pop.rs b/datafusion/functions-aggregate/src/kurtosis_pop.rs new file mode 100644 index 000000000000..ac173a0ee579 --- /dev/null +++ b/datafusion/functions-aggregate/src/kurtosis_pop.rs @@ -0,0 +1,190 @@ +// 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 arrow::array::{Array, ArrayRef, Float64Array, UInt64Array}; +use arrow_schema::{DataType, Field}; +use datafusion_common::cast::as_float64_array; +use datafusion_common::{downcast_value, DataFusionError, Result, ScalarValue}; +use datafusion_expr::{Accumulator, AggregateUDFImpl, Signature, Volatility}; +use datafusion_functions_aggregate_common::accumulator::{ + AccumulatorArgs, StateFieldsArgs, +}; +use std::any::Any; +use std::fmt::Debug; + +make_udaf_expr_and_func!( + KurtosisPopFunction, + kurtosis_pop, + x, + "Calculates the excess kurtosis (Fisher’s definition) without bias correction.", + kurtosis_pop_udaf +); + +pub struct KurtosisPopFunction { + signature: Signature, +} + +impl Debug for KurtosisPopFunction { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("KurtosisPopFunction") + .field("signature", &self.signature) + .finish() + } +} + +impl Default for KurtosisPopFunction { + fn default() -> Self { + Self::new() + } +} + +impl KurtosisPopFunction { + pub fn new() -> Self { + Self { + signature: Signature::coercible( + vec![DataType::Float64], + Volatility::Immutable, + ), + } + } +} + +impl AggregateUDFImpl for KurtosisPopFunction { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "kurtosis_pop" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Float64) + } + + fn state_fields(&self, _args: StateFieldsArgs) -> Result> { + Ok(vec![ + Field::new("count", DataType::UInt64, true), + Field::new("sum", DataType::Float64, true), + Field::new("sum_sqr", DataType::Float64, true), + Field::new("sum_cub", DataType::Float64, true), + Field::new("sum_four", DataType::Float64, true), + ]) + } + + fn accumulator(&self, _acc_args: AccumulatorArgs) -> Result> { + Ok(Box::new(KurtosisPopAccumulator::new())) + } +} + +/// Accumulator for calculating the excess kurtosis (Fisher’s definition) without bias correction. +/// This implementation follows the [DuckDB implementation]: +/// +#[derive(Debug, Default)] +pub struct KurtosisPopAccumulator { + count: u64, + sum: f64, + sum_sqr: f64, + sum_cub: f64, + sum_four: f64, +} + +impl KurtosisPopAccumulator { + pub fn new() -> Self { + Self { + count: 0, + sum: 0.0, + sum_sqr: 0.0, + sum_cub: 0.0, + sum_four: 0.0, + } + } +} + +impl Accumulator for KurtosisPopAccumulator { + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let array = as_float64_array(&values[0])?; + for value in array.iter().flatten() { + self.count += 1; + self.sum += value; + self.sum_sqr += value.powi(2); + self.sum_cub += value.powi(3); + self.sum_four += value.powi(4); + } + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + let counts = downcast_value!(states[0], UInt64Array); + let sums = downcast_value!(states[1], Float64Array); + let sum_sqrs = downcast_value!(states[2], Float64Array); + let sum_cubs = downcast_value!(states[3], Float64Array); + let sum_fours = downcast_value!(states[4], Float64Array); + + for i in 0..counts.len() { + let c = counts.value(i); + if c == 0 { + continue; + } + self.count += c; + self.sum += sums.value(i); + self.sum_sqr += sum_sqrs.value(i); + self.sum_cub += sum_cubs.value(i); + self.sum_four += sum_fours.value(i); + } + + Ok(()) + } + + fn evaluate(&mut self) -> Result { + if self.count < 1 { + return Ok(ScalarValue::Float64(None)); + } + + let count_64 = 1_f64 / self.count as f64; + let m4 = count_64 + * (self.sum_four - 4.0 * self.sum_cub * self.sum * count_64 + + 6.0 * self.sum_sqr * self.sum.powi(2) * count_64.powi(2) + - 3.0 * self.sum.powi(4) * count_64.powi(3)); + + let m2 = (self.sum_sqr - self.sum.powi(2) * count_64) * count_64; + if m2 <= 0.0 { + return Ok(ScalarValue::Float64(None)); + } + + let target = m4 / (m2.powi(2)) - 3.0; + Ok(ScalarValue::Float64(Some(target))) + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) + } + + fn state(&mut self) -> Result> { + Ok(vec![ + ScalarValue::from(self.count), + ScalarValue::from(self.sum), + ScalarValue::from(self.sum_sqr), + ScalarValue::from(self.sum_cub), + ScalarValue::from(self.sum_four), + ]) + } +} diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index ca0276d326a4..60e2602eb6ed 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -78,6 +78,7 @@ pub mod average; pub mod bit_and_or_xor; pub mod bool_and_or; pub mod grouping; +pub mod kurtosis_pop; pub mod nth_value; pub mod string_agg; @@ -170,6 +171,7 @@ pub fn all_default_aggregate_functions() -> Vec> { average::avg_udaf(), grouping::grouping_udaf(), nth_value::nth_value_udaf(), + kurtosis_pop::kurtosis_pop_udaf(), ] } diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 994ed8ad2352..dd3b99b0768b 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -73,6 +73,7 @@ use datafusion_functions_aggregate::expr_fn::{ approx_distinct, array_agg, avg, bit_and, bit_or, bit_xor, bool_and, bool_or, corr, nth_value, }; +use datafusion_functions_aggregate::kurtosis_pop::kurtosis_pop; use datafusion_functions_aggregate::string_agg::string_agg; use datafusion_proto::bytes::{ logical_plan_from_bytes, logical_plan_from_bytes_with_extension_codec, @@ -904,6 +905,7 @@ async fn roundtrip_expr_api() -> Result<()> { vec![lit(10), lit(20), lit(30)], ), row_number(), + kurtosis_pop(lit(1)), nth_value(col("b"), 1, vec![]), nth_value( col("b"), diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 45cb4d4615d7..c52445c561ee 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -5863,3 +5863,64 @@ ORDER BY k; ---- 1 1.8125 6.8007813 Float16 Float16 2 8.5 8.5 Float16 Float16 + +# The result is 0.19432323191699075 actually +query R +SELECT kurtosis_pop(col) FROM VALUES (1), (10), (100), (10), (1) as tab(col); +---- +0.194323231917 + +# The result is -1.153061224489787 actually +query R +SELECT kurtosis_pop(col) FROM VALUES (1), (2), (3), (2), (1) as tab(col); +---- +-1.15306122449 + +query R +SELECT kurtosis_pop(col) FROM VALUES (1.0), (10.0), (100.0), (10.0), (1.0) as tab(col); +---- +0.194323231917 + +query R +SELECT kurtosis_pop(col) FROM VALUES ('1'), ('10'), ('100'), ('10'), ('1') as tab(col); +---- +0.194323231917 + +query R +SELECT kurtosis_pop(col) FROM VALUES (1.0) as tab(col); +---- +NULL + +query R +SELECT kurtosis_pop(1) +---- +NULL + +query R +SELECT kurtosis_pop(1.0) +---- +NULL + +query R +SELECT kurtosis_pop(null) +---- +NULL + +statement ok +CREATE TABLE t1(c1 int); + +query R +SELECT kurtosis_pop(c1) FROM t1; +---- +NULL + +statement ok +INSERT INTO t1 VALUES (1), (10), (100), (10), (1); + +query R +SELECT kurtosis_pop(c1) FROM t1; +---- +0.194323231917 + +statement ok +DROP TABLE t1; diff --git a/docs/source/user-guide/sql/aggregate_functions.md b/docs/source/user-guide/sql/aggregate_functions.md index edb0e1d0c9f0..1c214084b3fa 100644 --- a/docs/source/user-guide/sql/aggregate_functions.md +++ b/docs/source/user-guide/sql/aggregate_functions.md @@ -252,6 +252,7 @@ last_value(expression [ORDER BY expression]) - [regr_sxx](#regr_sxx) - [regr_syy](#regr_syy) - [regr_sxy](#regr_sxy) +- [kurtosis_pop](#kurtosis_pop) ### `corr` @@ -527,6 +528,19 @@ regr_sxy(expression_y, expression_x) - **expression_x**: Independent variable. Can be a constant, column, or function, and any combination of arithmetic operators. +### `kurtosis_pop` + +Computes the excess kurtosis (Fisher’s definition) without bias correction. + +``` +kurtois_pop(expression) +``` + +#### Arguments + +- **expression**: Expression to operate on. + Can be a constant, column, or function, and any combination of arithmetic operators. + ## Approximate - [approx_distinct](#approx_distinct) From 6bbad7eda15cd1adcfb85182f6a20df6b5035842 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Wed, 4 Sep 2024 16:41:11 +0800 Subject: [PATCH 65/75] perf: avoid repeat format in calc_func_dependencies_for_project (#12305) --- datafusion/expr/src/logical_plan/plan.rs | 28 ++++++++++++++---------- 1 file changed, 17 insertions(+), 11 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 19be9143fd23..d0c1c3b2b3d6 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -2793,22 +2793,28 @@ fn calc_func_dependencies_for_project( .filter_map(|(qualifier, f)| { let flat_name = qualifier .map(|t| format!("{}.{}", t, f.name())) - .unwrap_or(f.name().clone()); + .unwrap_or_else(|| f.name().clone()); input_fields.iter().position(|item| *item == flat_name) }) .collect::>(), ) } - Expr::Alias(alias) => Ok(input_fields - .iter() - .position(|item| *item == format!("{}", alias.expr)) - .map(|i| vec![i]) - .unwrap_or(vec![])), - _ => Ok(input_fields - .iter() - .position(|item| *item == format!("{}", expr)) - .map(|i| vec![i]) - .unwrap_or(vec![])), + Expr::Alias(alias) => { + let name = format!("{}", alias.expr); + Ok(input_fields + .iter() + .position(|item| *item == name) + .map(|i| vec![i]) + .unwrap_or(vec![])) + } + _ => { + let name = format!("{}", expr); + Ok(input_fields + .iter() + .position(|item| *item == name) + .map(|i| vec![i]) + .unwrap_or(vec![])) + } }) .collect::>>()? .into_iter() From 9ab27248240bd798a4e814e4fe05f644c8ea3766 Mon Sep 17 00:00:00 2001 From: Alex Huang Date: Wed, 4 Sep 2024 17:54:29 +0800 Subject: [PATCH 66/75] feat: Enforce the uniqueness of map key name for the map/make_map function (#12153) * feat: Enforce the uniqueness of map key name for the map/make_map function * chore: Update tests * chore * chore: Update tests for nested type * refactor * chore * fix: Check unique key for the make_map function earlier * fix: Update bench * chore: Clean UP --- datafusion/common/src/utils/mod.rs | 7 +- datafusion/functions-nested/benches/map.rs | 20 +++--- datafusion/functions-nested/src/map.rs | 79 ++++++++++++++++------ datafusion/sqllogictest/test_files/map.slt | 39 +++++++++++ 4 files changed, 115 insertions(+), 30 deletions(-) diff --git a/datafusion/common/src/utils/mod.rs b/datafusion/common/src/utils/mod.rs index 418ea380bc2c..83f98ff9aff6 100644 --- a/datafusion/common/src/utils/mod.rs +++ b/datafusion/common/src/utils/mod.rs @@ -444,10 +444,15 @@ pub fn arrays_into_list_array( } /// Helper function to convert a ListArray into a vector of ArrayRefs. -pub fn list_to_arrays(a: ArrayRef) -> Vec { +pub fn list_to_arrays(a: &ArrayRef) -> Vec { a.as_list::().iter().flatten().collect::>() } +/// Helper function to convert a FixedSizeListArray into a vector of ArrayRefs. +pub fn fixed_size_list_to_arrays(a: &ArrayRef) -> Vec { + a.as_fixed_size_list().iter().flatten().collect::>() +} + /// Get the base type of a data type. /// /// Example diff --git a/datafusion/functions-nested/benches/map.rs b/datafusion/functions-nested/benches/map.rs index c9a12eefa4fa..ca23d8b7ff4c 100644 --- a/datafusion/functions-nested/benches/map.rs +++ b/datafusion/functions-nested/benches/map.rs @@ -23,6 +23,7 @@ use arrow_schema::{DataType, Field}; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use rand::prelude::ThreadRng; use rand::Rng; +use std::collections::HashSet; use std::sync::Arc; use datafusion_common::ScalarValue; @@ -32,19 +33,22 @@ use datafusion_functions_nested::map::map_udf; use datafusion_functions_nested::planner::NestedFunctionPlanner; fn keys(rng: &mut ThreadRng) -> Vec { - let mut keys = vec![]; - for _ in 0..1000 { - keys.push(rng.gen_range(0..9999).to_string()); + let mut keys = HashSet::with_capacity(1000); + + while keys.len() < 1000 { + keys.insert(rng.gen_range(0..10000).to_string()); } - keys + + keys.into_iter().collect() } fn values(rng: &mut ThreadRng) -> Vec { - let mut values = vec![]; - for _ in 0..1000 { - values.push(rng.gen_range(0..9999)); + let mut values = HashSet::with_capacity(1000); + + while values.len() < 1000 { + values.insert(rng.gen_range(0..10000)); } - values + values.into_iter().collect() } fn criterion_benchmark(c: &mut Criterion) { diff --git a/datafusion/functions-nested/src/map.rs b/datafusion/functions-nested/src/map.rs index b6068fdff0d5..29afe4a7f3be 100644 --- a/datafusion/functions-nested/src/map.rs +++ b/datafusion/functions-nested/src/map.rs @@ -16,7 +16,7 @@ // under the License. use std::any::Any; -use std::collections::VecDeque; +use std::collections::{HashSet, VecDeque}; use std::sync::Arc; use arrow::array::ArrayData; @@ -24,7 +24,8 @@ use arrow_array::{Array, ArrayRef, MapArray, OffsetSizeTrait, StructArray}; use arrow_buffer::{Buffer, ToByteSlice}; use arrow_schema::{DataType, Field, SchemaBuilder}; -use datafusion_common::{exec_err, ScalarValue}; +use datafusion_common::utils::{fixed_size_list_to_arrays, list_to_arrays}; +use datafusion_common::{exec_err, Result, ScalarValue}; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::{ColumnarValue, Expr, ScalarUDFImpl, Signature, Volatility}; @@ -51,7 +52,7 @@ fn can_evaluate_to_const(args: &[ColumnarValue]) -> bool { .all(|arg| matches!(arg, ColumnarValue::Scalar(_))) } -fn make_map_batch(args: &[ColumnarValue]) -> datafusion_common::Result { +fn make_map_batch(args: &[ColumnarValue]) -> Result { if args.len() != 2 { return exec_err!( "make_map requires exactly 2 arguments, got {} instead", @@ -59,16 +60,56 @@ fn make_map_batch(args: &[ColumnarValue]) -> datafusion_common::Result 0 { + return exec_err!("map key cannot be null"); + } + let key_array = keys.as_ref(); + + match &args[0] { + ColumnarValue::Array(_) => { + let row_keys = match key_array.data_type() { + DataType::List(_) => list_to_arrays::(&keys), + DataType::LargeList(_) => list_to_arrays::(&keys), + DataType::FixedSizeList(_, _) => fixed_size_list_to_arrays(&keys), + data_type => { + return exec_err!( + "Expected list, large_list or fixed_size_list, got {:?}", + data_type + ); + } + }; + + row_keys + .iter() + .try_for_each(|key| check_unique_keys(key.as_ref()))?; + } + ColumnarValue::Scalar(_) => { + check_unique_keys(key_array)?; + } + } + + let values = get_first_array_ref(&args[1])?; + make_map_batch_internal(keys, values, can_evaluate_to_const, args[0].data_type()) } -fn get_first_array_ref( - columnar_value: &ColumnarValue, -) -> datafusion_common::Result { +fn check_unique_keys(array: &dyn Array) -> Result<()> { + let mut seen_keys = HashSet::with_capacity(array.len()); + + for i in 0..array.len() { + let key = ScalarValue::try_from_array(array, i)?; + if seen_keys.contains(&key) { + return exec_err!("map key must be unique, duplicate key found: {}", key); + } + seen_keys.insert(key); + } + Ok(()) +} + +fn get_first_array_ref(columnar_value: &ColumnarValue) -> Result { match columnar_value { ColumnarValue::Scalar(value) => match value { ScalarValue::List(array) => Ok(array.value(0)), @@ -85,11 +126,7 @@ fn make_map_batch_internal( values: ArrayRef, can_evaluate_to_const: bool, data_type: DataType, -) -> datafusion_common::Result { - if keys.null_count() > 0 { - return exec_err!("map key cannot be null"); - } - +) -> Result { if keys.len() != values.len() { return exec_err!("map requires key and value lists to have the same length"); } @@ -173,7 +210,7 @@ impl ScalarUDFImpl for MapFunc { &self.signature } - fn return_type(&self, arg_types: &[DataType]) -> datafusion_common::Result { + fn return_type(&self, arg_types: &[DataType]) -> Result { if arg_types.len() % 2 != 0 { return exec_err!( "map requires an even number of arguments, got {} instead", @@ -198,11 +235,11 @@ impl ScalarUDFImpl for MapFunc { )) } - fn invoke(&self, args: &[ColumnarValue]) -> datafusion_common::Result { + fn invoke(&self, args: &[ColumnarValue]) -> Result { make_map_batch(args) } } -fn get_element_type(data_type: &DataType) -> datafusion_common::Result<&DataType> { +fn get_element_type(data_type: &DataType) -> Result<&DataType> { match data_type { DataType::List(element) => Ok(element.data_type()), DataType::LargeList(element) => Ok(element.data_type()), @@ -273,12 +310,12 @@ fn get_element_type(data_type: &DataType) -> datafusion_common::Result<&DataType fn make_map_array_internal( keys: ArrayRef, values: ArrayRef, -) -> datafusion_common::Result { +) -> Result { let mut offset_buffer = vec![O::zero()]; let mut running_offset = O::zero(); - let keys = datafusion_common::utils::list_to_arrays::(keys); - let values = datafusion_common::utils::list_to_arrays::(values); + let keys = list_to_arrays::(&keys); + let values = list_to_arrays::(&values); let mut key_array_vec = vec![]; let mut value_array_vec = vec![]; diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index c66334c4de2a..45e1b51a09b4 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -198,6 +198,45 @@ SELECT MAP(['POST', 'HEAD', 'PATCH'], [41, 33]); query error DataFusion error: Execution error: map key cannot be null SELECT MAP(['POST', 'HEAD', null], [41, 33, 30]); +statement error DataFusion error: Execution error: map key cannot be null +CREATE TABLE duplicated_keys_table +AS VALUES + (MAP {1: [1, NULL, 3], NULL: [4, NULL, 6]}); + +# Test duplicate keys +# key is a scalar type +query error DataFusion error: Execution error: map key must be unique, duplicate key found: POST +SELECT MAP(['POST', 'HEAD', 'POST'], [41, 33, null]); + +query error DataFusion error: Execution error: map key must be unique, duplicate key found: POST +SELECT MAP(make_array('POST', 'HEAD', 'POST'), make_array(41, 33, 30)); + +query error DataFusion error: Execution error: map key must be unique, duplicate key found: POST +SELECT make_map('POST', 41, 'HEAD', 33, 'POST', 30); + +statement error DataFusion error: Execution error: map key must be unique, duplicate key found: 1 +CREATE TABLE duplicated_keys_table +AS VALUES + (MAP {1: [1, NULL, 3], 1: [4, NULL, 6]}); + +statement ok +create table duplicate_keys_table as values +('a', 1, 'a', 10, ['k1', 'k1'], [1, 2]); + +query error DataFusion error: Execution error: map key must be unique, duplicate key found: a +SELECT make_map(column1, column2, column3, column4) FROM duplicate_keys_table; + +query error DataFusion error: Execution error: map key must be unique, duplicate key found: k1 +SELECT map(column5, column6) FROM duplicate_keys_table; + +# key is a nested type +query error DataFusion error: Execution error: map key must be unique, duplicate key found: \[1, 2\] +SELECT MAP([[1,2], [1,2], []], [41, 33, null]); + +query error DataFusion error: Execution error: map key must be unique, duplicate key found: \[\{1:1\}\] +SELECT MAP([Map {1:'1'}, Map {1:'1'}, Map {2:'2'}], [41, 33, null]); + + query ? SELECT MAP(make_array('POST', 'HEAD', 'PATCH'), make_array(41, 33, 30)); ---- From 1531c5d43b0327ec9b2b225ca7b3018b3b94342e Mon Sep 17 00:00:00 2001 From: Phillip LeBlanc Date: Wed, 4 Sep 2024 19:42:21 +0900 Subject: [PATCH 67/75] Update download page to reflect latest version (v41) (#12310) * Update download page to reflect latest version * Add note to update the download page --- dev/release/README.md | 4 ++++ docs/source/download.md | 8 ++++---- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/dev/release/README.md b/dev/release/README.md index 397369a41aa3..bd9c0621fdbc 100644 --- a/dev/release/README.md +++ b/dev/release/README.md @@ -407,3 +407,7 @@ Here is an example blog post PR: - https://github.com/apache/arrow-site/pull/217 Once the PR is merged, a GitHub action will publish the new blog post to https://arrow.apache.org/blog/. + +### Update the version on the download page + +Update the version on the [download page](https://datafusion.apache.org/download) to point to the latest release [here](../../docs/source/download.md). diff --git a/docs/source/download.md b/docs/source/download.md index 0c279dd033f2..33a6d7008877 100644 --- a/docs/source/download.md +++ b/docs/source/download.md @@ -26,12 +26,12 @@ official Apache DataFusion releases are provided as source artifacts. ## Releases -The latest source release is [37.0.0][source-link] ([asc][asc-link], +The latest source release is [41.0.0][source-link] ([asc][asc-link], [sha512][sha512-link]). -[source-link]: https://www.apache.org/dyn/closer.lua/arrow/arrow-datafusion-37.0.0/apache-arrow-datafusion-37.0.0.tar.gz?action=download -[asc-link]: https://downloads.apache.org/arrow/arrow-datafusion-37.0.0/apache-arrow-datafusion-37.0.0.tar.gz.asc -[sha512-link]: https://downloads.apache.org/arrow/arrow-datafusion-37.0.0/apache-arrow-datafusion-37.0.0.tar.gz.sha512 +[source-link]: https://www.apache.org/dyn/closer.lua/datafusion/datafusion-41.0.0/apache-datafusion-41.0.0.tar.gz?action=download +[asc-link]: https://downloads.apache.org/datafusion/datafusion-41.0.0/apache-datafusion-41.0.0.tar.gz.asc +[sha512-link]: https://downloads.apache.org/datafusion/datafusion-41.0.0/apache-datafusion-41.0.0.tar.gz.sha512 For previous releases, please check the [archive](https://archive.apache.org/dist/datafusion/). From e5a6cd5f6784f1d1b3d559d5356a6154a73e077c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BC=A0=E6=9E=97=E4=BC=9F?= Date: Wed, 4 Sep 2024 20:20:46 +0800 Subject: [PATCH 68/75] Improve binary scalars display (#12192) * Improve binary scalar display * Reduce allocation & add test * Display binary scalar as hex string --- datafusion/common/src/scalar/mod.rs | 79 ++++++++++++++++++++++++----- 1 file changed, 66 insertions(+), 13 deletions(-) diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 5acc2b6f188e..3cff0731dcee 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -3517,14 +3517,14 @@ impl fmt::Display for ScalarValue { | ScalarValue::FixedSizeBinary(_, e) | ScalarValue::LargeBinary(e) | ScalarValue::BinaryView(e) => match e { - Some(l) => write!( - f, - "{}", - l.iter() - .map(|v| format!("{v}")) - .collect::>() - .join(",") - )?, + Some(l) => { + let data = l.iter().map(|v| format!("{v:02X}")).collect::>(); + if data.len() > 10 { + write!(f, "{}...", data[..10].join(""))?; + } else { + write!(f, "{}", data.join(""))?; + } + } None => write!(f, "NULL")?, }, ScalarValue::List(arr) => fmt_list(arr.to_owned() as ArrayRef, f)?, @@ -3648,6 +3648,16 @@ fn fmt_list(arr: ArrayRef, f: &mut fmt::Formatter) -> fmt::Result { impl fmt::Debug for ScalarValue { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt_binary(data: &[u8], f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "{}", + data.iter() + .map(|v| format!("{v}")) + .collect::>() + .join(",") + ) + } match self { ScalarValue::Decimal128(_, _, _) => write!(f, "Decimal128({self})"), ScalarValue::Decimal256(_, _, _) => write!(f, "Decimal256({self})"), @@ -3682,17 +3692,31 @@ impl fmt::Debug for ScalarValue { ScalarValue::LargeUtf8(None) => write!(f, "LargeUtf8({self})"), ScalarValue::LargeUtf8(Some(_)) => write!(f, "LargeUtf8(\"{self}\")"), ScalarValue::Binary(None) => write!(f, "Binary({self})"), - ScalarValue::Binary(Some(_)) => write!(f, "Binary(\"{self}\")"), + ScalarValue::Binary(Some(b)) => { + write!(f, "Binary(\"")?; + fmt_binary(b.as_slice(), f)?; + write!(f, "\")") + } ScalarValue::BinaryView(None) => write!(f, "BinaryView({self})"), - ScalarValue::BinaryView(Some(_)) => write!(f, "BinaryView(\"{self}\")"), + ScalarValue::BinaryView(Some(b)) => { + write!(f, "BinaryView(\"")?; + fmt_binary(b.as_slice(), f)?; + write!(f, "\")") + } ScalarValue::FixedSizeBinary(size, None) => { write!(f, "FixedSizeBinary({size}, {self})") } - ScalarValue::FixedSizeBinary(size, Some(_)) => { - write!(f, "FixedSizeBinary({size}, \"{self}\")") + ScalarValue::FixedSizeBinary(size, Some(b)) => { + write!(f, "Binary({size}, \"")?; + fmt_binary(b.as_slice(), f)?; + write!(f, "\")") } ScalarValue::LargeBinary(None) => write!(f, "LargeBinary({self})"), - ScalarValue::LargeBinary(Some(_)) => write!(f, "LargeBinary(\"{self}\")"), + ScalarValue::LargeBinary(Some(b)) => { + write!(f, "LargeBinary(\"")?; + fmt_binary(b.as_slice(), f)?; + write!(f, "\")") + } ScalarValue::FixedSizeList(_) => write!(f, "FixedSizeList({self})"), ScalarValue::List(_) => write!(f, "List({self})"), ScalarValue::LargeList(_) => write!(f, "LargeList({self})"), @@ -6518,6 +6542,35 @@ mod tests { assert_batches_eq!(&expected, &[batch]); } + #[test] + fn test_binary_display() { + let small_binary_value = ScalarValue::Binary(Some(vec![1u8, 2, 3])); + assert_eq!(format!("{small_binary_value}"), "010203"); + let large_binary_value = + ScalarValue::Binary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); + assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + + let small_binary_value = ScalarValue::BinaryView(Some(vec![1u8, 2, 3])); + assert_eq!(format!("{small_binary_value}"), "010203"); + let large_binary_value = + ScalarValue::BinaryView(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); + assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + + let small_binary_value = ScalarValue::LargeBinary(Some(vec![1u8, 2, 3])); + assert_eq!(format!("{small_binary_value}"), "010203"); + let large_binary_value = + ScalarValue::LargeBinary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); + assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + + let small_binary_value = ScalarValue::FixedSizeBinary(3, Some(vec![1u8, 2, 3])); + assert_eq!(format!("{small_binary_value}"), "010203"); + let large_binary_value = ScalarValue::FixedSizeBinary( + 11, + Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]), + ); + assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + } + #[test] fn test_build_timestamp_millisecond_list() { let values = vec![ScalarValue::TimestampMillisecond(Some(1), None)]; From e6caf6206b516fe97fc2d2962900030d1782e7d5 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 4 Sep 2024 12:25:43 -0400 Subject: [PATCH 69/75] Minor: reduce allocations in ScalarValue::*Binary display (#12322) --- datafusion/common/src/scalar/mod.rs | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 3cff0731dcee..88802af23a4b 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -3517,12 +3517,13 @@ impl fmt::Display for ScalarValue { | ScalarValue::FixedSizeBinary(_, e) | ScalarValue::LargeBinary(e) | ScalarValue::BinaryView(e) => match e { - Some(l) => { - let data = l.iter().map(|v| format!("{v:02X}")).collect::>(); - if data.len() > 10 { - write!(f, "{}...", data[..10].join(""))?; - } else { - write!(f, "{}", data.join(""))?; + Some(bytes) => { + // print up to first 10 bytes, with trailing ... if needed + for b in bytes.iter().take(10) { + write!(f, "{b:02X}")?; + } + if bytes.len() > 10 { + write!(f, "...")?; } } None => write!(f, "NULL")?, @@ -6544,24 +6545,32 @@ mod tests { #[test] fn test_binary_display() { + let no_binary_value = ScalarValue::Binary(None); + assert_eq!(format!("{no_binary_value}"), "NULL"); let small_binary_value = ScalarValue::Binary(Some(vec![1u8, 2, 3])); assert_eq!(format!("{small_binary_value}"), "010203"); let large_binary_value = ScalarValue::Binary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + let no_binary_value = ScalarValue::BinaryView(None); + assert_eq!(format!("{no_binary_value}"), "NULL"); let small_binary_value = ScalarValue::BinaryView(Some(vec![1u8, 2, 3])); assert_eq!(format!("{small_binary_value}"), "010203"); let large_binary_value = ScalarValue::BinaryView(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + let no_binary_value = ScalarValue::LargeBinary(None); + assert_eq!(format!("{no_binary_value}"), "NULL"); let small_binary_value = ScalarValue::LargeBinary(Some(vec![1u8, 2, 3])); assert_eq!(format!("{small_binary_value}"), "010203"); let large_binary_value = ScalarValue::LargeBinary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11])); assert_eq!(format!("{large_binary_value}"), "0102030405060708090A..."); + let no_binary_value = ScalarValue::FixedSizeBinary(3, None); + assert_eq!(format!("{no_binary_value}"), "NULL"); let small_binary_value = ScalarValue::FixedSizeBinary(3, Some(vec![1u8, 2, 3])); assert_eq!(format!("{small_binary_value}"), "010203"); let large_binary_value = ScalarValue::FixedSizeBinary( From 91b1d2bfe8f603df94e846b91d8475a0af2e5240 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 4 Sep 2024 10:27:21 -0600 Subject: [PATCH 70/75] Add PartialEq, Eq traits to StatsType (#12327) --- datafusion/functions-aggregate-common/src/stats.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/functions-aggregate-common/src/stats.rs b/datafusion/functions-aggregate-common/src/stats.rs index 6a11ebe36c5f..bcd004db7831 100644 --- a/datafusion/functions-aggregate-common/src/stats.rs +++ b/datafusion/functions-aggregate-common/src/stats.rs @@ -17,7 +17,7 @@ /// TODO: Move this to functions-aggregate module /// Enum used for differentiating population and sample for statistical functions -#[derive(Debug, Clone, Copy)] +#[derive(PartialEq, Eq, Debug, Clone, Copy)] pub enum StatsType { /// Population Population, From 6034be42808b43e3f48f6e58ec38cc35fa253abb Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 5 Sep 2024 06:05:40 -0400 Subject: [PATCH 71/75] Update to `arrow`/`parquet` `53.0.0`, `tonic`, `prost`, `object_store`, `pyo3` (#12032) * Update prost, prost-derive, pbjson * udpate more * Update datafusion/substrait/Cargo.toml Co-authored-by: tison * Update vendored code * revert upgrade in datafusion-examples until arrow-flight is updated * Pin to pre-release arrow-rs * update pyo3 * Update to use new arrow apis * update for new api * Update tonic in examples * update prost * update datafusion-cli/cargo * update test output * update * updates * updates * update math * update more * fix scalar tests * Port statistics to use new API * factor into a function * update generated files * Update test * add new test * update tests * tapelo format * Update other tests * Update datafusion pin * Update for API change * Update to arrow 53.0.0 sha * Update cli deps * update cargo.lock * Update expected output * Remove patch * update datafusion-cli cargo * Pin some aws sdks whose update caused CI failures --------- Co-authored-by: tison --- Cargo.toml | 24 +- datafusion-cli/Cargo.lock | 216 +++++++++--------- datafusion-cli/Cargo.toml | 13 +- datafusion-cli/src/functions.rs | 113 +++++---- datafusion-examples/Cargo.toml | 4 +- datafusion/common/Cargo.toml | 2 +- datafusion/common/src/scalar/mod.rs | 6 +- .../src/datasource/file_format/parquet.rs | 2 +- .../physical_plan/parquet/page_filter.rs | 5 +- .../physical_plan/parquet/row_group_filter.rs | 150 +++++++++--- .../functions/src/regex/regexpreplace.rs | 3 +- .../src/binary_view_map.rs | 2 +- datafusion/physical-plan/src/coalesce/mod.rs | 5 +- datafusion/proto-common/Cargo.toml | 2 +- datafusion/proto-common/gen/Cargo.toml | 4 +- datafusion/proto/Cargo.toml | 2 +- datafusion/proto/gen/Cargo.toml | 4 +- .../src/generated/datafusion_proto_common.rs | 54 ++--- datafusion/proto/src/generated/pbjson.rs | 31 +++ datafusion/proto/src/generated/prost.rs | 14 +- datafusion/sql/src/unparser/expr.rs | 18 +- datafusion/sql/tests/cases/plan_to_sql.rs | 4 +- .../sqllogictest/test_files/aggregate.slt | 6 +- .../sqllogictest/test_files/arrow_typeof.slt | 12 +- datafusion/sqllogictest/test_files/ddl.slt | 2 +- datafusion/sqllogictest/test_files/expr.slt | 80 +++---- .../sqllogictest/test_files/interval.slt | 102 ++++----- datafusion/sqllogictest/test_files/math.slt | 2 +- .../test_files/repartition_scan.slt | 8 +- .../sqllogictest/test_files/timestamps.slt | 16 +- 30 files changed, 526 insertions(+), 380 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 877cead93673..c155e475a026 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -69,22 +69,22 @@ version = "41.0.0" ahash = { version = "0.8", default-features = false, features = [ "runtime-rng", ] } -arrow = { version = "52.2.0", features = [ +arrow = { version = "53.0.0", features = [ "prettyprint", ] } -arrow-array = { version = "52.2.0", default-features = false, features = [ +arrow-array = { version = "53.0.0", default-features = false, features = [ "chrono-tz", ] } -arrow-buffer = { version = "52.2.0", default-features = false } -arrow-flight = { version = "52.2.0", features = [ +arrow-buffer = { version = "53.0.0", default-features = false } +arrow-flight = { version = "53.0.0", features = [ "flight-sql-experimental", ] } -arrow-ipc = { version = "52.2.0", default-features = false, features = [ +arrow-ipc = { version = "53.0.0", default-features = false, features = [ "lz4", ] } -arrow-ord = { version = "52.2.0", default-features = false } -arrow-schema = { version = "52.2.0", default-features = false } -arrow-string = { version = "52.2.0", default-features = false } +arrow-ord = { version = "53.0.0", default-features = false } +arrow-schema = { version = "53.0.0", default-features = false } +arrow-string = { version = "53.0.0", default-features = false } async-trait = "0.1.73" bigdecimal = "=0.4.1" bytes = "1.4" @@ -122,15 +122,17 @@ indexmap = "2.0.0" itertools = "0.13" log = "^0.4" num_cpus = "1.13.0" -object_store = { version = "0.10.2", default-features = false } +object_store = { version = "0.11.0", default-features = false } parking_lot = "0.12" -parquet = { version = "52.2.0", default-features = false, features = [ +parquet = { version = "53.0.0", default-features = false, features = [ "arrow", "async", "object_store", ] } +pbjson = { version = "0.7.0" } # Should match arrow-flight's version of prost. -prost = "0.12.3" +prost = "0.13.1" +prost-derive = "0.13.1" rand = "0.8" regex = "1.8" rstest = "0.22.0" diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index ddc6242977d3..039f3fb9a6aa 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -179,9 +179,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05048a8932648b63f21c37d88b552ccc8a65afb6dfe9fc9f30ce79174c2e7a85" +checksum = "45aef0d9cf9a039bf6cd1acc451b137aca819977b0928dece52bd92811b640ba" dependencies = [ "arrow-arith", "arrow-array", @@ -200,9 +200,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d8a57966e43bfe9a3277984a14c24ec617ad874e4c0e1d2a1b083a39cfbf22c" +checksum = "03675e42d1560790f3524800e41403b40d0da1c793fe9528929fde06d8c7649a" dependencies = [ "arrow-array", "arrow-buffer", @@ -215,9 +215,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16f4a9468c882dc66862cef4e1fd8423d47e67972377d85d80e022786427768c" +checksum = "cd2bf348cf9f02a5975c5962c7fa6dee107a2009a7b41ac5fb1a027e12dc033f" dependencies = [ "ahash", "arrow-buffer", @@ -232,9 +232,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c975484888fc95ec4a632cdc98be39c085b1bb518531b0c80c5d462063e5daa1" +checksum = "3092e37715f168976012ce52273c3989b5793b0db5f06cbaa246be25e5f0924d" dependencies = [ "bytes", "half", @@ -243,9 +243,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da26719e76b81d8bc3faad1d4dbdc1bcc10d14704e63dc17fc9f3e7e1e567c8e" +checksum = "7ce1018bb710d502f9db06af026ed3561552e493e989a79d0d0f5d9cf267a785" dependencies = [ "arrow-array", "arrow-buffer", @@ -264,9 +264,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c13c36dc5ddf8c128df19bab27898eea64bf9da2b555ec1cd17a8ff57fba9ec2" +checksum = "fd178575f45624d045e4ebee714e246a05d9652e41363ee3f57ec18cca97f740" dependencies = [ "arrow-array", "arrow-buffer", @@ -283,9 +283,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd9d6f18c65ef7a2573ab498c374d8ae364b4a4edf67105357491c031f716ca5" +checksum = "4e4ac0c4ee79150afe067dc4857154b3ee9c1cd52b5f40d59a77306d0ed18d65" dependencies = [ "arrow-buffer", "arrow-schema", @@ -295,9 +295,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e786e1cdd952205d9a8afc69397b317cfbb6e0095e445c69cda7e8da5c1eeb0f" +checksum = "bb307482348a1267f91b0912e962cd53440e5de0f7fb24c5f7b10da70b38c94a" dependencies = [ "arrow-array", "arrow-buffer", @@ -310,9 +310,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fb22284c5a2a01d73cebfd88a33511a3234ab45d66086b2ca2d1228c3498e445" +checksum = "d24805ba326758effdd6f2cbdd482fcfab749544f21b134701add25b33f474e6" dependencies = [ "arrow-array", "arrow-buffer", @@ -330,9 +330,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42745f86b1ab99ef96d1c0bcf49180848a64fe2c7a7a0d945bc64fa2b21ba9bc" +checksum = "644046c479d80ae8ed02a7f1e1399072ea344ca6a7b0e293ab2d5d9ed924aa3b" dependencies = [ "arrow-array", "arrow-buffer", @@ -345,9 +345,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4cd09a518c602a55bd406bcc291a967b284cfa7a63edfbf8b897ea4748aad23c" +checksum = "a29791f8eb13b340ce35525b723f5f0df17ecb955599e11f65c2a94ab34e2efb" dependencies = [ "ahash", "arrow-array", @@ -359,15 +359,15 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e972cd1ff4a4ccd22f86d3e53e835c2ed92e0eea6a3e8eadb72b4f1ac802cf8" +checksum = "c85320a3a2facf2b2822b57aa9d6d9d55edb8aee0b6b5d3b8df158e503d10858" [[package]] name = "arrow-select" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "600bae05d43483d216fb3494f8c32fdbefd8aa4e1de237e790dbb3d9f44690a3" +checksum = "9cc7e6b582e23855fd1625ce46e51647aa440c20ea2e71b1d748e0839dd73cba" dependencies = [ "ahash", "arrow-array", @@ -379,9 +379,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0dc1985b67cb45f6606a248ac2b4a288849f196bab8c657ea5589f47cdd55e6" +checksum = "0775b6567c66e56ded19b87a954b6b1beffbdd784ef95a3a2b03f59570c1d230" dependencies = [ "arrow-array", "arrow-buffer", @@ -430,13 +430,13 @@ dependencies = [ [[package]] name = "async-trait" -version = "0.1.81" +version = "0.1.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e0c28dcc82d7c8ead5cb13beb15405b57b8546e93215673ff8ca0349a028107" +checksum = "a27b8a3a6e1a44fa4c8baf1f653e4172e81486d4941f2237e20dc2d0cf4ddff1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -503,9 +503,9 @@ dependencies = [ [[package]] name = "aws-credential-types" -version = "1.2.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e16838e6c9e12125face1c1eff1343c75e3ff540de98ff7ebd61874a89bcfeb9" +checksum = "60e8f6b615cb5fc60a98132268508ad104310f0cfb25a1c22eee76efdf9154da" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -515,14 +515,15 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.4.0" +version = "1.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f42c2d4218de4dcd890a109461e2f799a1a2ba3bcd2cde9af88360f5df9266c6" +checksum = "2424565416eef55906f9f8cece2072b6b6a76075e3ff81483ebe938a89a4c05f" dependencies = [ "aws-credential-types", "aws-sigv4", "aws-smithy-async", "aws-smithy-http", + "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", "aws-types", @@ -730,6 +731,7 @@ dependencies = [ "base64-simd", "bytes", "bytes-utils", + "futures-core", "http 0.2.12", "http 1.1.0", "http-body 0.4.6", @@ -742,6 +744,8 @@ dependencies = [ "ryu", "serde", "time", + "tokio", + "tokio-util", ] [[package]] @@ -930,9 +934,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.1.13" +version = "1.1.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72db2f7947ecee9b03b510377e8bb9077afa27176fdbff55c51027e976fdcc48" +checksum = "57b6a275aa2903740dc87da01c62040406b8812552e97129a63ea8850a17c6e6" dependencies = [ "jobserver", "libc", @@ -1011,7 +1015,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -1070,9 +1074,9 @@ dependencies = [ [[package]] name = "constant_time_eq" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7144d30dcf0fafbce74250a3963025d8d52177934239851c917d29f1df280c2" +checksum = "7c74b8349d32d297c9134b8c88677813a227df8f779daa29bfc29c183fe3dca6" [[package]] name = "core-foundation" @@ -1167,7 +1171,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "edb49164822f3ee45b17acd4a208cfc1251410cf0cad9a833234c9890774dd9f" dependencies = [ "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -1268,6 +1272,9 @@ dependencies = [ "async-trait", "aws-config", "aws-credential-types", + "aws-sdk-sso", + "aws-sdk-ssooidc", + "aws-sdk-sts", "clap", "ctor", "datafusion", @@ -1697,9 +1704,9 @@ dependencies = [ [[package]] name = "fastrand" -version = "2.1.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fc0510504f03c51ada170672ac806f1f105a88aa97a5281117e1ddc3368e51a" +checksum = "e8c02a5121d4ea3eb16a80748c74f5549a5665e4c21333c6098f283870fbdea6" [[package]] name = "fd-lock" @@ -1730,9 +1737,9 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.32" +version = "1.0.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c0596c1eac1f9e04ed902702e9878208b336edc9d6fddc8a48387349bab3666" +checksum = "324a1be68054ef05ad64b861cc9eaf1d623d2d8cb25b4bf2cb9cdd902b4bf253" dependencies = [ "crc32fast", "miniz_oxide 0.8.0", @@ -1818,7 +1825,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -2136,7 +2143,7 @@ dependencies = [ "hyper 1.4.1", "hyper-util", "rustls 0.23.12", - "rustls-native-certs 0.7.2", + "rustls-native-certs 0.7.3", "rustls-pki-types", "tokio", "tokio-rustls 0.26.0", @@ -2198,9 +2205,9 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.4.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93ead53efc7ea8ed3cfb0c79fc8023fbb782a5432b52830b6518941cebe6505c" +checksum = "68b900aa2f7301e21c36462b170ee99994de34dff39a4a6a528e80e7376d07e5" dependencies = [ "equivalent", "hashbrown", @@ -2616,18 +2623,18 @@ dependencies = [ [[package]] name = "object" -version = "0.36.3" +version = "0.36.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27b64972346851a39438c60b341ebc01bba47464ae329e55cf343eb93964efd9" +checksum = "084f1a5821ac4c651660a94a7153d27ac9d8a53736203f58b31945ded098070a" dependencies = [ "memchr", ] [[package]] name = "object_store" -version = "0.10.2" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6da452820c715ce78221e8202ccc599b4a52f3e1eb3eedb487b680c81a8e3f3" +checksum = "25a0c4b3a0e31f8b66f71ad8064521efa773910196e2cde791436f13409f3b45" dependencies = [ "async-trait", "base64 0.22.1", @@ -2706,9 +2713,9 @@ dependencies = [ [[package]] name = "parquet" -version = "52.2.0" +version = "53.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e977b9066b4d3b03555c22bdc442f3fadebd96a39111249113087d0edb2691cd" +checksum = "f0fbf928021131daaa57d334ca8e3904fe9ae22f73c56244fc7db9b04eedc3d8" dependencies = [ "ahash", "arrow-array", @@ -2826,7 +2833,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -2919,9 +2926,9 @@ dependencies = [ [[package]] name = "quinn" -version = "0.11.3" +version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b22d8e7369034b9a7132bc2008cac12f2013c8132b45e0554e6e20e2617f2156" +checksum = "8c7c5fdde3cdae7203427dc4f0a68fe0ed09833edc525a03456b153b79828684" dependencies = [ "bytes", "pin-project-lite", @@ -2937,9 +2944,9 @@ dependencies = [ [[package]] name = "quinn-proto" -version = "0.11.6" +version = "0.11.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba92fb39ec7ad06ca2582c0ca834dfeadcaf06ddfc8e635c80aa7e1c05315fdd" +checksum = "fadfaed2cd7f389d0161bb73eeb07b7b78f8691047a6f3e73caaeae55310a4a6" dependencies = [ "bytes", "rand", @@ -2954,22 +2961,22 @@ dependencies = [ [[package]] name = "quinn-udp" -version = "0.5.4" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bffec3605b73c6f1754535084a85229fa8a30f86014e6c81aeec4abb68b0285" +checksum = "4fe68c2e9e1a1234e218683dbdf9f9dfcb094113c5ac2b938dfcb9bab4c4140b" dependencies = [ "libc", "once_cell", "socket2", "tracing", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] [[package]] name = "quote" -version = "1.0.36" +version = "1.0.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fa76aaf39101c457836aec0ce2316dbdc3ab723cdda1c6bd4e6ad4208acaca7" +checksum = "b5b9d34b8991d19d98081b46eacdd8eb58c6f2b201139f7c5f643cc155a633af" dependencies = [ "proc-macro2", ] @@ -3095,7 +3102,7 @@ dependencies = [ "pin-project-lite", "quinn", "rustls 0.23.12", - "rustls-native-certs 0.7.2", + "rustls-native-certs 0.7.3", "rustls-pemfile 2.1.3", "rustls-pki-types", "serde", @@ -3175,18 +3182,18 @@ checksum = "583034fd73374156e66797ed8e5b0d5690409c9226b22d87cb7f19821c05d152" [[package]] name = "rustc_version" -version = "0.4.0" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bfa0f585226d2e68097d4f95d113b15b83a82e819ab25717ec0590d9584ef366" +checksum = "cfcb3a22ef46e85b45de6ee7e79d063319ebb6594faafcf1c225ea92ab6e9b92" dependencies = [ "semver", ] [[package]] name = "rustix" -version = "0.38.34" +version = "0.38.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70dc5ec042f7a43c4a73241207cecc9873a06d45debb38b329f8541d85c2730f" +checksum = "a85d50532239da68e9addb745ba38ff4612a242c1c7ceea689c4bc7c2f43c36f" dependencies = [ "bitflags 2.6.0", "errno", @@ -3216,7 +3223,7 @@ dependencies = [ "once_cell", "ring", "rustls-pki-types", - "rustls-webpki 0.102.6", + "rustls-webpki 0.102.7", "subtle", "zeroize", ] @@ -3235,9 +3242,9 @@ dependencies = [ [[package]] name = "rustls-native-certs" -version = "0.7.2" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04182dffc9091a404e0fc069ea5cd60e5b866c3adf881eff99a32d048242dffa" +checksum = "e5bfb394eeed242e909609f56089eecfe5fda225042e8b171791b9c95f5931e5" dependencies = [ "openssl-probe", "rustls-pemfile 2.1.3", @@ -3283,9 +3290,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.102.6" +version = "0.102.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e6b52d4fda176fd835fdc55a835d4a89b8499cad995885a21149d5ad62f852e" +checksum = "84678086bd54edf2b415183ed7a94d0efb049f1b646a33e22a36f3794be6ae56" dependencies = [ "ring", "rustls-pki-types", @@ -3398,29 +3405,29 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.208" +version = "1.0.209" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cff085d2cb684faa248efb494c39b68e522822ac0de72ccf08109abde717cfb2" +checksum = "99fce0ffe7310761ca6bf9faf5115afbc19688edd00171d81b1bb1b116c63e09" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.208" +version = "1.0.209" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24008e81ff7613ed8e5ba0cfaf24e2c2f1e5b8a0495711e44fcd4882fca62bcf" +checksum = "a5831b979fd7b5439637af1752d535ff49f4860c0f341d1baeb6faf0f4242170" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] name = "serde_json" -version = "1.0.125" +version = "1.0.127" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83c8e735a073ccf5be70aa8066aa984eaf2fa000db6c8d0100ae605b366d31ed" +checksum = "8043c06d9f82bd7271361ed64f415fe5e12a77fdb52e573e7f06a516dea329ad" dependencies = [ "itoa", "memchr", @@ -3489,24 +3496,23 @@ checksum = "3c5e1a9a646d36c3599cd173a41282daf47c44583ad367b8e6837255952e5c67" [[package]] name = "snafu" -version = "0.7.5" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4de37ad025c587a29e8f3f5605c00f70b98715ef90b9061a815b9e59e9042d6" +checksum = "2b835cb902660db3415a672d862905e791e54d306c6e8189168c7f3d9ae1c79d" dependencies = [ - "doc-comment", "snafu-derive", ] [[package]] name = "snafu-derive" -version = "0.7.5" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "990079665f075b699031e9c08fd3ab99be5029b96f3b78dc0709e8f77e4efebf" +checksum = "38d1e02fca405f6280643174a50c942219f0bbf4dbf7d480f1dd864d6f211ae5" dependencies = [ - "heck 0.4.1", + "heck 0.5.0", "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.77", ] [[package]] @@ -3549,7 +3555,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -3595,7 +3601,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -3608,7 +3614,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -3630,9 +3636,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.75" +version = "2.0.77" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6af063034fc1935ede7be0122941bafa9bacb949334d090b77ca98b5817c7d9" +checksum = "9f35bcdf61fd8e7be6caf75f429fdca8beb3ed76584befb503b1569faee373ed" dependencies = [ "proc-macro2", "quote", @@ -3693,7 +3699,7 @@ checksum = "a4558b58466b9ad7ca0f102865eccc95938dca1a74a856f2b57b6629050da261" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -3763,9 +3769,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.39.3" +version = "1.40.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9babc99b9923bfa4804bd74722ff02c0381021eafa4db9949217e3be8e84fff5" +checksum = "e2b070231665d27ad9ec9b8df639893f46727666c6767db40317fbe920a5d998" dependencies = [ "backtrace", "bytes", @@ -3787,7 +3793,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -3870,7 +3876,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -3915,7 +3921,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -4064,7 +4070,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", "wasm-bindgen-shared", ] @@ -4098,7 +4104,7 @@ checksum = "afc340c74d9005395cf9dd098506f7f44e38f2b4a21c6aaacf9a105ea5e1e836" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -4383,7 +4389,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.75", + "syn 2.0.77", ] [[package]] @@ -4431,9 +4437,9 @@ dependencies = [ [[package]] name = "zstd-sys" -version = "2.0.12+zstd.1.5.6" +version = "2.0.13+zstd.1.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a4e40c320c3cb459d9a9ff6de98cff88f4751ee9275d140e2be94a2b74e4c13" +checksum = "38ff0f21cfee8f97d94cef41359e0c89aa6113028ab0291aa8ca0038995a95aa" dependencies = [ "cc", "pkg-config", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index f477bad69a2c..f2f52846ab54 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -30,9 +30,16 @@ rust-version = "1.76" readme = "README.md" [dependencies] -arrow = { version = "52.2.0" } +arrow = { version = "53.0.0" } async-trait = "0.1.73" aws-config = "1.5.5" +# begin pin aws-sdk crates otherwise CI MSRV check fails +# We can't update these libraries yet as it requires Rust 1.78, which is not available until Nov 2024 +# per https://github.com/apache/datafusion?tab=readme-ov-file#rust-version-compatibility-policy +aws-sdk-sso = "=1.39.0" +aws-sdk-ssooidc = "=1.40.0" +aws-sdk-sts = "=1.39.0" +# end pin aws-sdk crates aws-credential-types = "1.2.0" clap = { version = "4.5.16", features = ["derive", "cargo"] } datafusion = { path = "../datafusion/core", version = "41.0.0", features = [ @@ -49,9 +56,9 @@ dirs = "4.0.0" env_logger = "0.9" futures = "0.3" mimalloc = { version = "0.1", default-features = false } -object_store = { version = "0.10.1", features = ["aws", "gcp", "http"] } +object_store = { version = "0.11.0", features = ["aws", "gcp", "http"] } parking_lot = { version = "0.12" } -parquet = { version = "52.2.0", default-features = false } +parquet = { version = "53.0.0", default-features = false } regex = "1.8" rustyline = "11.0" tokio = { version = "1.24", features = ["macros", "rt", "rt-multi-thread", "sync", "parking_lot", "signal"] } diff --git a/datafusion-cli/src/functions.rs b/datafusion-cli/src/functions.rs index a85c43f3576f..99511e969386 100644 --- a/datafusion-cli/src/functions.rs +++ b/datafusion-cli/src/functions.rs @@ -32,6 +32,7 @@ use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::ExecutionPlan; use datafusion::scalar::ScalarValue; use parquet::basic::ConvertedType; +use parquet::data_type::{ByteArray, FixedLenByteArray}; use parquet::file::reader::FileReader; use parquet::file::serialized_reader::SerializedFileReader; use parquet::file::statistics::Statistics; @@ -250,49 +251,69 @@ impl TableProvider for ParquetMetadataTable { fn convert_parquet_statistics( value: &Statistics, converted_type: ConvertedType, -) -> (String, String) { +) -> (Option, Option) { match (value, converted_type) { - (Statistics::Boolean(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::Int32(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::Int64(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::Int96(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::Float(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::Double(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::ByteArray(val), ConvertedType::UTF8) => { - let min_bytes = val.min(); - let max_bytes = val.max(); - let min = min_bytes - .as_utf8() - .map(|v| v.to_string()) - .unwrap_or_else(|_| min_bytes.to_string()); - - let max = max_bytes - .as_utf8() - .map(|v| v.to_string()) - .unwrap_or_else(|_| max_bytes.to_string()); - (min, max) - } - (Statistics::ByteArray(val), _) => (val.min().to_string(), val.max().to_string()), - (Statistics::FixedLenByteArray(val), ConvertedType::UTF8) => { - let min_bytes = val.min(); - let max_bytes = val.max(); - let min = min_bytes - .as_utf8() - .map(|v| v.to_string()) - .unwrap_or_else(|_| min_bytes.to_string()); - - let max = max_bytes - .as_utf8() - .map(|v| v.to_string()) - .unwrap_or_else(|_| max_bytes.to_string()); - (min, max) - } - (Statistics::FixedLenByteArray(val), _) => { - (val.min().to_string(), val.max().to_string()) - } + (Statistics::Boolean(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::Int32(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::Int64(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::Int96(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::Float(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::Double(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::ByteArray(val), ConvertedType::UTF8) => ( + byte_array_to_string(val.min_opt()), + byte_array_to_string(val.max_opt()), + ), + (Statistics::ByteArray(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), + (Statistics::FixedLenByteArray(val), ConvertedType::UTF8) => ( + fixed_len_byte_array_to_string(val.min_opt()), + fixed_len_byte_array_to_string(val.max_opt()), + ), + (Statistics::FixedLenByteArray(val), _) => ( + val.min_opt().map(|v| v.to_string()), + val.max_opt().map(|v| v.to_string()), + ), } } +/// Convert to a string if it has utf8 encoding, otherwise print bytes directly +fn byte_array_to_string(val: Option<&ByteArray>) -> Option { + val.map(|v| { + v.as_utf8() + .map(|s| s.to_string()) + .unwrap_or_else(|_e| v.to_string()) + }) +} + +/// Convert to a string if it has utf8 encoding, otherwise print bytes directly +fn fixed_len_byte_array_to_string(val: Option<&FixedLenByteArray>) -> Option { + val.map(|v| { + v.as_utf8() + .map(|s| s.to_string()) + .unwrap_or_else(|_e| v.to_string()) + }) +} + pub struct ParquetMetadataFunc {} impl TableFunctionImpl for ParquetMetadataFunc { @@ -376,17 +397,13 @@ impl TableFunctionImpl for ParquetMetadataFunc { let converted_type = column.column_descr().converted_type(); if let Some(s) = column.statistics() { - let (min_val, max_val) = if s.has_min_max_set() { - let (min_val, max_val) = - convert_parquet_statistics(s, converted_type); - (Some(min_val), Some(max_val)) - } else { - (None, None) - }; + let (min_val, max_val) = + convert_parquet_statistics(s, converted_type); stats_min_arr.push(min_val.clone()); stats_max_arr.push(max_val.clone()); - stats_null_count_arr.push(Some(s.null_count() as i64)); - stats_distinct_count_arr.push(s.distinct_count().map(|c| c as i64)); + stats_null_count_arr.push(s.null_count_opt().map(|c| c as i64)); + stats_distinct_count_arr + .push(s.distinct_count_opt().map(|c| c as i64)); stats_min_value_arr.push(min_val); stats_max_value_arr.push(max_val); } else { diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 57b9930177d4..f430a87e190d 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -73,13 +73,13 @@ mimalloc = { version = "0.1", default-features = false } num_cpus = { workspace = true } object_store = { workspace = true, features = ["aws", "http"] } prost = { workspace = true } -prost-derive = { version = "0.13", default-features = false } +prost-derive = { workspace = true } serde = { version = "1.0.136", features = ["derive"] } serde_json = { workspace = true } tempfile = { workspace = true } test-utils = { path = "../test-utils" } tokio = { workspace = true, features = ["rt-multi-thread", "parking_lot"] } -tonic = "0.11" +tonic = "0.12.1" url = { workspace = true } uuid = "1.7" diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 79e20ba1215c..1ac27b40c219 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -61,7 +61,7 @@ num_cpus = { workspace = true } object_store = { workspace = true, optional = true } parquet = { workspace = true, optional = true, default-features = true } paste = "1.0.15" -pyo3 = { version = "0.21.0", optional = true } +pyo3 = { version = "0.22.0", optional = true } sqlparser = { workspace = true } tokio = { workspace = true } diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 88802af23a4b..22e39404cdb5 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -4356,7 +4356,7 @@ mod tests { .strip_backtrace(); assert_eq!( err, - "Arrow error: Compute error: Overflow happened on: 2147483647 - -2147483648" + "Arrow error: Arithmetic overflow: Overflow happened on: 2147483647 - -2147483648" ) } @@ -4377,7 +4377,7 @@ mod tests { .sub_checked(&int_value_2) .unwrap_err() .strip_backtrace(); - assert_eq!(err, "Arrow error: Compute error: Overflow happened on: 9223372036854775807 - -9223372036854775808") + assert_eq!(err, "Arrow error: Arithmetic overflow: Overflow happened on: 9223372036854775807 - -9223372036854775808") } #[test] @@ -5893,7 +5893,7 @@ mod tests { let root_err = err.find_root(); match root_err{ DataFusionError::ArrowError( - ArrowError::ComputeError(_), + ArrowError::ArithmeticOverflow(_), _, ) => {} _ => return Err(err), diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 23e765f0f2cd..76e8ad9da559 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -2010,7 +2010,7 @@ mod tests { // test result in int_col let int_col_index = page_index.get(4).unwrap(); - let int_col_offset = offset_index.get(4).unwrap(); + let int_col_offset = offset_index.get(4).unwrap().page_locations(); // 325 pages in int_col assert_eq!(int_col_offset.len(), 325); 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 e4d26a460ecd..4e71993b5153 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs @@ -392,13 +392,16 @@ impl<'a> PagesPruningStatistics<'a> { trace!("No page offsets for row group {row_group_index}, skipping"); return None; }; - let Some(page_offsets) = row_group_page_offsets.get(parquet_column_index) else { + let Some(offset_index_metadata) = + row_group_page_offsets.get(parquet_column_index) + else { trace!( "No page offsets for column {:?} in row group {row_group_index}, skipping", converter.arrow_field() ); return None; }; + let page_offsets = offset_index_metadata.page_locations(); Some(Self { row_group_index, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs index 6a6910748fc8..ccd77d90be57 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs @@ -487,11 +487,23 @@ mod tests { let schema_descr = get_test_schema_descr(vec![field]); let rgm1 = get_row_group_meta_data( &schema_descr, - vec![ParquetStatistics::int32(Some(1), Some(10), None, 0, false)], + vec![ParquetStatistics::int32( + Some(1), + Some(10), + None, + Some(0), + false, + )], ); let rgm2 = get_row_group_meta_data( &schema_descr, - vec![ParquetStatistics::int32(Some(11), Some(20), None, 0, false)], + vec![ParquetStatistics::int32( + Some(11), + Some(20), + None, + Some(0), + false, + )], ); let metrics = parquet_file_metrics(); @@ -520,11 +532,17 @@ mod tests { let schema_descr = get_test_schema_descr(vec![field]); let rgm1 = get_row_group_meta_data( &schema_descr, - vec![ParquetStatistics::int32(None, None, None, 0, false)], + vec![ParquetStatistics::int32(None, None, None, Some(0), false)], ); let rgm2 = get_row_group_meta_data( &schema_descr, - vec![ParquetStatistics::int32(Some(11), Some(20), None, 0, false)], + vec![ParquetStatistics::int32( + Some(11), + Some(20), + None, + Some(0), + false, + )], ); let metrics = parquet_file_metrics(); // missing statistics for first row group mean that the result from the predicate expression @@ -560,15 +578,15 @@ mod tests { let rgm1 = get_row_group_meta_data( &schema_descr, vec![ - ParquetStatistics::int32(Some(1), Some(10), None, 0, false), - ParquetStatistics::int32(Some(1), Some(10), None, 0, false), + ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false), + ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false), ], ); let rgm2 = get_row_group_meta_data( &schema_descr, vec![ - ParquetStatistics::int32(Some(11), Some(20), None, 0, false), - ParquetStatistics::int32(Some(11), Some(20), None, 0, false), + ParquetStatistics::int32(Some(11), Some(20), None, Some(0), false), + ParquetStatistics::int32(Some(11), Some(20), None, Some(0), false), ], ); @@ -633,16 +651,16 @@ mod tests { let rgm1 = get_row_group_meta_data( &schema_descr, vec![ - ParquetStatistics::int32(Some(-10), Some(-1), None, 0, false), // c2 - ParquetStatistics::int32(Some(1), Some(10), None, 0, false), + ParquetStatistics::int32(Some(-10), Some(-1), None, Some(0), false), // c2 + ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false), ], ); // rg1 has c2 greater than zero, c1 less than zero let rgm2 = get_row_group_meta_data( &schema_descr, vec![ - ParquetStatistics::int32(Some(1), Some(10), None, 0, false), - ParquetStatistics::int32(Some(-10), Some(-1), None, 0, false), + ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false), + ParquetStatistics::int32(Some(-10), Some(-1), None, Some(0), false), ], ); @@ -669,15 +687,15 @@ mod tests { let rgm1 = get_row_group_meta_data( &schema_descr, vec![ - ParquetStatistics::int32(Some(1), Some(10), None, 0, false), - ParquetStatistics::boolean(Some(false), Some(true), None, 0, false), + ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false), + ParquetStatistics::boolean(Some(false), Some(true), None, Some(0), false), ], ); let rgm2 = get_row_group_meta_data( &schema_descr, vec![ - ParquetStatistics::int32(Some(11), Some(20), None, 0, false), - ParquetStatistics::boolean(Some(false), Some(true), None, 1, false), + ParquetStatistics::int32(Some(11), Some(20), None, Some(0), false), + ParquetStatistics::boolean(Some(false), Some(true), None, Some(1), false), ], ); vec![rgm1, rgm2] @@ -775,7 +793,7 @@ mod tests { Some(100), Some(600), None, - 0, + Some(0), false, )], ); @@ -783,13 +801,25 @@ mod tests { &schema_descr, // [0.1, 0.2] // c1 > 5, this row group will not be included in the results. - vec![ParquetStatistics::int32(Some(10), Some(20), None, 0, false)], + vec![ParquetStatistics::int32( + Some(10), + Some(20), + None, + Some(0), + false, + )], ); let rgm3 = get_row_group_meta_data( &schema_descr, // [1, None] // c1 > 5, this row group can not be filtered out, so will be included in the results. - vec![ParquetStatistics::int32(Some(100), None, None, 0, false)], + vec![ParquetStatistics::int32( + Some(100), + None, + None, + Some(0), + false, + )], ); let metrics = parquet_file_metrics(); let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(3)); @@ -837,7 +867,7 @@ mod tests { Some(100), Some(600), None, - 0, + Some(0), false, )], ); @@ -845,30 +875,62 @@ mod tests { &schema_descr, // [10, 20] // c1 > 5, this row group will be included in the results. - vec![ParquetStatistics::int32(Some(10), Some(20), None, 0, false)], + vec![ParquetStatistics::int32( + Some(10), + Some(20), + None, + Some(0), + false, + )], ); let rgm3 = get_row_group_meta_data( &schema_descr, // [0, 2] // c1 > 5, this row group will not be included in the results. - vec![ParquetStatistics::int32(Some(0), Some(2), None, 0, false)], + vec![ParquetStatistics::int32( + Some(0), + Some(2), + None, + Some(0), + false, + )], ); let rgm4 = get_row_group_meta_data( &schema_descr, // [None, 2] - // c1 > 5, this row group can not be filtered out, so will be included in the results. - vec![ParquetStatistics::int32(None, Some(2), None, 0, false)], + // c1 > 5, this row group will also not be included in the results + // (the min value is unknown, but the max value is 2, so no values can be greater than 5) + vec![ParquetStatistics::int32( + None, + Some(2), + None, + Some(0), + false, + )], + ); + let rgm5 = get_row_group_meta_data( + &schema_descr, + // [2, None] + // c1 > 5, this row group must be included + // (the min value is 2, but the max value is unknown, so it may have values greater than 5) + vec![ParquetStatistics::int32( + Some(2), + None, + None, + Some(0), + false, + )], ); let metrics = parquet_file_metrics(); - let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(4)); + let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(5)); row_groups.prune_by_statistics( &schema, &schema_descr, - &[rgm1, rgm2, rgm3, rgm4], + &[rgm1, rgm2, rgm3, rgm4, rgm5], &pruning_predicate, &metrics, ); - assert_pruned(row_groups, ExpectedPruning::Some(vec![0, 1, 3])); + assert_pruned(row_groups, ExpectedPruning::Some(vec![0, 1, 4])); } #[test] fn row_group_pruning_predicate_decimal_type3() { @@ -896,19 +958,25 @@ mod tests { Some(600), Some(800), None, - 0, + Some(0), false, )], ); let rgm2 = get_row_group_meta_data( &schema_descr, // [0.1, 0.2] - vec![ParquetStatistics::int64(Some(10), Some(20), None, 0, false)], + vec![ParquetStatistics::int64( + Some(10), + Some(20), + None, + Some(0), + false, + )], ); let rgm3 = get_row_group_meta_data( &schema_descr, // [0.1, 0.2] - vec![ParquetStatistics::int64(None, None, None, 0, false)], + vec![ParquetStatistics::int64(None, None, None, Some(0), false)], ); let metrics = parquet_file_metrics(); let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(3)); @@ -957,7 +1025,7 @@ mod tests { 8000i128.to_be_bytes().to_vec(), ))), None, - 0, + Some(0), false, )], ); @@ -973,7 +1041,7 @@ mod tests { 20000i128.to_be_bytes().to_vec(), ))), None, - 0, + Some(0), false, )], ); @@ -981,7 +1049,11 @@ mod tests { let rgm3 = get_row_group_meta_data( &schema_descr, vec![ParquetStatistics::fixed_len_byte_array( - None, None, None, 0, false, + None, + None, + None, + Some(0), + false, )], ); let metrics = parquet_file_metrics(); @@ -1027,7 +1099,7 @@ mod tests { // 80.00 Some(ByteArray::from(8000i128.to_be_bytes().to_vec())), None, - 0, + Some(0), false, )], ); @@ -1039,13 +1111,19 @@ mod tests { // 200.00 Some(ByteArray::from(20000i128.to_be_bytes().to_vec())), None, - 0, + Some(0), false, )], ); let rgm3 = get_row_group_meta_data( &schema_descr, - vec![ParquetStatistics::byte_array(None, None, None, 0, false)], + vec![ParquetStatistics::byte_array( + None, + None, + None, + Some(0), + false, + )], ); let metrics = parquet_file_metrics(); let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(3)); diff --git a/datafusion/functions/src/regex/regexpreplace.rs b/datafusion/functions/src/regex/regexpreplace.rs index d28c6cd36d65..0b0f7287e1ec 100644 --- a/datafusion/functions/src/regex/regexpreplace.rs +++ b/datafusion/functions/src/regex/regexpreplace.rs @@ -401,8 +401,7 @@ fn _regexp_replace_static_pattern_replace( DataType::Utf8View => { let string_view_array = as_string_view_array(&args[0])?; - let mut builder = StringViewBuilder::with_capacity(string_view_array.len()) - .with_block_size(1024 * 1024 * 2); + let mut builder = StringViewBuilder::with_capacity(string_view_array.len()); for val in string_view_array.iter() { if let Some(val) = val { diff --git a/datafusion/physical-expr-common/src/binary_view_map.rs b/datafusion/physical-expr-common/src/binary_view_map.rs index 18bc6801aa60..bdcf7bbacc69 100644 --- a/datafusion/physical-expr-common/src/binary_view_map.rs +++ b/datafusion/physical-expr-common/src/binary_view_map.rs @@ -149,7 +149,7 @@ where output_type, map: hashbrown::raw::RawTable::with_capacity(INITIAL_MAP_CAPACITY), map_size: 0, - builder: GenericByteViewBuilder::new().with_block_size(2 * 1024 * 1024), + builder: GenericByteViewBuilder::new(), random_state: RandomState::new(), hashes_buffer: vec![], null: None, diff --git a/datafusion/physical-plan/src/coalesce/mod.rs b/datafusion/physical-plan/src/coalesce/mod.rs index ce5a1e53abfd..46875fae94fc 100644 --- a/datafusion/physical-plan/src/coalesce/mod.rs +++ b/datafusion/physical-plan/src/coalesce/mod.rs @@ -248,7 +248,7 @@ fn gc_string_view_batch(batch: &RecordBatch) -> RecordBatch { // See https://github.com/apache/arrow-rs/issues/6094 for more details. let mut builder = StringViewBuilder::with_capacity(s.len()); if ideal_buffer_size > 0 { - builder = builder.with_block_size(ideal_buffer_size as u32); + builder = builder.with_fixed_block_size(ideal_buffer_size as u32); } for v in s.iter() { @@ -580,7 +580,8 @@ mod tests { impl StringViewTest { /// Create a `StringViewArray` with the parameters specified in this struct fn build(self) -> StringViewArray { - let mut builder = StringViewBuilder::with_capacity(100).with_block_size(8192); + let mut builder = + StringViewBuilder::with_capacity(100).with_fixed_block_size(8192); loop { for &v in self.strings.iter() { builder.append_option(v); diff --git a/datafusion/proto-common/Cargo.toml b/datafusion/proto-common/Cargo.toml index 33a7ecd2daab..7ba503171520 100644 --- a/datafusion/proto-common/Cargo.toml +++ b/datafusion/proto-common/Cargo.toml @@ -44,7 +44,7 @@ arrow = { workspace = true } chrono = { workspace = true } datafusion-common = { workspace = true } object_store = { workspace = true } -pbjson = { version = "0.6.0", optional = true } +pbjson = { workspace = true, optional = true } prost = { workspace = true } serde = { version = "1.0", optional = true } serde_json = { workspace = true, optional = true } diff --git a/datafusion/proto-common/gen/Cargo.toml b/datafusion/proto-common/gen/Cargo.toml index 54ec0e44694b..cca49dba7ed3 100644 --- a/datafusion/proto-common/gen/Cargo.toml +++ b/datafusion/proto-common/gen/Cargo.toml @@ -34,5 +34,5 @@ workspace = true [dependencies] # Pin these dependencies so that the generated output is deterministic -pbjson-build = "=0.6.2" -prost-build = "=0.12.6" +pbjson-build = "=0.7.0" +prost-build = "=0.13.1" diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 2804ed019b61..32678246c005 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -52,7 +52,7 @@ datafusion-common = { workspace = true, default-features = true } datafusion-expr = { workspace = true } datafusion-proto-common = { workspace = true } object_store = { workspace = true } -pbjson = { version = "0.6.0", optional = true } +pbjson = { workspace = true, optional = true } prost = { workspace = true } serde = { version = "1.0", optional = true } serde_json = { workspace = true, optional = true } diff --git a/datafusion/proto/gen/Cargo.toml b/datafusion/proto/gen/Cargo.toml index 401c51c94563..1dc5f7e0dddc 100644 --- a/datafusion/proto/gen/Cargo.toml +++ b/datafusion/proto/gen/Cargo.toml @@ -34,5 +34,5 @@ workspace = true [dependencies] # Pin these dependencies so that the generated output is deterministic -pbjson-build = "=0.6.2" -prost-build = "=0.12.6" +pbjson-build = "=0.7.0" +prost-build = "=0.13.1" diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index ebc05718a458..3d7b1007b04e 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -45,10 +45,10 @@ pub struct ParquetFormat { pub options: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct AvroFormat {} #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct NdJsonFormat { #[prost(message, optional, tag = "1")] pub options: ::core::option::Option, @@ -89,10 +89,10 @@ pub struct Constraints { pub constraints: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct AvroOptions {} #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct ArrowOptions {} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -137,7 +137,7 @@ pub struct Timestamp { pub timezone: ::prost::alloc::string::String, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct Decimal { #[prost(uint32, tag = "3")] pub precision: u32, @@ -145,7 +145,7 @@ pub struct Decimal { pub scale: i32, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct Decimal256Type { #[prost(uint32, tag = "3")] pub precision: u32, @@ -223,7 +223,7 @@ pub mod scalar_nested_value { } } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct ScalarTime32Value { #[prost(oneof = "scalar_time32_value::Value", tags = "1, 2")] pub value: ::core::option::Option, @@ -231,7 +231,7 @@ pub struct ScalarTime32Value { /// Nested message and enum types in `ScalarTime32Value`. pub mod scalar_time32_value { #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum Value { #[prost(int32, tag = "1")] Time32SecondValue(i32), @@ -240,7 +240,7 @@ pub mod scalar_time32_value { } } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct ScalarTime64Value { #[prost(oneof = "scalar_time64_value::Value", tags = "1, 2")] pub value: ::core::option::Option, @@ -248,7 +248,7 @@ pub struct ScalarTime64Value { /// Nested message and enum types in `ScalarTime64Value`. pub mod scalar_time64_value { #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum Value { #[prost(int64, tag = "1")] Time64MicrosecondValue(i64), @@ -267,7 +267,7 @@ pub struct ScalarTimestampValue { /// Nested message and enum types in `ScalarTimestampValue`. pub mod scalar_timestamp_value { #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum Value { #[prost(int64, tag = "1")] TimeMicrosecondValue(i64), @@ -288,7 +288,7 @@ pub struct ScalarDictionaryValue { pub value: ::core::option::Option<::prost::alloc::boxed::Box>, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct IntervalDayTimeValue { #[prost(int32, tag = "1")] pub days: i32, @@ -296,7 +296,7 @@ pub struct IntervalDayTimeValue { pub milliseconds: i32, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct IntervalMonthDayNanoValue { #[prost(int32, tag = "1")] pub months: i32, @@ -558,10 +558,10 @@ pub mod arrow_type { /// } /// } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct EmptyMessage {} #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct JsonWriterOptions { #[prost(enumeration = "CompressionTypeVariant", tag = "1")] pub compression: i32, @@ -655,7 +655,7 @@ pub struct CsvOptions { } /// Options controlling CSV format #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct JsonOptions { /// Compression type #[prost(enumeration = "CompressionTypeVariant", tag = "1")] @@ -720,7 +720,7 @@ pub struct ParquetColumnOptions { /// Nested message and enum types in `ParquetColumnOptions`. pub mod parquet_column_options { #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterEnabledOpt { #[prost(bool, tag = "1")] BloomFilterEnabled(bool), @@ -732,7 +732,7 @@ pub mod parquet_column_options { Encoding(::prost::alloc::string::String), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum DictionaryEnabledOpt { #[prost(bool, tag = "3")] DictionaryEnabled(bool), @@ -750,19 +750,19 @@ pub mod parquet_column_options { StatisticsEnabled(::prost::alloc::string::String), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterFppOpt { #[prost(double, tag = "6")] BloomFilterFpp(f64), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterNdvOpt { #[prost(uint64, tag = "7")] BloomFilterNdv(u64), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum MaxStatisticsSizeOpt { #[prost(uint32, tag = "8")] MaxStatisticsSize(u32), @@ -857,7 +857,7 @@ pub struct ParquetOptions { /// Nested message and enum types in `ParquetOptions`. pub mod parquet_options { #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum MetadataSizeHintOpt { #[prost(uint64, tag = "4")] MetadataSizeHint(u64), @@ -869,7 +869,7 @@ pub mod parquet_options { Compression(::prost::alloc::string::String), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum DictionaryEnabledOpt { #[prost(bool, tag = "11")] DictionaryEnabled(bool), @@ -881,13 +881,13 @@ pub mod parquet_options { StatisticsEnabled(::prost::alloc::string::String), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum MaxStatisticsSizeOpt { #[prost(uint64, tag = "14")] MaxStatisticsSize(u64), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum ColumnIndexTruncateLengthOpt { #[prost(uint64, tag = "17")] ColumnIndexTruncateLength(u64), @@ -899,13 +899,13 @@ pub mod parquet_options { Encoding(::prost::alloc::string::String), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterFppOpt { #[prost(double, tag = "21")] BloomFilterFpp(f64), } #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] + #[derive(Clone, Copy, PartialEq, ::prost::Oneof)] pub enum BloomFilterNdvOpt { #[prost(uint64, tag = "22")] BloomFilterNdv(u64), diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index cff58d3ddc4a..1c433c2572c4 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -12,6 +12,7 @@ impl serde::Serialize for AggLimit { let mut struct_ser = serializer.serialize_struct("datafusion.AggLimit", len)?; if self.limit != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("limit", ToString::to_string(&self.limit).as_str())?; } struct_ser.end() @@ -613,6 +614,7 @@ impl serde::Serialize for AggregateUdfExprNode { } if let Some(v) = self.fun_definition.as_ref() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("funDefinition", pbjson::private::base64::encode(&v).as_str())?; } struct_ser.end() @@ -2348,6 +2350,7 @@ impl serde::Serialize for CopyToNode { } if !self.file_type.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("fileType", pbjson::private::base64::encode(&self.file_type).as_str())?; } if !self.partition_by.is_empty() { @@ -3953,6 +3956,7 @@ impl serde::Serialize for CustomTableScanNode { } if !self.custom_table_data.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("customTableData", pbjson::private::base64::encode(&self.custom_table_data).as_str())?; } struct_ser.end() @@ -5037,10 +5041,12 @@ impl serde::Serialize for FileRange { let mut struct_ser = serializer.serialize_struct("datafusion.FileRange", len)?; if self.start != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("start", ToString::to_string(&self.start).as_str())?; } if self.end != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("end", ToString::to_string(&self.end).as_str())?; } struct_ser.end() @@ -5922,6 +5928,7 @@ impl serde::Serialize for GlobalLimitExecNode { } if self.fetch != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } struct_ser.end() @@ -6357,6 +6364,7 @@ impl serde::Serialize for HashRepartition { } if self.partition_count != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("partitionCount", ToString::to_string(&self.partition_count).as_str())?; } struct_ser.end() @@ -8409,10 +8417,12 @@ impl serde::Serialize for LimitNode { } if self.skip != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("skip", ToString::to_string(&self.skip).as_str())?; } if self.fetch != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } struct_ser.end() @@ -9860,6 +9870,7 @@ impl serde::Serialize for LogicalExtensionNode { let mut struct_ser = serializer.serialize_struct("datafusion.LogicalExtensionNode", len)?; if !self.node.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("node", pbjson::private::base64::encode(&self.node).as_str())?; } if !self.inputs.is_empty() { @@ -11982,14 +11993,17 @@ impl serde::Serialize for PartitionStats { let mut struct_ser = serializer.serialize_struct("datafusion.PartitionStats", len)?; if self.num_rows != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("numRows", ToString::to_string(&self.num_rows).as_str())?; } if self.num_batches != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("numBatches", ToString::to_string(&self.num_batches).as_str())?; } if self.num_bytes != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("numBytes", ToString::to_string(&self.num_bytes).as_str())?; } if !self.column_stats.is_empty() { @@ -12146,10 +12160,12 @@ impl serde::Serialize for PartitionedFile { } if self.size != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("size", ToString::to_string(&self.size).as_str())?; } if self.last_modified_ns != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("lastModifiedNs", ToString::to_string(&self.last_modified_ns).as_str())?; } if !self.partition_values.is_empty() { @@ -12314,6 +12330,7 @@ impl serde::Serialize for Partitioning { match v { partitioning::PartitionMethod::RoundRobin(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("roundRobin", ToString::to_string(&v).as_str())?; } partitioning::PartitionMethod::Hash(v) => { @@ -12321,6 +12338,7 @@ impl serde::Serialize for Partitioning { } partitioning::PartitionMethod::Unknown(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("unknown", ToString::to_string(&v).as_str())?; } } @@ -12462,6 +12480,7 @@ impl serde::Serialize for PhysicalAggregateExprNode { } if let Some(v) = self.fun_definition.as_ref() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("funDefinition", pbjson::private::base64::encode(&v).as_str())?; } if let Some(v) = self.aggregate_function.as_ref() { @@ -13644,6 +13663,7 @@ impl serde::Serialize for PhysicalExtensionExprNode { let mut struct_ser = serializer.serialize_struct("datafusion.PhysicalExtensionExprNode", len)?; if !self.expr.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("expr", pbjson::private::base64::encode(&self.expr).as_str())?; } if !self.inputs.is_empty() { @@ -13755,6 +13775,7 @@ impl serde::Serialize for PhysicalExtensionNode { let mut struct_ser = serializer.serialize_struct("datafusion.PhysicalExtensionNode", len)?; if !self.node.is_empty() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("node", pbjson::private::base64::encode(&self.node).as_str())?; } if !self.inputs.is_empty() { @@ -13869,6 +13890,7 @@ impl serde::Serialize for PhysicalHashRepartition { } if self.partition_count != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("partitionCount", ToString::to_string(&self.partition_count).as_str())?; } struct_ser.end() @@ -15085,6 +15107,7 @@ impl serde::Serialize for PhysicalScalarUdfNode { } if let Some(v) = self.fun_definition.as_ref() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("funDefinition", pbjson::private::base64::encode(&v).as_str())?; } if let Some(v) = self.return_type.as_ref() { @@ -15687,6 +15710,7 @@ impl serde::Serialize for PhysicalWindowExprNode { } if let Some(v) = self.fun_definition.as_ref() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("funDefinition", pbjson::private::base64::encode(&v).as_str())?; } if let Some(v) = self.window_function.as_ref() { @@ -16901,6 +16925,7 @@ impl serde::Serialize for RepartitionNode { match v { repartition_node::PartitionMethod::RoundRobin(v) => { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("roundRobin", ToString::to_string(&v).as_str())?; } repartition_node::PartitionMethod::Hash(v) => { @@ -17123,6 +17148,7 @@ impl serde::Serialize for ScalarUdfExprNode { } if let Some(v) = self.fun_definition.as_ref() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("funDefinition", pbjson::private::base64::encode(&v).as_str())?; } struct_ser.end() @@ -17691,6 +17717,7 @@ impl serde::Serialize for SortExecNode { } if self.fetch != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } if self.preserve_partitioning { @@ -18052,6 +18079,7 @@ impl serde::Serialize for SortNode { } if self.fetch != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } struct_ser.end() @@ -18180,6 +18208,7 @@ impl serde::Serialize for SortPreservingMergeExecNode { } if self.fetch != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } struct_ser.end() @@ -19625,6 +19654,7 @@ impl serde::Serialize for ValuesNode { let mut struct_ser = serializer.serialize_struct("datafusion.ValuesNode", len)?; if self.n_cols != 0 { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("nCols", ToString::to_string(&self.n_cols).as_str())?; } if !self.values_list.is_empty() { @@ -20299,6 +20329,7 @@ impl serde::Serialize for WindowExprNode { } if let Some(v) = self.fun_definition.as_ref() { #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("funDefinition", pbjson::private::base64::encode(&v).as_str())?; } if let Some(v) = self.window_function.as_ref() { diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 2ce8004e3248..dbcf7672a48c 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -239,7 +239,7 @@ pub struct HashRepartition { pub partition_count: u64, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct EmptyRelationNode { #[prost(bool, tag = "1")] pub produce_one_row: bool, @@ -443,7 +443,7 @@ pub struct UnnestNode { pub options: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct UnnestOptions { #[prost(bool, tag = "1")] pub preserve_nulls: bool, @@ -912,7 +912,7 @@ pub struct WindowFrameBound { pub bound_value: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct FixedSizeBinary { #[prost(int32, tag = "1")] pub length: i32, @@ -1474,7 +1474,7 @@ pub struct FileGroup { pub files: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct ScanLimit { /// wrap into a message to make it optional #[prost(uint32, tag = "1")] @@ -1721,7 +1721,7 @@ pub struct MaybePhysicalSortExprs { pub sort_expr: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct AggLimit { /// wrap into a message to make it optional #[prost(uint64, tag = "1")] @@ -1877,7 +1877,7 @@ pub struct JoinFilter { pub schema: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct ColumnIndex { #[prost(uint32, tag = "1")] pub index: u32, @@ -1903,7 +1903,7 @@ pub struct PartitionedFile { pub statistics: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] +#[derive(Clone, Copy, PartialEq, ::prost::Message)] pub struct FileRange { #[prost(int64, tag = "1")] pub start: i64, diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index fe34d87bfeae..516833a39f1e 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -2083,49 +2083,49 @@ mod tests { "1 YEAR 1 MONTH 1 DAY 3 HOUR 10 MINUTE 20 SECOND", ), IntervalStyle::PostgresVerbose, - r#"INTERVAL '0 YEARS 13 MONS 1 DAYS 3 HOURS 10 MINS 20.000000000 SECS'"#, + r#"INTERVAL '13 MONS 1 DAYS 3 HOURS 10 MINS 20.000000000 SECS'"#, ), ( interval_month_day_nano_lit("1.5 MONTH"), IntervalStyle::PostgresVerbose, - r#"INTERVAL '0 YEARS 1 MONS 15 DAYS 0 HOURS 0 MINS 0.000000000 SECS'"#, + r#"INTERVAL '1 MONS 15 DAYS'"#, ), ( interval_month_day_nano_lit("-3 MONTH"), IntervalStyle::PostgresVerbose, - r#"INTERVAL '0 YEARS -3 MONS 0 DAYS 0 HOURS 0 MINS 0.000000000 SECS'"#, + r#"INTERVAL '-3 MONS'"#, ), ( interval_month_day_nano_lit("1 MONTH") .add(interval_month_day_nano_lit("1 DAY")), IntervalStyle::PostgresVerbose, - r#"(INTERVAL '0 YEARS 1 MONS 0 DAYS 0 HOURS 0 MINS 0.000000000 SECS' + INTERVAL '0 YEARS 0 MONS 1 DAYS 0 HOURS 0 MINS 0.000000000 SECS')"#, + r#"(INTERVAL '1 MONS' + INTERVAL '1 DAYS')"#, ), ( interval_month_day_nano_lit("1 MONTH") .sub(interval_month_day_nano_lit("1 DAY")), IntervalStyle::PostgresVerbose, - r#"(INTERVAL '0 YEARS 1 MONS 0 DAYS 0 HOURS 0 MINS 0.000000000 SECS' - INTERVAL '0 YEARS 0 MONS 1 DAYS 0 HOURS 0 MINS 0.000000000 SECS')"#, + r#"(INTERVAL '1 MONS' - INTERVAL '1 DAYS')"#, ), ( interval_datetime_lit("10 DAY 1 HOUR 10 MINUTE 20 SECOND"), IntervalStyle::PostgresVerbose, - r#"INTERVAL '0 YEARS 0 MONS 10 DAYS 1 HOURS 10 MINS 20.000 SECS'"#, + r#"INTERVAL '10 DAYS 1 HOURS 10 MINS 20.000 SECS'"#, ), ( interval_datetime_lit("10 DAY 1.5 HOUR 10 MINUTE 20 SECOND"), IntervalStyle::PostgresVerbose, - r#"INTERVAL '0 YEARS 0 MONS 10 DAYS 1 HOURS 40 MINS 20.000 SECS'"#, + r#"INTERVAL '10 DAYS 1 HOURS 40 MINS 20.000 SECS'"#, ), ( interval_year_month_lit("1 YEAR 1 MONTH"), IntervalStyle::PostgresVerbose, - r#"INTERVAL '1 YEARS 1 MONS 0 DAYS 0 HOURS 0 MINS 0.00 SECS'"#, + r#"INTERVAL '1 YEARS 1 MONS'"#, ), ( interval_year_month_lit("1.5 YEAR 1 MONTH"), IntervalStyle::PostgresVerbose, - r#"INTERVAL '1 YEARS 7 MONS 0 DAYS 0 HOURS 0 MINS 0.00 SECS'"#, + r#"INTERVAL '1 YEARS 7 MONS'"#, ), ( interval_year_month_lit("1 YEAR 1 MONTH"), diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index cdc7bef06afd..d4e189f5f66a 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -611,7 +611,7 @@ fn sql_round_trip(query: &str, expect: &str) { fn test_interval_lhs_eq() { sql_round_trip( "select interval '2 seconds' = interval '2 seconds'", - "SELECT (INTERVAL '0 YEARS 0 MONS 0 DAYS 0 HOURS 0 MINS 2.000000000 SECS' = INTERVAL '0 YEARS 0 MONS 0 DAYS 0 HOURS 0 MINS 2.000000000 SECS')", + "SELECT (INTERVAL '2.000000000 SECS' = INTERVAL '2.000000000 SECS')", ); } @@ -619,6 +619,6 @@ fn test_interval_lhs_eq() { fn test_interval_lhs_lt() { sql_round_trip( "select interval '2 seconds' < interval '2 seconds'", - "SELECT (INTERVAL '0 YEARS 0 MONS 0 DAYS 0 HOURS 0 MINS 2.000000000 SECS' < INTERVAL '0 YEARS 0 MONS 0 DAYS 0 HOURS 0 MINS 2.000000000 SECS')", + "SELECT (INTERVAL '2.000000000 SECS' < INTERVAL '2.000000000 SECS')", ); } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index c52445c561ee..83f4e4f03055 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -1916,7 +1916,7 @@ from values (interval '2 month 15 days'), (interval '-2 month') ---- -Interval(MonthDayNano) 0 years -2 mons 0 days 0 hours 0 mins 0.000000000 secs 0 years 2 mons 15 days 0 hours 0 mins 0.000000000 secs +Interval(MonthDayNano) -2 mons 2 mons 15 days # aggregate Interval(DayTime) min/max query T?? @@ -1927,7 +1927,7 @@ from values (arrow_cast('-3 minutes', 'Interval(DayTime)')), (arrow_cast('30 minutes', 'Interval(DayTime)')); ---- -Interval(DayTime) 0 years 0 mons 0 days 0 hours -3 mins 0.000 secs 0 years 0 mons 0 days 1 hours 0 mins 0.000 secs +Interval(DayTime) -3 mins 1 hours # aggregate Interval(YearMonth) min/max query T?? @@ -1938,7 +1938,7 @@ from values (arrow_cast('13 months', 'Interval(YearMonth)')), (arrow_cast('1 year', 'Interval(YearMonth)')); ---- -Interval(YearMonth) -1 years 0 mons 0 days 0 hours 0 mins 0.00 secs 1 years 1 mons 0 days 0 hours 0 mins 0.00 secs +Interval(YearMonth) -1 years 0 mons 1 years 1 mons # aggregate query II diff --git a/datafusion/sqllogictest/test_files/arrow_typeof.slt b/datafusion/sqllogictest/test_files/arrow_typeof.slt index bae6dc33c8cf..77b10b41ccb3 100644 --- a/datafusion/sqllogictest/test_files/arrow_typeof.slt +++ b/datafusion/sqllogictest/test_files/arrow_typeof.slt @@ -290,22 +290,22 @@ query ? --- select arrow_cast(interval '30 minutes', 'Interval(MonthDayNano)'); ---- -0 years 0 mons 0 days 0 hours 30 mins 0.000000000 secs +30 mins query ? select arrow_cast('30 minutes', 'Interval(DayTime)'); ---- -0 years 0 mons 0 days 0 hours 30 mins 0.000 secs +30 mins query ? select arrow_cast('1 year 5 months', 'Interval(YearMonth)'); ---- -1 years 5 mons 0 days 0 hours 0 mins 0.00 secs +1 years 5 mons query ? select arrow_cast('30 minutes', 'Interval(MonthDayNano)'); ---- -0 years 0 mons 0 days 0 hours 30 mins 0.000000000 secs +30 mins ## Duration @@ -432,5 +432,7 @@ MyAwesomeString Utf8View # Fails until we update to use the arrow-cast release with support for casting utf8 types to BinaryView # refer to merge commit https://github.com/apache/arrow-rs/commit/4bd737dab2aa17aca200259347909d48ed793ba1 -query error DataFusion error: This feature is not implemented: Unsupported CAST from Utf8 to BinaryView +query ?T select arrow_cast('MyAwesomeString', 'BinaryView'), arrow_typeof(arrow_cast('MyAwesomeString', 'BinaryView')) +---- +4d79417765736f6d65537472696e67 BinaryView diff --git a/datafusion/sqllogictest/test_files/ddl.slt b/datafusion/sqllogictest/test_files/ddl.slt index 7164425fc0f5..21edb458fe56 100644 --- a/datafusion/sqllogictest/test_files/ddl.slt +++ b/datafusion/sqllogictest/test_files/ddl.slt @@ -710,7 +710,7 @@ create table t (i interval, x int) as values (interval '5 days 3 nanoseconds', C query ?I select * from t; ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000003 secs 1 +5 days 0.000000003 secs 1 statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index 81ae60f3ba93..002e8db2132d 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -122,203 +122,203 @@ SELECT query ? SELECT interval '1' ---- -0 years 0 mons 0 days 0 hours 0 mins 1.000000000 secs +1.000000000 secs query ? SELECT interval '1 second' ---- -0 years 0 mons 0 days 0 hours 0 mins 1.000000000 secs +1.000000000 secs query ? SELECT interval '500 milliseconds' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.500000000 secs +0.500000000 secs query ? SELECT interval '5 second' ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs query ? SELECT interval '0.5 minute' ---- -0 years 0 mons 0 days 0 hours 0 mins 30.000000000 secs +30.000000000 secs query ? SELECT interval '.5 minute' ---- -0 years 0 mons 0 days 0 hours 0 mins 30.000000000 secs +30.000000000 secs query ? SELECT interval '5 minute' ---- -0 years 0 mons 0 days 0 hours 5 mins 0.000000000 secs +5 mins query ? SELECT interval '5 minute 1 second' ---- -0 years 0 mons 0 days 0 hours 5 mins 1.000000000 secs +5 mins 1.000000000 secs query ? SELECT interval '1 hour' ---- -0 years 0 mons 0 days 1 hours 0 mins 0.000000000 secs +1 hours query ? SELECT interval '5 hour' ---- -0 years 0 mons 0 days 5 hours 0 mins 0.000000000 secs +5 hours query ? SELECT interval '1 day' ---- -0 years 0 mons 1 days 0 hours 0 mins 0.000000000 secs +1 days query ? SELECT interval '1 week' ---- -0 years 0 mons 7 days 0 hours 0 mins 0.000000000 secs +7 days query ? SELECT interval '2 weeks' ---- -0 years 0 mons 14 days 0 hours 0 mins 0.000000000 secs +14 days query ? SELECT interval '1 day 1' ---- -0 years 0 mons 1 days 0 hours 0 mins 1.000000000 secs +1 days 1.000000000 secs query ? SELECT interval '0.5' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.500000000 secs +0.500000000 secs query ? SELECT interval '0.5 day 1' ---- -0 years 0 mons 0 days 12 hours 0 mins 1.000000000 secs +12 hours 1.000000000 secs query ? SELECT interval '0.49 day' ---- -0 years 0 mons 0 days 11 hours 45 mins 36.000000000 secs +11 hours 45 mins 36.000000000 secs query ? SELECT interval '0.499 day' ---- -0 years 0 mons 0 days 11 hours 58 mins 33.600000000 secs +11 hours 58 mins 33.600000000 secs query ? SELECT interval '0.4999 day' ---- -0 years 0 mons 0 days 11 hours 59 mins 51.360000000 secs +11 hours 59 mins 51.360000000 secs query ? SELECT interval '0.49999 day' ---- -0 years 0 mons 0 days 11 hours 59 mins 59.136000000 secs +11 hours 59 mins 59.136000000 secs query ? SELECT interval '0.49999999999 day' ---- -0 years 0 mons 0 days 11 hours 59 mins 59.999999136 secs +11 hours 59 mins 59.999999136 secs query ? SELECT interval '5 day' ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000000 secs +5 days # Hour is ignored, this matches PostgreSQL query ? SELECT interval '5 day' hour ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000000 secs +5 days query ? SELECT interval '5 day 4 hours 3 minutes 2 seconds 100 milliseconds' ---- -0 years 0 mons 5 days 4 hours 3 mins 2.100000000 secs +5 days 4 hours 3 mins 2.100000000 secs query ? SELECT interval '0.5 month' ---- -0 years 0 mons 15 days 0 hours 0 mins 0.000000000 secs +15 days query ? SELECT interval '0.5' month ---- -0 years 0 mons 15 days 0 hours 0 mins 0.000000000 secs +15 days query ? SELECT interval '1 month' ---- -0 years 1 mons 0 days 0 hours 0 mins 0.000000000 secs +1 mons query ? SELECT interval '1' MONTH ---- -0 years 1 mons 0 days 0 hours 0 mins 0.000000000 secs +1 mons query ? SELECT interval '5 month' ---- -0 years 5 mons 0 days 0 hours 0 mins 0.000000000 secs +5 mons query ? SELECT interval '13 month' ---- -0 years 13 mons 0 days 0 hours 0 mins 0.000000000 secs +13 mons query ? SELECT interval '0.5 year' ---- -0 years 6 mons 0 days 0 hours 0 mins 0.000000000 secs +6 mons query ? SELECT interval '1 year' ---- -0 years 12 mons 0 days 0 hours 0 mins 0.000000000 secs +12 mons query ? SELECT interval '1 decade' ---- -0 years 120 mons 0 days 0 hours 0 mins 0.000000000 secs +120 mons query ? SELECT interval '2 decades' ---- -0 years 240 mons 0 days 0 hours 0 mins 0.000000000 secs +240 mons query ? SELECT interval '1 century' ---- -0 years 1200 mons 0 days 0 hours 0 mins 0.000000000 secs +1200 mons query ? SELECT interval '2 year' ---- -0 years 24 mons 0 days 0 hours 0 mins 0.000000000 secs +24 mons query ? SELECT interval '1 year 1 day' ---- -0 years 12 mons 1 days 0 hours 0 mins 0.000000000 secs +12 mons 1 days query ? SELECT interval '1 year 1 day 1 hour' ---- -0 years 12 mons 1 days 1 hours 0 mins 0.000000000 secs +12 mons 1 days 1 hours query ? SELECT interval '1 year 1 day 1 hour 1 minute' ---- -0 years 12 mons 1 days 1 hours 1 mins 0.000000000 secs +12 mons 1 days 1 hours 1 mins query ? SELECT interval '1 year 1 day 1 hour 1 minute 1 second' ---- -0 years 12 mons 1 days 1 hours 1 mins 1.000000000 secs +12 mons 1 days 1 hours 1 mins 1.000000000 secs query I SELECT ascii('') diff --git a/datafusion/sqllogictest/test_files/interval.slt b/datafusion/sqllogictest/test_files/interval.slt index afb262cf95a5..077f38d5d5bb 100644 --- a/datafusion/sqllogictest/test_files/interval.slt +++ b/datafusion/sqllogictest/test_files/interval.slt @@ -45,250 +45,250 @@ Interval(MonthDayNano) Interval(MonthDayNano) query ? select interval '5' years ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs # check all different kinds of intervals query ? select interval '5' year ---- -0 years 60 mons 0 days 0 hours 0 mins 0.000000000 secs +60 mons query ? select interval '5' month ---- -0 years 5 mons 0 days 0 hours 0 mins 0.000000000 secs +5 mons query ? select interval '5' months ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs query ? select interval '5' week ---- -0 years 0 mons 35 days 0 hours 0 mins 0.000000000 secs +35 days query ? select interval '5' day ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000000 secs +5 days query ? select interval '5' hour ---- -0 years 0 mons 0 days 5 hours 0 mins 0.000000000 secs +5 hours ## This seems wrong (5 mons) query ? select interval '5' hours ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs query ? select interval '5' minute ---- -0 years 0 mons 0 days 0 hours 5 mins 0.000000000 secs +5 mins query ? select interval '5' second ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs query ? select interval '5' millisecond ---- -0 years 0 mons 0 days 0 hours 0 mins 0.005000000 secs +0.005000000 secs query ? select interval '5' milliseconds ---- -0 years 0 mons 0 days 0 hours 0 mins 0.005000000 secs +0.005000000 secs query ? select interval '5' microsecond ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000005000 secs +0.000005000 secs query ? select interval '5' microseconds ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000005000 secs +0.000005000 secs query ? select interval '5' nanosecond ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000000005 secs +0.000000005 secs query ? select interval '5' nanoseconds ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000000005 secs +0.000000005 secs query ? select interval '5 YEAR' ---- -0 years 60 mons 0 days 0 hours 0 mins 0.000000000 secs +60 mons query ? select interval '5 MONTH' ---- -0 years 5 mons 0 days 0 hours 0 mins 0.000000000 secs +5 mons query ? select interval '5 WEEK' ---- -0 years 0 mons 35 days 0 hours 0 mins 0.000000000 secs +35 days query ? select interval '5 DAY' ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000000 secs +5 days query ? select interval '5 HOUR' ---- -0 years 0 mons 0 days 5 hours 0 mins 0.000000000 secs +5 hours query ? select interval '5 HOURS' ---- -0 years 0 mons 0 days 5 hours 0 mins 0.000000000 secs +5 hours query ? select interval '5 MINUTE' ---- -0 years 0 mons 0 days 0 hours 5 mins 0.000000000 secs +5 mins query ? select interval '5 SECOND' ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs query ? select interval '5 SECONDS' ---- -0 years 0 mons 0 days 0 hours 0 mins 5.000000000 secs +5.000000000 secs query ? select interval '5 MILLISECOND' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.005000000 secs +0.005000000 secs query ? select interval '5 MILLISECONDS' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.005000000 secs +0.005000000 secs query ? select interval '5 MICROSECOND' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000005000 secs +0.000005000 secs query ? select interval '5 MICROSECONDS' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000005000 secs +0.000005000 secs query ? select interval '5 NANOSECOND' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000000005 secs +0.000000005 secs query ? select interval '5 NANOSECONDS' ---- -0 years 0 mons 0 days 0 hours 0 mins 0.000000005 secs +0.000000005 secs query ? select interval '5 YEAR 5 MONTH 5 DAY 5 HOUR 5 MINUTE 5 SECOND 5 MILLISECOND 5 MICROSECOND 5 NANOSECOND' ---- -0 years 65 mons 5 days 5 hours 5 mins 5.005005005 secs +65 mons 5 days 5 hours 5 mins 5.005005005 secs # Interval with string literal addition query ? select interval '1 month' + '1 month' ---- -0 years 2 mons 0 days 0 hours 0 mins 0.000000000 secs +2 mons # Interval with string literal addition and leading field query ? select interval '1' + '1' month ---- -0 years 2 mons 0 days 0 hours 0 mins 0.000000000 secs +2 mons # Interval with nested string literal addition query ? select interval '1 month' + '1 month' + '1 month' ---- -0 years 3 mons 0 days 0 hours 0 mins 0.000000000 secs +3 mons # Interval with nested string literal addition and leading field query ? select interval '1' + '1' + '1' month ---- -0 years 3 mons 0 days 0 hours 0 mins 0.000000000 secs +3 mons # Interval mega nested string literal addition query ? select interval '1 year' + '1 month' + '1 day' + '1 hour' + '1 minute' + '1 second' + '1 millisecond' + '1 microsecond' + '1 nanosecond' ---- -0 years 13 mons 1 days 1 hours 1 mins 1.001001001 secs +13 mons 1 days 1 hours 1 mins 1.001001001 secs # Interval with string literal subtraction query ? select interval '1 month' - '1 day'; ---- -0 years 1 mons -1 days 0 hours 0 mins 0.000000000 secs +1 mons -1 days # Interval with string literal subtraction and leading field query ? select interval '5' - '1' - '2' year; ---- -0 years 24 mons 0 days 0 hours 0 mins 0.000000000 secs +24 mons # Interval with nested string literal subtraction query ? select interval '1 month' - '1 day' - '1 hour'; ---- -0 years 1 mons -1 days -1 hours 0 mins 0.000000000 secs +1 mons -1 days -1 hours # Interval with nested string literal subtraction and leading field query ? select interval '10' - '1' - '1' month; ---- -0 years 8 mons 0 days 0 hours 0 mins 0.000000000 secs +8 mons # Interval mega nested string literal subtraction query ? select interval '1 year' - '1 month' - '1 day' - '1 hour' - '1 minute' - '1 second' - '1 millisecond' - '1 microsecond' - '1 nanosecond' ---- -0 years 11 mons -1 days -1 hours -1 mins -1.001001001 secs +11 mons -1 days -1 hours -1 mins -1.001001001 secs # Interval with string literal negation and leading field query ? select -interval '5' - '1' - '2' year; ---- -0 years -96 mons 0 days 0 hours 0 mins 0.000000000 secs +-96 mons # Interval with nested string literal negation query ? select -interval '1 month' + '1 day' + '1 hour'; ---- -0 years -1 mons 1 days 1 hours 0 mins 0.000000000 secs +-1 mons 1 days 1 hours # Interval with nested string literal negation and leading field query ? select -interval '10' - '1' - '1' month; ---- -0 years -12 mons 0 days 0 hours 0 mins 0.000000000 secs +-12 mons # Interval mega nested string literal negation query ? select -interval '1 year' - '1 month' - '1 day' - '1 hour' - '1 minute' - '1 second' - '1 millisecond' - '1 microsecond' - '1 nanosecond' ---- -0 years -13 mons -1 days -1 hours -1 mins -1.001001001 secs +-13 mons -1 days -1 hours -1 mins -1.001001001 secs # Interval string literal + date query D @@ -343,7 +343,7 @@ select arrow_typeof(i) from t; ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000003 secs Interval(MonthDayNano) +5 days 0.000000003 secs Interval(MonthDayNano) statement ok @@ -359,8 +359,8 @@ insert into t values ('6 days 7 nanoseconds'::interval) query ? rowsort select -i from t order by 1; ---- -0 years 0 mons -5 days 0 hours 0 mins -0.000000003 secs -0 years 0 mons -6 days 0 hours 0 mins -0.000000007 secs +-5 days -0.000000003 secs +-6 days -0.000000007 secs query ?T rowsort select @@ -368,8 +368,8 @@ select arrow_typeof(i) from t; ---- -0 years 0 mons 5 days 0 hours 0 mins 0.000000003 secs Interval(MonthDayNano) -0 years 0 mons 6 days 0 hours 0 mins 0.000000007 secs Interval(MonthDayNano) +5 days 0.000000003 secs Interval(MonthDayNano) +6 days 0.000000007 secs Interval(MonthDayNano) statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/math.slt b/datafusion/sqllogictest/test_files/math.slt index 6884d762612d..eece56942317 100644 --- a/datafusion/sqllogictest/test_files/math.slt +++ b/datafusion/sqllogictest/test_files/math.slt @@ -673,7 +673,7 @@ query error DataFusion error: Arrow error: Compute error: Signed integer overflo select lcm(2, 9223372036854775803); -query error DataFusion error: Arrow error: Compute error: Overflow happened on: 2107754225 \^ 1221660777 +query error DataFusion error: Arrow error: Arithmetic overflow: Overflow happened on: 2107754225 \^ 1221660777 select power(2107754225, 1221660777); # factorial overflow diff --git a/datafusion/sqllogictest/test_files/repartition_scan.slt b/datafusion/sqllogictest/test_files/repartition_scan.slt index 6b9cb521f5f8..4c86312f9e51 100644 --- a/datafusion/sqllogictest/test_files/repartition_scan.slt +++ b/datafusion/sqllogictest/test_files/repartition_scan.slt @@ -61,7 +61,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..104], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:104..208], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:208..312], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:312..414]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] +03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..87], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:87..174], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:174..261], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:261..347]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] # disable round robin repartitioning statement ok @@ -77,7 +77,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 != 42 -03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..104], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:104..208], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:208..312], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:312..414]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] +03)----ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..87], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:87..174], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:174..261], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:261..347]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] # enable round robin repartitioning again statement ok @@ -102,7 +102,7 @@ physical_plan 02)--SortExec: expr=[column1@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: column1@0 != 42 -05)--------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..205], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:205..405, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..5], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:5..210], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:210..414]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] +05)--------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..172], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:172..338, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..6], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:6..178], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:178..347]]}, projection=[column1], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] ## Read the files as though they are ordered @@ -138,7 +138,7 @@ physical_plan 01)SortPreservingMergeExec: [column1@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: column1@0 != 42 -04)------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..202], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..207], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:207..414], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:202..405]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] +04)------ParquetExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:0..169], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:0..173], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/2.parquet:173..347], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition_scan/parquet_table/1.parquet:169..338]]}, projection=[column1], output_ordering=[column1@0 ASC NULLS LAST], predicate=column1@0 != 42, pruning_predicate=CASE WHEN column1_null_count@2 = column1_row_count@3 THEN false ELSE column1_min@0 != 42 OR 42 != column1_max@1 END, required_guarantees=[column1 not in (42)] # Cleanup statement ok diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index fb0fd8397f2d..4b11e338da70 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -1509,19 +1509,19 @@ SELECT val, ts1 - ts2 FROM foo ORDER BY ts2 - ts1; query ? SELECT i1 - i2 FROM bar; ---- -0 years 0 mons -1 days 0 hours 0 mins 0.000000000 secs -0 years 2 mons -13 days 0 hours 0 mins 0.000000000 secs -0 years 0 mons 1 days 2 hours 56 mins 0.000000000 secs -0 years 0 mons 1 days 0 hours 0 mins -3.999999993 secs +-1 days +2 mons -13 days +1 days 2 hours 56 mins +1 days -3.999999993 secs # Interval + Interval query ? SELECT i1 + i2 FROM bar; ---- -0 years 0 mons 3 days 0 hours 0 mins 0.000000000 secs -0 years 2 mons 13 days 0 hours 0 mins 0.000000000 secs -0 years 0 mons 1 days 3 hours 4 mins 0.000000000 secs -0 years 0 mons 1 days 0 hours 0 mins 4.000000007 secs +3 days +2 mons 13 days +1 days 3 hours 4 mins +1 days 4.000000007 secs # Timestamp - Interval query P From 7561cbc057ddecea220cc052011d9a8f5e2eef02 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Thu, 5 Sep 2024 13:58:44 +0300 Subject: [PATCH 72/75] Minor: Update Sanity Checker Error Messages (#12333) * Update sanity_checker.rs * Update datafusion/core/src/physical_optimizer/sanity_checker.rs Co-authored-by: Marco Neumann --------- Co-authored-by: Marco Neumann --- .../src/physical_optimizer/sanity_checker.rs | 30 +++++++++++-------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index bd80d31224ef..e392105fbcb7 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -120,32 +120,36 @@ pub fn check_plan_sanity( ) -> Result>> { check_finiteness_requirements(plan.clone(), optimizer_options)?; - for (child, child_sort_req, child_dist_req) in izip!( - plan.children().iter(), + for ((idx, child), sort_req, dist_req) in izip!( + plan.children().iter().enumerate(), plan.required_input_ordering().iter(), plan.required_input_distribution().iter() ) { let child_eq_props = child.equivalence_properties(); - if let Some(child_sort_req) = child_sort_req { - if !child_eq_props.ordering_satisfy_requirement(child_sort_req) { - let child_plan_str = get_plan_string(child); + if let Some(sort_req) = sort_req { + if !child_eq_props.ordering_satisfy_requirement(sort_req) { + let plan_str = get_plan_string(&plan); return plan_err!( - "Child: {:?} does not satisfy parent order requirements: {:?}", - child_plan_str, - child_sort_req + "Plan: {:?} does not satisfy order requirements: {:?}. Child-{} order: {:?}", + plan_str, + sort_req, + idx, + child_eq_props.oeq_class ); } } if !child .output_partitioning() - .satisfy(child_dist_req, child_eq_props) + .satisfy(dist_req, child_eq_props) { - let child_plan_str = get_plan_string(child); + let plan_str = get_plan_string(&plan); return plan_err!( - "Child: {:?} does not satisfy parent distribution requirements: {:?}", - child_plan_str, - child_dist_req + "Plan: {:?} does not satisfy distribution requirements: {:?}. Child-{} output partitioning: {:?}", + plan_str, + dist_req, + idx, + child.output_partitioning() ); } } From ab1e3e29402d0ab8467d29665ba090d8300270d5 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 5 Sep 2024 13:19:37 +0200 Subject: [PATCH 73/75] Improve & unify validation in LogicalPlan::with_new_exprs (#12264) * Improve & unify validation in LogicalPlan::with_new_exprs When adding new plan node type, `LogicalPlan::with_new_exprs` needs to be updated. Different code branches apply different inputs validation style (no validation, just assert, or assert with messages), so it's unclear which code pattern to follow. This commit unifies the validation and adds it to the branches where there was none. * #inline * rename and return Result * Add doc line --- datafusion/expr/src/logical_plan/plan.rs | 280 ++++++++++++++++------- 1 file changed, 193 insertions(+), 87 deletions(-) diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index d0c1c3b2b3d6..18a624dd9cb2 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -799,40 +799,49 @@ impl LogicalPlan { pub fn with_new_exprs( &self, mut expr: Vec, - mut inputs: Vec, + inputs: Vec, ) -> Result { match self { // Since expr may be different than the previous expr, schema of the projection // may change. We need to use try_new method instead of try_new_with_schema method. LogicalPlan::Projection(Projection { .. }) => { - Projection::try_new(expr, Arc::new(inputs.swap_remove(0))) - .map(LogicalPlan::Projection) + let input = self.only_input(inputs)?; + Projection::try_new(expr, Arc::new(input)).map(LogicalPlan::Projection) } LogicalPlan::Dml(DmlStatement { table_name, table_schema, op, .. - }) => Ok(LogicalPlan::Dml(DmlStatement::new( - table_name.clone(), - Arc::clone(table_schema), - op.clone(), - Arc::new(inputs.swap_remove(0)), - ))), + }) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Dml(DmlStatement::new( + table_name.clone(), + Arc::clone(table_schema), + op.clone(), + Arc::new(input), + ))) + } LogicalPlan::Copy(CopyTo { input: _, output_url, file_type, options, partition_by, - }) => Ok(LogicalPlan::Copy(CopyTo { - input: Arc::new(inputs.swap_remove(0)), - output_url: output_url.clone(), - file_type: Arc::clone(file_type), - options: options.clone(), - partition_by: partition_by.clone(), - })), + }) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Copy(CopyTo { + input: Arc::new(input), + output_url: output_url.clone(), + file_type: Arc::clone(file_type), + options: options.clone(), + partition_by: partition_by.clone(), + })) + } LogicalPlan::Values(Values { schema, .. }) => { + self.assert_no_inputs(inputs)?; Ok(LogicalPlan::Values(Values { schema: Arc::clone(schema), values: expr @@ -842,54 +851,63 @@ impl LogicalPlan { })) } LogicalPlan::Filter { .. } => { - assert_eq!(1, expr.len()); - let predicate = expr.pop().unwrap(); + let predicate = self.only_expr(expr)?; + let input = self.only_input(inputs)?; - Filter::try_new(predicate, Arc::new(inputs.swap_remove(0))) - .map(LogicalPlan::Filter) + Filter::try_new(predicate, Arc::new(input)).map(LogicalPlan::Filter) } LogicalPlan::Repartition(Repartition { partitioning_scheme, .. }) => match partitioning_scheme { Partitioning::RoundRobinBatch(n) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; Ok(LogicalPlan::Repartition(Repartition { partitioning_scheme: Partitioning::RoundRobinBatch(*n), - input: Arc::new(inputs.swap_remove(0)), + input: Arc::new(input), + })) + } + Partitioning::Hash(_, n) => { + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Repartition(Repartition { + partitioning_scheme: Partitioning::Hash(expr, *n), + input: Arc::new(input), })) } - Partitioning::Hash(_, n) => Ok(LogicalPlan::Repartition(Repartition { - partitioning_scheme: Partitioning::Hash(expr, *n), - input: Arc::new(inputs.swap_remove(0)), - })), Partitioning::DistributeBy(_) => { + let input = self.only_input(inputs)?; Ok(LogicalPlan::Repartition(Repartition { partitioning_scheme: Partitioning::DistributeBy(expr), - input: Arc::new(inputs.swap_remove(0)), + input: Arc::new(input), })) } }, LogicalPlan::Window(Window { window_expr, .. }) => { assert_eq!(window_expr.len(), expr.len()); - Window::try_new(expr, Arc::new(inputs.swap_remove(0))) - .map(LogicalPlan::Window) + let input = self.only_input(inputs)?; + Window::try_new(expr, Arc::new(input)).map(LogicalPlan::Window) } LogicalPlan::Aggregate(Aggregate { group_expr, .. }) => { + let input = self.only_input(inputs)?; // group exprs are the first expressions let agg_expr = expr.split_off(group_expr.len()); - Aggregate::try_new(Arc::new(inputs.swap_remove(0)), expr, agg_expr) + Aggregate::try_new(Arc::new(input), expr, agg_expr) .map(LogicalPlan::Aggregate) } LogicalPlan::Sort(Sort { expr: sort_expr, fetch, .. - }) => Ok(LogicalPlan::Sort(Sort { - expr: replace_sort_expressions(sort_expr.clone(), expr), - input: Arc::new(inputs.swap_remove(0)), - fetch: *fetch, - })), + }) => { + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Sort(Sort { + expr: replace_sort_expressions(sort_expr.clone(), expr), + input: Arc::new(input), + fetch: *fetch, + })) + } LogicalPlan::Join(Join { join_type, join_constraint, @@ -897,8 +915,8 @@ impl LogicalPlan { null_equals_null, .. }) => { - let schema = - build_join_schema(inputs[0].schema(), inputs[1].schema(), join_type)?; + let (left, right) = self.only_two_inputs(inputs)?; + let schema = build_join_schema(left.schema(), right.schema(), join_type)?; let equi_expr_count = on.len(); assert!(expr.len() >= equi_expr_count); @@ -927,8 +945,8 @@ impl LogicalPlan { }).collect::>>()?; Ok(LogicalPlan::Join(Join { - left: Arc::new(inputs.swap_remove(0)), - right: Arc::new(inputs.swap_remove(0)), + left: Arc::new(left), + right: Arc::new(right), join_type: *join_type, join_constraint: *join_constraint, on: new_on, @@ -938,28 +956,34 @@ impl LogicalPlan { })) } LogicalPlan::CrossJoin(_) => { - let left = inputs.swap_remove(0); - let right = inputs.swap_remove(0); + self.assert_no_expressions(expr)?; + let (left, right) = self.only_two_inputs(inputs)?; LogicalPlanBuilder::from(left).cross_join(right)?.build() } LogicalPlan::Subquery(Subquery { outer_ref_columns, .. }) => { - let subquery = LogicalPlanBuilder::from(inputs.swap_remove(0)).build()?; + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + let subquery = LogicalPlanBuilder::from(input).build()?; Ok(LogicalPlan::Subquery(Subquery { subquery: Arc::new(subquery), outer_ref_columns: outer_ref_columns.clone(), })) } LogicalPlan::SubqueryAlias(SubqueryAlias { alias, .. }) => { - SubqueryAlias::try_new(Arc::new(inputs.swap_remove(0)), alias.clone()) + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + SubqueryAlias::try_new(Arc::new(input), alias.clone()) .map(LogicalPlan::SubqueryAlias) } LogicalPlan::Limit(Limit { skip, fetch, .. }) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; Ok(LogicalPlan::Limit(Limit { skip: *skip, fetch: *fetch, - input: Arc::new(inputs.swap_remove(0)), + input: Arc::new(input), })) } LogicalPlan::Ddl(DdlStatement::CreateMemoryTable(CreateMemoryTable { @@ -968,31 +992,40 @@ impl LogicalPlan { or_replace, column_defaults, .. - })) => Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( - CreateMemoryTable { - input: Arc::new(inputs.swap_remove(0)), - constraints: Constraints::empty(), - name: name.clone(), - if_not_exists: *if_not_exists, - or_replace: *or_replace, - column_defaults: column_defaults.clone(), - }, - ))), + })) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Ddl(DdlStatement::CreateMemoryTable( + CreateMemoryTable { + input: Arc::new(input), + constraints: Constraints::empty(), + name: name.clone(), + if_not_exists: *if_not_exists, + or_replace: *or_replace, + column_defaults: column_defaults.clone(), + }, + ))) + } LogicalPlan::Ddl(DdlStatement::CreateView(CreateView { name, or_replace, definition, .. - })) => Ok(LogicalPlan::Ddl(DdlStatement::CreateView(CreateView { - input: Arc::new(inputs.swap_remove(0)), - name: name.clone(), - or_replace: *or_replace, - definition: definition.clone(), - }))), + })) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Ddl(DdlStatement::CreateView(CreateView { + input: Arc::new(input), + name: name.clone(), + or_replace: *or_replace, + definition: definition.clone(), + }))) + } LogicalPlan::Extension(e) => Ok(LogicalPlan::Extension(Extension { node: e.node.with_exprs_and_inputs(expr, inputs)?, })), LogicalPlan::Union(Union { schema, .. }) => { + self.assert_no_expressions(expr)?; let input_schema = inputs[0].schema(); // If inputs are not pruned do not change schema. let schema = if schema.fields().len() == input_schema.fields().len() { @@ -1007,12 +1040,17 @@ impl LogicalPlan { } LogicalPlan::Distinct(distinct) => { let distinct = match distinct { - Distinct::All(_) => Distinct::All(Arc::new(inputs.swap_remove(0))), + Distinct::All(_) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + Distinct::All(Arc::new(input)) + } Distinct::On(DistinctOn { on_expr, select_expr, .. }) => { + let input = self.only_input(inputs)?; let sort_expr = expr.split_off(on_expr.len() + select_expr.len()); let select_expr = expr.split_off(on_expr.len()); assert!(sort_expr.is_empty(), "with_new_exprs for Distinct does not support sort expressions"); @@ -1020,7 +1058,7 @@ impl LogicalPlan { expr, select_expr, None, // no sort expressions accepted - Arc::new(inputs.swap_remove(0)), + Arc::new(input), )?) } }; @@ -1028,30 +1066,31 @@ impl LogicalPlan { } LogicalPlan::RecursiveQuery(RecursiveQuery { name, is_distinct, .. - }) => Ok(LogicalPlan::RecursiveQuery(RecursiveQuery { - name: name.clone(), - static_term: Arc::new(inputs.swap_remove(0)), - recursive_term: Arc::new(inputs.swap_remove(0)), - is_distinct: *is_distinct, - })), + }) => { + self.assert_no_expressions(expr)?; + let (static_term, recursive_term) = self.only_two_inputs(inputs)?; + Ok(LogicalPlan::RecursiveQuery(RecursiveQuery { + name: name.clone(), + static_term: Arc::new(static_term), + recursive_term: Arc::new(recursive_term), + is_distinct: *is_distinct, + })) + } LogicalPlan::Analyze(a) => { - assert!(expr.is_empty()); - assert_eq!(inputs.len(), 1); + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; Ok(LogicalPlan::Analyze(Analyze { verbose: a.verbose, schema: Arc::clone(&a.schema), - input: Arc::new(inputs.swap_remove(0)), + input: Arc::new(input), })) } LogicalPlan::Explain(e) => { - assert!( - expr.is_empty(), - "Invalid EXPLAIN command. Expression should empty" - ); - assert_eq!(inputs.len(), 1, "Invalid EXPLAIN command. Inputs are empty"); + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; Ok(LogicalPlan::Explain(Explain { verbose: e.verbose, - plan: Arc::new(inputs.swap_remove(0)), + plan: Arc::new(input), stringified_plans: e.stringified_plans.clone(), schema: Arc::clone(&e.schema), logical_optimization_succeeded: e.logical_optimization_succeeded, @@ -1059,13 +1098,17 @@ impl LogicalPlan { } LogicalPlan::Prepare(Prepare { name, data_types, .. - }) => Ok(LogicalPlan::Prepare(Prepare { - name: name.clone(), - data_types: data_types.clone(), - input: Arc::new(inputs.swap_remove(0)), - })), + }) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Prepare(Prepare { + name: name.clone(), + data_types: data_types.clone(), + input: Arc::new(input), + })) + } LogicalPlan::TableScan(ts) => { - assert!(inputs.is_empty(), "{self:?} should have no inputs"); + self.assert_no_inputs(inputs)?; Ok(LogicalPlan::TableScan(TableScan { filters: expr, ..ts.clone() @@ -1073,26 +1116,89 @@ impl LogicalPlan { } LogicalPlan::EmptyRelation(_) | LogicalPlan::Ddl(_) - | LogicalPlan::Statement(_) => { + | LogicalPlan::Statement(_) + | LogicalPlan::DescribeTable(_) => { // All of these plan types have no inputs / exprs so should not be called - assert!(expr.is_empty(), "{self:?} should have no exprs"); - assert!(inputs.is_empty(), "{self:?} should have no inputs"); + self.assert_no_expressions(expr)?; + self.assert_no_inputs(inputs)?; Ok(self.clone()) } - LogicalPlan::DescribeTable(_) => Ok(self.clone()), LogicalPlan::Unnest(Unnest { exec_columns: columns, options, .. }) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; // Update schema with unnested column type. - let input = inputs.swap_remove(0); let new_plan = unnest_with_options(input, columns.clone(), options.clone())?; Ok(new_plan) } } } + + /// Helper for [Self::with_new_exprs] to use when no expressions are expected. + #[inline] + #[allow(clippy::needless_pass_by_value)] // expr is moved intentionally to ensure it's not used again + fn assert_no_expressions(&self, expr: Vec) -> Result<()> { + if !expr.is_empty() { + return internal_err!("{self:?} should have no exprs, got {:?}", expr); + } + Ok(()) + } + + /// Helper for [Self::with_new_exprs] to use when no inputs are expected. + #[inline] + #[allow(clippy::needless_pass_by_value)] // inputs is moved intentionally to ensure it's not used again + fn assert_no_inputs(&self, inputs: Vec) -> Result<()> { + if !inputs.is_empty() { + return internal_err!("{self:?} should have no inputs, got: {:?}", inputs); + } + Ok(()) + } + + /// Helper for [Self::with_new_exprs] to use when exactly one expression is expected. + #[inline] + fn only_expr(&self, mut expr: Vec) -> Result { + if expr.len() != 1 { + return internal_err!( + "{self:?} should have exactly one expr, got {:?}", + expr + ); + } + Ok(expr.remove(0)) + } + + /// Helper for [Self::with_new_exprs] to use when exactly one input is expected. + #[inline] + fn only_input(&self, mut inputs: Vec) -> Result { + if inputs.len() != 1 { + return internal_err!( + "{self:?} should have exactly one input, got {:?}", + inputs + ); + } + Ok(inputs.remove(0)) + } + + /// Helper for [Self::with_new_exprs] to use when exactly two inputs are expected. + #[inline] + fn only_two_inputs( + &self, + mut inputs: Vec, + ) -> Result<(LogicalPlan, LogicalPlan)> { + if inputs.len() != 2 { + return internal_err!( + "{self:?} should have exactly two inputs, got {:?}", + inputs + ); + } + let right = inputs.remove(1); + let left = inputs.remove(0); + Ok((left, right)) + } + /// Replaces placeholder param values (like `$1`, `$2`) in [`LogicalPlan`] /// with the specified `param_values`. /// From 008c94210c1342d6183ca3d4936a68623759980b Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Thu, 5 Sep 2024 19:20:30 +0800 Subject: [PATCH 74/75] Support the custom terminator for the CSV file format (#12263) * add terminator config to CsvConfig * add test and fix missing builder * remove the debug message and fix the doc * support EscapedStringLiteral * add create external table tests * refactor the error assertion * add issue reference --- datafusion-examples/examples/csv_opener.rs | 1 + datafusion/common/src/config.rs | 13 ++ .../core/src/datasource/file_format/csv.rs | 8 + .../src/datasource/file_format/options.rs | 10 ++ .../core/src/datasource/physical_plan/csv.rs | 139 +++++++++++++++++- datafusion/core/tests/data/cr_terminator.csv | 1 + .../data/newlines_in_values_cr_terminator.csv | 1 + .../proto/datafusion_common.proto | 1 + datafusion/proto-common/src/from_proto/mod.rs | 1 + .../proto-common/src/generated/pbjson.rs | 20 +++ .../proto-common/src/generated/prost.rs | 3 + datafusion/proto-common/src/to_proto/mod.rs | 1 + .../src/generated/datafusion_proto_common.rs | 3 + .../proto/src/logical_plan/file_formats.rs | 6 + datafusion/sql/src/utils.rs | 2 +- .../sqllogictest/test_files/csv_files.slt | 21 +++ 16 files changed, 228 insertions(+), 3 deletions(-) create mode 100644 datafusion/core/tests/data/cr_terminator.csv create mode 100644 datafusion/core/tests/data/newlines_in_values_cr_terminator.csv diff --git a/datafusion-examples/examples/csv_opener.rs b/datafusion-examples/examples/csv_opener.rs index 1f45026a214d..e7b7ead109bc 100644 --- a/datafusion-examples/examples/csv_opener.rs +++ b/datafusion-examples/examples/csv_opener.rs @@ -47,6 +47,7 @@ async fn main() -> Result<()> { true, b',', b'"', + None, object_store, Some(b'#'), ); diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 7c247103f6e7..19978e102cc8 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -1604,6 +1604,7 @@ config_namespace! { pub has_header: Option, default = None pub delimiter: u8, default = b',' pub quote: u8, default = b'"' + pub terminator: Option, default = None pub escape: Option, default = None pub double_quote: Option, default = None /// Specifies whether newlines in (quoted) values are supported. @@ -1672,6 +1673,13 @@ impl CsvOptions { self } + /// The character that terminates a row. + /// - default to None (CRLF) + pub fn with_terminator(mut self, terminator: Option) -> Self { + self.terminator = terminator; + self + } + /// The escape character in a row. /// - default is None pub fn with_escape(mut self, escape: Option) -> Self { @@ -1718,6 +1726,11 @@ impl CsvOptions { self.quote } + /// The terminator character. + pub fn terminator(&self) -> Option { + self.terminator + } + /// The escape character. pub fn escape(&self) -> Option { self.escape diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index e43f6ab29abc..99e8f13776fc 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -247,6 +247,13 @@ impl CsvFormat { self } + /// The character used to indicate the end of a row. + /// - default to None (CRLF) + pub fn with_terminator(mut self, terminator: Option) -> Self { + self.options.terminator = terminator; + self + } + /// Specifies whether newlines in (quoted) values are supported. /// /// Parsing newlines in quoted values may be affected by execution behaviour such as @@ -359,6 +366,7 @@ impl FileFormat for CsvFormat { .with_has_header(has_header) .with_delimeter(self.options.delimiter) .with_quote(self.options.quote) + .with_terminator(self.options.terminator) .with_escape(self.options.escape) .with_comment(self.options.comment) .with_newlines_in_values(newlines_in_values) diff --git a/datafusion/core/src/datasource/file_format/options.rs b/datafusion/core/src/datasource/file_format/options.rs index db90262edbf8..7ef5a2182d1c 100644 --- a/datafusion/core/src/datasource/file_format/options.rs +++ b/datafusion/core/src/datasource/file_format/options.rs @@ -59,6 +59,8 @@ pub struct CsvReadOptions<'a> { pub delimiter: u8, /// An optional quote character. Defaults to `b'"'`. pub quote: u8, + /// An optional terminator character. Defaults to None (CRLF). + pub terminator: Option, /// An optional escape character. Defaults to None. pub escape: Option, /// If enabled, lines beginning with this byte are ignored. @@ -102,6 +104,7 @@ impl<'a> CsvReadOptions<'a> { schema_infer_max_records: DEFAULT_SCHEMA_INFER_MAX_RECORD, delimiter: b',', quote: b'"', + terminator: None, escape: None, newlines_in_values: false, file_extension: DEFAULT_CSV_EXTENSION, @@ -136,6 +139,12 @@ impl<'a> CsvReadOptions<'a> { self } + /// Specify terminator to use for CSV read + pub fn terminator(mut self, terminator: Option) -> Self { + self.terminator = terminator; + self + } + /// Specify delimiter to use for CSV read pub fn escape(mut self, escape: u8) -> Self { self.escape = Some(escape); @@ -511,6 +520,7 @@ impl ReadOptions<'_> for CsvReadOptions<'_> { .with_delimiter(self.delimiter) .with_quote(self.quote) .with_escape(self.escape) + .with_terminator(self.terminator) .with_newlines_in_values(self.newlines_in_values) .with_schema_infer_max_rec(self.schema_infer_max_records) .with_file_compression_type(self.file_compression_type.to_owned()); diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 5ab32ed36e53..6cd1864deb1d 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -77,6 +77,7 @@ pub struct CsvExec { has_header: bool, delimiter: u8, quote: u8, + terminator: Option, escape: Option, comment: Option, newlines_in_values: bool, @@ -98,6 +99,7 @@ pub struct CsvExecBuilder { has_header: bool, delimiter: u8, quote: u8, + terminator: Option, escape: Option, comment: Option, newlines_in_values: bool, @@ -112,6 +114,7 @@ impl CsvExecBuilder { has_header: false, delimiter: b',', quote: b'"', + terminator: None, escape: None, comment: None, newlines_in_values: false, @@ -143,6 +146,14 @@ impl CsvExecBuilder { self } + /// Set the line terminator. If not set, the default is CRLF. + /// + /// The default is None. + pub fn with_terminator(mut self, terminator: Option) -> Self { + self.terminator = terminator; + self + } + /// Set the escape character. /// /// The default is `None` (i.e. quotes cannot be escaped). @@ -191,6 +202,7 @@ impl CsvExecBuilder { has_header, delimiter, quote, + terminator, escape, comment, newlines_in_values, @@ -210,6 +222,7 @@ impl CsvExecBuilder { has_header, delimiter, quote, + terminator, escape, newlines_in_values, metrics: ExecutionPlanMetricsSet::new(), @@ -229,6 +242,7 @@ impl CsvExec { has_header: bool, delimiter: u8, quote: u8, + terminator: Option, escape: Option, comment: Option, newlines_in_values: bool, @@ -238,6 +252,7 @@ impl CsvExec { .with_has_header(has_header) .with_delimeter(delimiter) .with_quote(quote) + .with_terminator(terminator) .with_escape(escape) .with_comment(comment) .with_newlines_in_values(newlines_in_values) @@ -270,6 +285,11 @@ impl CsvExec { self.quote } + /// The line terminator + pub fn terminator(&self) -> Option { + self.terminator + } + /// Lines beginning with this byte are ignored. pub fn comment(&self) -> Option { self.comment @@ -406,10 +426,10 @@ impl ExecutionPlan for CsvExec { delimiter: self.delimiter, quote: self.quote, escape: self.escape, + terminator: self.terminator, object_store, comment: self.comment, }); - let opener = CsvOpener { config, file_compression_type: self.file_compression_type.to_owned(), @@ -441,6 +461,7 @@ impl ExecutionPlan for CsvExec { delimiter: self.delimiter, quote: self.quote, escape: self.escape, + terminator: self.terminator, comment: self.comment, newlines_in_values: self.newlines_in_values, metrics: self.metrics.clone(), @@ -459,6 +480,7 @@ pub struct CsvConfig { has_header: bool, delimiter: u8, quote: u8, + terminator: Option, escape: Option, object_store: Arc, comment: Option, @@ -474,6 +496,7 @@ impl CsvConfig { has_header: bool, delimiter: u8, quote: u8, + terminator: Option, object_store: Arc, comment: Option, ) -> Self { @@ -484,6 +507,7 @@ impl CsvConfig { has_header, delimiter, quote, + terminator, escape: None, object_store, comment, @@ -502,7 +526,9 @@ impl CsvConfig { .with_batch_size(self.batch_size) .with_header(self.has_header) .with_quote(self.quote); - + if let Some(terminator) = self.terminator { + builder = builder.with_terminator(terminator); + } if let Some(proj) = &self.file_projection { builder = builder.with_projection(proj.clone()); } @@ -775,6 +801,7 @@ mod tests { .with_has_header(true) .with_delimeter(b',') .with_quote(b'"') + .with_terminator(None) .with_escape(None) .with_comment(None) .with_newlines_in_values(false) @@ -844,6 +871,7 @@ mod tests { .with_has_header(true) .with_delimeter(b',') .with_quote(b'"') + .with_terminator(None) .with_escape(None) .with_comment(None) .with_newlines_in_values(false) @@ -913,6 +941,7 @@ mod tests { .with_has_header(true) .with_delimeter(b',') .with_quote(b'"') + .with_terminator(None) .with_escape(None) .with_comment(None) .with_newlines_in_values(false) @@ -979,6 +1008,7 @@ mod tests { .with_has_header(true) .with_delimeter(b',') .with_quote(b'"') + .with_terminator(None) .with_escape(None) .with_comment(None) .with_newlines_in_values(false) @@ -1044,6 +1074,7 @@ mod tests { .with_has_header(true) .with_delimeter(b',') .with_quote(b'"') + .with_terminator(None) .with_escape(None) .with_comment(None) .with_newlines_in_values(false) @@ -1139,6 +1170,7 @@ mod tests { .with_has_header(true) .with_delimeter(b',') .with_quote(b'"') + .with_terminator(None) .with_escape(None) .with_comment(None) .with_newlines_in_values(false) @@ -1210,6 +1242,107 @@ mod tests { crate::assert_batches_eq!(expected, &result); } + #[tokio::test] + async fn test_terminator() { + let session_ctx = SessionContext::new(); + let store = object_store::memory::InMemory::new(); + + let data = bytes::Bytes::from("a,b\r1,2\r3,4"); + let path = object_store::path::Path::from("a.csv"); + store.put(&path, data.into()).await.unwrap(); + + let url = Url::parse("memory://").unwrap(); + session_ctx.register_object_store(&url, Arc::new(store)); + + let df = session_ctx + .read_csv("memory:///", CsvReadOptions::new().terminator(Some(b'\r'))) + .await + .unwrap(); + + let result = df.collect().await.unwrap(); + + let expected = [ + "+---+---+", + "| a | b |", + "+---+---+", + "| 1 | 2 |", + "| 3 | 4 |", + "+---+---+", + ]; + + crate::assert_batches_eq!(expected, &result); + + let e = session_ctx + .read_csv("memory:///", CsvReadOptions::new().terminator(Some(b'\n'))) + .await + .unwrap() + .collect() + .await + .unwrap_err(); + assert_eq!(e.strip_backtrace(), "Arrow error: Csv error: incorrect number of fields for line 1, expected 2 got more than 2") + } + + #[tokio::test] + async fn test_create_external_table_with_terminator() -> Result<()> { + let ctx = SessionContext::new(); + ctx.sql( + r#" + CREATE EXTERNAL TABLE t1 ( + col1 TEXT, + col2 TEXT + ) STORED AS CSV + LOCATION 'tests/data/cr_terminator.csv' + OPTIONS ('format.terminator' E'\r', 'format.has_header' 'true'); + "#, + ) + .await? + .collect() + .await?; + + let df = ctx.sql(r#"select * from t1"#).await?.collect().await?; + let expected = [ + "+------+--------+", + "| col1 | col2 |", + "+------+--------+", + "| id0 | value0 |", + "| id1 | value1 |", + "| id2 | value2 |", + "| id3 | value3 |", + "+------+--------+", + ]; + crate::assert_batches_eq!(expected, &df); + Ok(()) + } + + #[tokio::test] + async fn test_create_external_table_with_terminator_with_newlines_in_values( + ) -> Result<()> { + let ctx = SessionContext::new(); + ctx.sql(r#" + CREATE EXTERNAL TABLE t1 ( + col1 TEXT, + col2 TEXT + ) STORED AS CSV + LOCATION 'tests/data/newlines_in_values_cr_terminator.csv' + OPTIONS ('format.terminator' E'\r', 'format.has_header' 'true', 'format.newlines_in_values' 'true'); + "#).await?.collect().await?; + + let df = ctx.sql(r#"select * from t1"#).await?.collect().await?; + let expected = [ + "+-------+-----------------------------+", + "| col1 | col2 |", + "+-------+-----------------------------+", + "| 1 | hello\rworld |", + "| 2 | something\relse |", + "| 3 | \rmany\rlines\rmake\rgood test\r |", + "| 4 | unquoted |", + "| value | end |", + "+-------+-----------------------------+", + ]; + crate::assert_batches_eq!(expected, &df); + Ok(()) + } + #[tokio::test] async fn write_csv_results_error_handling() -> Result<()> { let ctx = SessionContext::new(); @@ -1365,6 +1498,7 @@ mod tests { has_header, delimiter, quote, + terminator, escape, comment, newlines_in_values, @@ -1374,6 +1508,7 @@ mod tests { assert_eq!(has_header, default_options.has_header.unwrap_or(false)); assert_eq!(delimiter, default_options.delimiter); assert_eq!(quote, default_options.quote); + assert_eq!(terminator, default_options.terminator); assert_eq!(escape, default_options.escape); assert_eq!(comment, default_options.comment); assert_eq!( diff --git a/datafusion/core/tests/data/cr_terminator.csv b/datafusion/core/tests/data/cr_terminator.csv new file mode 100644 index 000000000000..f2a5d09a4c19 --- /dev/null +++ b/datafusion/core/tests/data/cr_terminator.csv @@ -0,0 +1 @@ +c1,c2 id0,value0 id1,value1 id2,value2 id3,value3 \ No newline at end of file diff --git a/datafusion/core/tests/data/newlines_in_values_cr_terminator.csv b/datafusion/core/tests/data/newlines_in_values_cr_terminator.csv new file mode 100644 index 000000000000..2f6557d60ec5 --- /dev/null +++ b/datafusion/core/tests/data/newlines_in_values_cr_terminator.csv @@ -0,0 +1 @@ +id,message 1,"hello world" 2,"something else" 3," many lines make good test " 4,unquoted value,end \ No newline at end of file diff --git a/datafusion/proto-common/proto/datafusion_common.proto b/datafusion/proto-common/proto/datafusion_common.proto index 9268ccca0b70..51e94d2caaf4 100644 --- a/datafusion/proto-common/proto/datafusion_common.proto +++ b/datafusion/proto-common/proto/datafusion_common.proto @@ -423,6 +423,7 @@ message CsvOptions { bytes comment = 13; // Optional comment character as a byte bytes double_quote = 14; // Indicates if quotes are doubled bytes newlines_in_values = 15; // Indicates if newlines are supported in values + bytes terminator = 16; // Optional terminator character as a byte } // Options controlling CSV format diff --git a/datafusion/proto-common/src/from_proto/mod.rs b/datafusion/proto-common/src/from_proto/mod.rs index feb4c11aa809..45d275fb488e 100644 --- a/datafusion/proto-common/src/from_proto/mod.rs +++ b/datafusion/proto-common/src/from_proto/mod.rs @@ -863,6 +863,7 @@ impl TryFrom<&protobuf::CsvOptions> for CsvOptions { has_header: proto_opts.has_header.first().map(|h| *h != 0), delimiter: proto_opts.delimiter[0], quote: proto_opts.quote[0], + terminator: proto_opts.terminator.first().copied(), escape: proto_opts.escape.first().copied(), double_quote: proto_opts.has_header.first().map(|h| *h != 0), newlines_in_values: proto_opts.newlines_in_values.first().map(|h| *h != 0), diff --git a/datafusion/proto-common/src/generated/pbjson.rs b/datafusion/proto-common/src/generated/pbjson.rs index 05e57f5585a6..78ba829f8c50 100644 --- a/datafusion/proto-common/src/generated/pbjson.rs +++ b/datafusion/proto-common/src/generated/pbjson.rs @@ -1542,6 +1542,9 @@ impl serde::Serialize for CsvOptions { if !self.newlines_in_values.is_empty() { len += 1; } + if !self.terminator.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion_common.CsvOptions", len)?; if !self.has_header.is_empty() { #[allow(clippy::needless_borrow)] @@ -1598,6 +1601,10 @@ impl serde::Serialize for CsvOptions { #[allow(clippy::needless_borrow)] struct_ser.serialize_field("newlinesInValues", pbjson::private::base64::encode(&self.newlines_in_values).as_str())?; } + if !self.terminator.is_empty() { + #[allow(clippy::needless_borrow)] + struct_ser.serialize_field("terminator", pbjson::private::base64::encode(&self.terminator).as_str())?; + } struct_ser.end() } } @@ -1633,6 +1640,7 @@ impl<'de> serde::Deserialize<'de> for CsvOptions { "doubleQuote", "newlines_in_values", "newlinesInValues", + "terminator", ]; #[allow(clippy::enum_variant_names)] @@ -1652,6 +1660,7 @@ impl<'de> serde::Deserialize<'de> for CsvOptions { Comment, DoubleQuote, NewlinesInValues, + Terminator, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -1688,6 +1697,7 @@ impl<'de> serde::Deserialize<'de> for CsvOptions { "comment" => Ok(GeneratedField::Comment), "doubleQuote" | "double_quote" => Ok(GeneratedField::DoubleQuote), "newlinesInValues" | "newlines_in_values" => Ok(GeneratedField::NewlinesInValues), + "terminator" => Ok(GeneratedField::Terminator), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -1722,6 +1732,7 @@ impl<'de> serde::Deserialize<'de> for CsvOptions { let mut comment__ = None; let mut double_quote__ = None; let mut newlines_in_values__ = None; + let mut terminator__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::HasHeader => { @@ -1830,6 +1841,14 @@ impl<'de> serde::Deserialize<'de> for CsvOptions { Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) ; } + GeneratedField::Terminator => { + if terminator__.is_some() { + return Err(serde::de::Error::duplicate_field("terminator")); + } + terminator__ = + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + ; + } } } Ok(CsvOptions { @@ -1848,6 +1867,7 @@ impl<'de> serde::Deserialize<'de> for CsvOptions { comment: comment__.unwrap_or_default(), double_quote: double_quote__.unwrap_or_default(), newlines_in_values: newlines_in_values__.unwrap_or_default(), + terminator: terminator__.unwrap_or_default(), }) } } diff --git a/datafusion/proto-common/src/generated/prost.rs b/datafusion/proto-common/src/generated/prost.rs index ebc05718a458..cb8f86a022a6 100644 --- a/datafusion/proto-common/src/generated/prost.rs +++ b/datafusion/proto-common/src/generated/prost.rs @@ -652,6 +652,9 @@ pub struct CsvOptions { /// Indicates if newlines are supported in values #[prost(bytes = "vec", tag = "15")] pub newlines_in_values: ::prost::alloc::vec::Vec, + /// Optional terminator character as a byte + #[prost(bytes = "vec", tag = "16")] + pub terminator: ::prost::alloc::vec::Vec, } /// Options controlling CSV format #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/datafusion/proto-common/src/to_proto/mod.rs b/datafusion/proto-common/src/to_proto/mod.rs index 4cf7e73ac912..3718ccbb0f85 100644 --- a/datafusion/proto-common/src/to_proto/mod.rs +++ b/datafusion/proto-common/src/to_proto/mod.rs @@ -910,6 +910,7 @@ impl TryFrom<&CsvOptions> for protobuf::CsvOptions { has_header: opts.has_header.map_or_else(Vec::new, |h| vec![h as u8]), delimiter: vec![opts.delimiter], quote: vec![opts.quote], + terminator: opts.terminator.map_or_else(Vec::new, |e| vec![e]), escape: opts.escape.map_or_else(Vec::new, |e| vec![e]), double_quote: opts.double_quote.map_or_else(Vec::new, |h| vec![h as u8]), newlines_in_values: opts diff --git a/datafusion/proto/src/generated/datafusion_proto_common.rs b/datafusion/proto/src/generated/datafusion_proto_common.rs index 3d7b1007b04e..dc8d0017d3fd 100644 --- a/datafusion/proto/src/generated/datafusion_proto_common.rs +++ b/datafusion/proto/src/generated/datafusion_proto_common.rs @@ -652,6 +652,9 @@ pub struct CsvOptions { /// Indicates if newlines are supported in values #[prost(bytes = "vec", tag = "15")] pub newlines_in_values: ::prost::alloc::vec::Vec, + /// Optional terminator character as a byte + #[prost(bytes = "vec", tag = "16")] + pub terminator: ::prost::alloc::vec::Vec, } /// Options controlling CSV format #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/datafusion/proto/src/logical_plan/file_formats.rs b/datafusion/proto/src/logical_plan/file_formats.rs index 607a3d8642fd..2e3476da6ac0 100644 --- a/datafusion/proto/src/logical_plan/file_formats.rs +++ b/datafusion/proto/src/logical_plan/file_formats.rs @@ -53,6 +53,7 @@ impl CsvOptionsProto { has_header: options.has_header.map_or(vec![], |v| vec![v as u8]), delimiter: vec![options.delimiter], quote: vec![options.quote], + terminator: options.terminator.map_or(vec![], |v| vec![v]), escape: options.escape.map_or(vec![], |v| vec![v]), double_quote: options.double_quote.map_or(vec![], |v| vec![v as u8]), compression: options.compression as i32, @@ -87,6 +88,11 @@ impl From<&CsvOptionsProto> for CsvOptions { }, delimiter: proto.delimiter.first().copied().unwrap_or(b','), quote: proto.quote.first().copied().unwrap_or(b'"'), + terminator: if !proto.terminator.is_empty() { + Some(proto.terminator[0]) + } else { + None + }, escape: if !proto.escape.is_empty() { Some(proto.escape[0]) } else { diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs index c32acecaae5f..2531795a1630 100644 --- a/datafusion/sql/src/utils.rs +++ b/datafusion/sql/src/utils.rs @@ -268,8 +268,8 @@ pub(crate) fn value_to_string(value: &Value) -> Option { Value::DollarQuotedString(s) => Some(s.to_string()), Value::Number(_, _) | Value::Boolean(_) => Some(value.to_string()), Value::UnicodeStringLiteral(s) => Some(s.to_string()), + Value::EscapedStringLiteral(s) => Some(s.to_string()), Value::DoubleQuotedString(_) - | Value::EscapedStringLiteral(_) | Value::NationalStringLiteral(_) | Value::SingleQuotedByteStringLiteral(_) | Value::DoubleQuotedByteStringLiteral(_) diff --git a/datafusion/sqllogictest/test_files/csv_files.slt b/datafusion/sqllogictest/test_files/csv_files.slt index 7cb21abdba10..d6600e06dc1c 100644 --- a/datafusion/sqllogictest/test_files/csv_files.slt +++ b/datafusion/sqllogictest/test_files/csv_files.slt @@ -336,3 +336,24 @@ id message 05)good test 4 unquoted value end + +statement ok +CREATE EXTERNAL TABLE stored_table_with_cr_terminator ( +col1 TEXT, +col2 TEXT +) STORED AS CSV +LOCATION '../core/tests/data/cr_terminator.csv' +OPTIONS ('format.terminator' E'\r', 'format.has_header' 'true'); + +# TODO: It should be passed but got the error: External error: query failed: DataFusion error: Object Store error: Generic LocalFileSystem error: Requested range was invalid +# See the issue: https://github.com/apache/datafusion/issues/12328 +# query TT +# select * from stored_table_with_cr_terminator; +# ---- +# id0 value0 +# id1 value1 +# id2 value2 +# id3 value3 + +statement ok +drop table stored_table_with_cr_terminator; From d6f3f738e88988daba725468eb552a40ba98a1de Mon Sep 17 00:00:00 2001 From: Georgi Krastev Date: Thu, 5 Sep 2024 14:20:47 +0300 Subject: [PATCH 75/75] Support try_from_array and eq_array for ScalarValue::Union (#12208) --- datafusion/common/src/scalar/mod.rs | 124 +++++++++++++++++++++++++++- 1 file changed, 122 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 22e39404cdb5..0cb325e0b02b 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -2800,6 +2800,13 @@ impl ScalarValue { let a = array.slice(index, 1); Self::Map(Arc::new(a.as_map().to_owned())) } + DataType::Union(fields, mode) => { + let array = as_union_array(array); + let ti = array.type_id(index); + let index = array.value_offset(index); + let value = ScalarValue::try_from_array(array.child(ti), index)?; + ScalarValue::Union(Some((ti, Box::new(value))), fields.clone(), *mode) + } other => { return _not_impl_err!( "Can't create a scalar from array of type \"{other:?}\"" @@ -3035,8 +3042,15 @@ impl ScalarValue { ScalarValue::DurationNanosecond(val) => { eq_array_primitive!(array, index, DurationNanosecondArray, val)? } - ScalarValue::Union(_, _, _) => { - return _not_impl_err!("Union is not supported yet") + ScalarValue::Union(value, _, _) => { + let array = as_union_array(array); + let ti = array.type_id(index); + let index = array.value_offset(index); + if let Some((ti_v, value)) = value { + ti_v == &ti && value.eq_array(array.child(ti), index)? + } else { + array.child(ti).is_null(index) + } } ScalarValue::Dictionary(key_type, v) => { let (values_array, values_index) = match key_type.as_ref() { @@ -5536,6 +5550,112 @@ mod tests { assert_eq!(&array, &expected); } + #[test] + fn test_scalar_union_sparse() { + let field_a = Arc::new(Field::new("A", DataType::Int32, true)); + let field_b = Arc::new(Field::new("B", DataType::Boolean, true)); + let field_c = Arc::new(Field::new("C", DataType::Utf8, true)); + let fields = UnionFields::from_iter([(0, field_a), (1, field_b), (2, field_c)]); + + let mut values_a = vec![None; 6]; + values_a[0] = Some(42); + let mut values_b = vec![None; 6]; + values_b[1] = Some(true); + let mut values_c = vec![None; 6]; + values_c[2] = Some("foo"); + let children: Vec = vec![ + Arc::new(Int32Array::from(values_a)), + Arc::new(BooleanArray::from(values_b)), + Arc::new(StringArray::from(values_c)), + ]; + + let type_ids = ScalarBuffer::from(vec![0, 1, 2, 0, 1, 2]); + let array: ArrayRef = Arc::new( + UnionArray::try_new(fields.clone(), type_ids, None, children) + .expect("UnionArray"), + ); + + let expected = [ + (0, ScalarValue::from(42)), + (1, ScalarValue::from(true)), + (2, ScalarValue::from("foo")), + (0, ScalarValue::Int32(None)), + (1, ScalarValue::Boolean(None)), + (2, ScalarValue::Utf8(None)), + ]; + + for (i, (ti, value)) in expected.into_iter().enumerate() { + let is_null = value.is_null(); + let value = Some((ti, Box::new(value))); + let expected = ScalarValue::Union(value, fields.clone(), UnionMode::Sparse); + let actual = ScalarValue::try_from_array(&array, i).expect("try_from_array"); + + assert_eq!( + actual, expected, + "[{i}] {actual} was not equal to {expected}" + ); + + assert!( + expected.eq_array(&array, i).expect("eq_array"), + "[{i}] {expected}.eq_array was false" + ); + + if is_null { + assert!(actual.is_null(), "[{i}] {actual} was not null") + } + } + } + + #[test] + fn test_scalar_union_dense() { + let field_a = Arc::new(Field::new("A", DataType::Int32, true)); + let field_b = Arc::new(Field::new("B", DataType::Boolean, true)); + let field_c = Arc::new(Field::new("C", DataType::Utf8, true)); + let fields = UnionFields::from_iter([(0, field_a), (1, field_b), (2, field_c)]); + let children: Vec = vec![ + Arc::new(Int32Array::from(vec![Some(42), None])), + Arc::new(BooleanArray::from(vec![Some(true), None])), + Arc::new(StringArray::from(vec![Some("foo"), None])), + ]; + + let type_ids = ScalarBuffer::from(vec![0, 1, 2, 0, 1, 2]); + let offsets = ScalarBuffer::from(vec![0, 0, 0, 1, 1, 1]); + let array: ArrayRef = Arc::new( + UnionArray::try_new(fields.clone(), type_ids, Some(offsets), children) + .expect("UnionArray"), + ); + + let expected = [ + (0, ScalarValue::from(42)), + (1, ScalarValue::from(true)), + (2, ScalarValue::from("foo")), + (0, ScalarValue::Int32(None)), + (1, ScalarValue::Boolean(None)), + (2, ScalarValue::Utf8(None)), + ]; + + for (i, (ti, value)) in expected.into_iter().enumerate() { + let is_null = value.is_null(); + let value = Some((ti, Box::new(value))); + let expected = ScalarValue::Union(value, fields.clone(), UnionMode::Dense); + let actual = ScalarValue::try_from_array(&array, i).expect("try_from_array"); + + assert_eq!( + actual, expected, + "[{i}] {actual} was not equal to {expected}" + ); + + assert!( + expected.eq_array(&array, i).expect("eq_array"), + "[{i}] {expected}.eq_array was false" + ); + + if is_null { + assert!(actual.is_null(), "[{i}] {actual} was not null") + } + } + } + #[test] fn test_lists_in_struct() { let field_a = Arc::new(Field::new("A", DataType::Utf8, false));