From 95f419b077437f6a33c6c2c134a78f9a725ac827 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 14 Sep 2023 17:26:26 +0300 Subject: [PATCH 01/53] analysis context refactored --- datafusion/common/src/stats.rs | 46 ++++++ datafusion/core/src/physical_plan/filter.rs | 147 +++++++++++++----- datafusion/physical-expr/src/analysis.rs | 54 ++++--- .../src/intervals/interval_aritmetic.rs | 71 +++++---- 4 files changed, 212 insertions(+), 106 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index db788efef7cd..392d812eec7e 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -17,6 +17,7 @@ //! This module provides data structures to represent statistics +use arrow::datatypes::{DataType, SchemaRef}; use std::fmt::Display; use crate::ScalarValue; @@ -39,6 +40,38 @@ pub struct Statistics { pub is_exact: bool, } +impl Statistics { + /// Returns a [`Statistics`] instance corresponding to the given schema by assigning infinite + /// bounds to each column in the schema. This is useful when the input statistics are not + /// known to given an opportunity to the current executor to shrink the bounds of some columns. + pub fn new_with_unbounded_columns(schema: SchemaRef) -> Self { + let data_types = schema + .fields() + .iter() + .map(|field| field.data_type()) + .collect::>(); + Self { + num_rows: None, + total_byte_size: None, + column_statistics: Some( + data_types + .into_iter() + .map(|data_type| { + let dt = ScalarValue::try_from(data_type.clone()).ok(); + ColumnStatistics { + null_count: None, + max_value: dt.clone(), + min_value: dt, + distinct_count: None, + } + }) + .collect(), + ), + is_exact: false, + } + } +} + impl Display for Statistics { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { if self.num_rows.is_none() && self.total_byte_size.is_none() && !self.is_exact { @@ -70,3 +103,16 @@ pub struct ColumnStatistics { /// Number of distinct values pub distinct_count: Option, } + +impl ColumnStatistics { + /// Returns the [`ColumnStatistics`] corresponding to the given datatype by assigning infinite bounds. + pub fn new_with_unbounded_column(dt: &DataType) -> ColumnStatistics { + let null = ScalarValue::try_from(dt.clone()).ok(); + ColumnStatistics { + null_count: None, + max_value: null.clone(), + min_value: null, + distinct_count: None, + } + } +} diff --git a/datafusion/core/src/physical_plan/filter.rs b/datafusion/core/src/physical_plan/filter.rs index 0b878814e305..8d15401f74ad 100644 --- a/datafusion/core/src/physical_plan/filter.rs +++ b/datafusion/core/src/physical_plan/filter.rs @@ -36,6 +36,7 @@ use crate::physical_plan::{ use arrow::compute::filter_record_batch; use arrow::datatypes::{DataType, SchemaRef}; use arrow::record_batch::RecordBatch; +use arrow_schema::Schema; use datafusion_common::cast::as_boolean_array; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; @@ -190,46 +191,64 @@ impl ExecutionPlan for FilterExec { fn statistics(&self) -> Statistics { let predicate = self.predicate(); - if !check_support(predicate) { - return Statistics::default(); + if check_support(predicate) { + let input_stats = self.input.statistics(); + + if let Some(column_stats) = input_stats.column_statistics { + if let Ok(analysis_ctx) = + analyze_with_context(&self.input.schema(), &column_stats, predicate) + { + return calculate_statistics( + input_stats.num_rows, + input_stats.total_byte_size, + &column_stats, + input_stats.is_exact, + analysis_ctx, + ); + } + } } - let input_stats = self.input.statistics(); - let input_column_stats = match input_stats.column_statistics { - Some(stats) => stats, - None => return Statistics::default(), - }; - - let starter_ctx = - AnalysisContext::from_statistics(&self.input.schema(), &input_column_stats); - - let analysis_ctx = match analyze(predicate, starter_ctx) { - Ok(ctx) => ctx, - Err(_) => return Statistics::default(), - }; - - let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); - - let num_rows = input_stats - .num_rows - .map(|num| (num as f64 * selectivity).ceil() as usize); - let total_byte_size = input_stats - .total_byte_size - .map(|size| (size as f64 * selectivity).ceil() as usize); + Statistics::new_with_unbounded_columns(self.schema()) + } +} - let column_statistics = if let Some(analysis_boundaries) = analysis_ctx.boundaries - { - collect_new_statistics(input_column_stats, selectivity, analysis_boundaries) - } else { - input_column_stats - }; +fn analyze_with_context( + input_schema: &Schema, + column_stats: &[ColumnStatistics], + predicate: &Arc, +) -> Result { + let starter_ctx = AnalysisContext::from_statistics(input_schema, column_stats); + analyze(predicate, starter_ctx) +} - Statistics { +fn calculate_statistics( + num_rows: Option, + total_byte_size: Option, + input_stats: &[ColumnStatistics], + is_exact: bool, + analysis_ctx: AnalysisContext, +) -> Statistics { + let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); + let num_rows = num_rows.map(|num| (num as f64 * selectivity).ceil() as usize); + let total_byte_size = + total_byte_size.map(|size| (size as f64 * selectivity).ceil() as usize); + + if let Some(analysis_boundaries) = analysis_ctx.boundaries { + let column_statistics = collect_new_statistics(input_stats, analysis_boundaries); + return Statistics { num_rows, total_byte_size, column_statistics: Some(column_statistics), - is_exact: Default::default(), - } + is_exact, + }; + } + + Statistics { + num_rows, + total_byte_size, + column_statistics: Some(input_stats.to_vec()), + is_exact, } } @@ -238,11 +257,9 @@ impl ExecutionPlan for FilterExec { /// is adjusted by using the next/previous value for its data type to convert /// it into a closed bound. fn collect_new_statistics( - input_column_stats: Vec, - selectivity: f64, + input_column_stats: &[ColumnStatistics], analysis_boundaries: Vec, ) -> Vec { - let nonempty_columns = selectivity > 0.0; analysis_boundaries .into_iter() .enumerate() @@ -258,8 +275,8 @@ fn collect_new_statistics( let closed_interval = interval.close_bounds(); ColumnStatistics { null_count: input_column_stats[idx].null_count, - max_value: nonempty_columns.then_some(closed_interval.upper.value), - min_value: nonempty_columns.then_some(closed_interval.lower.value), + max_value: Some(closed_interval.upper.value), + min_value: Some(closed_interval.lower.value), distinct_count, } }, @@ -906,13 +923,13 @@ mod tests { statistics.column_statistics, Some(vec![ ColumnStatistics { - min_value: None, - max_value: None, + min_value: Some(ScalarValue::Int32(Some(1))), + max_value: Some(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: None, - max_value: None, + min_value: Some(ScalarValue::Int32(Some(1))), + max_value: Some(ScalarValue::Int32(Some(3))), ..Default::default() }, ]) @@ -977,4 +994,50 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_empty_input_statistics() -> Result<()> { + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let input = Arc::new(StatisticsExec::new( + Statistics::new_with_unbounded_columns(Arc::new(schema.clone())), + schema, + )); + // WHERE a <= 10 AND 0 <= a - 5 + let predicate = Arc::new(BinaryExpr::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::LtEq, + Arc::new(Literal::new(ScalarValue::Int32(Some(10)))), + )), + Operator::And, + Arc::new(BinaryExpr::new( + Arc::new(Literal::new(ScalarValue::Int32(Some(0)))), + Operator::LtEq, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Minus, + Arc::new(Literal::new(ScalarValue::Int32(Some(5)))), + )), + )), + )); + let filter: Arc = + Arc::new(FilterExec::try_new(predicate, input)?); + let filter_statistics = filter.statistics(); + + let expected_filter_statistics = Statistics { + num_rows: None, + total_byte_size: None, + column_statistics: Some(vec![ColumnStatistics { + null_count: None, + min_value: Some(ScalarValue::Int32(Some(5))), + max_value: Some(ScalarValue::Int32(Some(10))), + distinct_count: None, + }]), + is_exact: false, + }; + + assert_eq!(filter_statistics, expected_filter_statistics); + + Ok(()) + } } diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index d3fcdc11ad52..d0f386991473 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -17,20 +17,19 @@ //! Interval and selectivity in [`AnalysisContext`] +use std::fmt::Debug; +use std::sync::Arc; + +use arrow::datatypes::Schema; + +use datafusion_common::{ColumnStatistics, DataFusionError, Result, ScalarValue}; + use crate::expressions::Column; use crate::intervals::cp_solver::PropagationResult; use crate::intervals::{cardinality_ratio, ExprIntervalGraph, Interval, IntervalBound}; use crate::utils::collect_columns; use crate::PhysicalExpr; -use arrow::datatypes::Schema; -use datafusion_common::{ - internal_err, ColumnStatistics, DataFusionError, Result, ScalarValue, -}; - -use std::fmt::Debug; -use std::sync::Arc; - /// The shared context used during the analysis of an expression. Includes /// the boundaries for all known columns. #[derive(Clone, Debug, PartialEq)] @@ -150,9 +149,12 @@ pub fn analyze( .collect(); match graph.update_ranges(&mut target_indices_and_boundaries)? { - PropagationResult::Success => { - shrink_boundaries(expr, graph, target_boundaries, target_expr_and_indices) - } + PropagationResult::Success => Ok(shrink_boundaries( + expr, + graph, + target_boundaries, + target_expr_and_indices, + )), PropagationResult::Infeasible => { Ok(AnalysisContext::new(target_boundaries).with_selectivity(0.0)) } @@ -171,7 +173,7 @@ fn shrink_boundaries( mut graph: ExprIntervalGraph, mut target_boundaries: Vec, target_expr_and_indices: Vec<(Arc, usize)>, -) -> Result { +) -> AnalysisContext { let initial_boundaries = target_boundaries.clone(); target_expr_and_indices.iter().for_each(|(expr, i)| { if let Some(column) = expr.as_any().downcast_ref::() { @@ -184,23 +186,19 @@ fn shrink_boundaries( } }); let graph_nodes = graph.gather_node_indices(&[expr.clone()]); - let (_, root_index) = graph_nodes.first().ok_or_else(|| { - DataFusionError::Internal("Error in constructing predicate graph".to_string()) - })?; - let final_result = graph.get_interval(*root_index); + // Since the propagation result success, the graph has at least one element, and + // empty check is also done at the outer scope. + let (_, root_index) = graph_nodes[0]; + let final_result = graph.get_interval(root_index); let selectivity = calculate_selectivity( &final_result.lower.value, &final_result.upper.value, &target_boundaries, &initial_boundaries, - )?; - - if !(0.0..=1.0).contains(&selectivity) { - return internal_err!("Selectivity is out of limit: {}", selectivity); - } + ); - Ok(AnalysisContext::new(target_boundaries).with_selectivity(selectivity)) + AnalysisContext::new(target_boundaries).with_selectivity(selectivity) } /// This function calculates the filter predicate's selectivity by comparing @@ -217,20 +215,20 @@ fn calculate_selectivity( upper_value: &ScalarValue, target_boundaries: &[ExprBoundaries], initial_boundaries: &[ExprBoundaries], -) -> Result { +) -> f64 { match (lower_value, upper_value) { - (ScalarValue::Boolean(Some(true)), ScalarValue::Boolean(Some(true))) => Ok(1.0), - (ScalarValue::Boolean(Some(false)), ScalarValue::Boolean(Some(false))) => Ok(0.0), + (ScalarValue::Boolean(Some(true)), ScalarValue::Boolean(Some(true))) => 1.0, + (ScalarValue::Boolean(Some(false)), ScalarValue::Boolean(Some(false))) => 0.0, _ => { // Since the intervals are assumed uniform and the values // are not correlated, we need to multiply the selectivities // of multiple columns to get the overall selectivity. - target_boundaries.iter().enumerate().try_fold( + target_boundaries.iter().enumerate().fold( 1.0, |acc, (i, ExprBoundaries { interval, .. })| { let temp = - cardinality_ratio(&initial_boundaries[i].interval, interval)?; - Ok(acc * temp) + cardinality_ratio(&initial_boundaries[i].interval, interval); + acc * temp }, ) } diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index 5501c8cae090..d3d6634755b3 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -22,16 +22,17 @@ use std::fmt; use std::fmt::{Display, Formatter}; use std::ops::{AddAssign, SubAssign}; -use crate::aggregate::min_max::{max, min}; -use crate::intervals::rounding::{alter_fp_rounding_mode, next_down, next_up}; - use arrow::compute::{cast_with_options, CastOptions}; use arrow::datatypes::DataType; use arrow_array::ArrowNativeTypeOp; -use datafusion_common::{exec_err, internal_err, DataFusionError, Result, ScalarValue}; + +use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::type_coercion::binary::get_result_type; use datafusion_expr::Operator; +use crate::aggregate::min_max::{max, min}; +use crate::intervals::rounding::{alter_fp_rounding_mode, next_down, next_up}; + /// This type represents a single endpoint of an [`Interval`]. An /// endpoint can be open (does not include the endpoint) or closed /// (includes the endpoint). @@ -498,20 +499,21 @@ impl Interval { upper: IntervalBound::new_closed(ScalarValue::Boolean(Some(true))), }; - /// Returns the cardinality of this interval, which is the number of all - /// distinct points inside it. - pub fn cardinality(&self) -> Result { + /// Returns the cardinality of this interval which is the number of all + /// distinct points inside it. Under these conditions the function return None: + /// - If any of the bounds is an infinite bound + /// - If the type is not implemented yet + /// - If there is an overflow during a computation + pub fn cardinality(&self) -> Option { match self.get_datatype() { Ok(data_type) if data_type.is_integer() => { - if let Some(diff) = self.upper.value.distance(&self.lower.value) { - Ok(calculate_cardinality_based_on_bounds( + self.upper.value.distance(&self.lower.value).map(|diff| { + calculate_cardinality_based_on_bounds( self.lower.open, self.upper.open, diff as u64, - )) - } else { - exec_err!("Cardinality cannot be calculated for {:?}", self) - } + ) + }) } // Ordering floating-point numbers according to their binary representations // coincide with their natural ordering. Therefore, we can consider their @@ -520,9 +522,9 @@ impl Interval { Ok(data_type) if data_type.is_floating() => { // If the minimum value is a negative number, we need to // switch sides to ensure an unsigned result. - let (min, max) = if self.lower.value - < ScalarValue::new_zero(&self.lower.value.data_type())? - { + let new_zero = + ScalarValue::new_zero(&self.lower.value.data_type()).ok()?; + let (min, max) = if self.lower.value < new_zero { (self.upper.value.clone(), self.lower.value.clone()) } else { (self.lower.value.clone(), self.upper.value.clone()) @@ -532,27 +534,23 @@ impl Interval { ( ScalarValue::Float32(Some(lower)), ScalarValue::Float32(Some(upper)), - ) => Ok(calculate_cardinality_based_on_bounds( + ) => Some(calculate_cardinality_based_on_bounds( self.lower.open, self.upper.open, - (upper.to_bits().sub_checked(lower.to_bits()))? as u64, + (upper.to_bits().sub_checked(lower.to_bits()).ok()?) as u64, )), ( ScalarValue::Float64(Some(lower)), ScalarValue::Float64(Some(upper)), - ) => Ok(calculate_cardinality_based_on_bounds( + ) => Some(calculate_cardinality_based_on_bounds( self.lower.open, self.upper.open, - upper.to_bits().sub_checked(lower.to_bits())?, + upper.to_bits().sub_checked(lower.to_bits()).ok()?, )), - _ => exec_err!( - "Cardinality cannot be calculated for the datatype {:?}", - data_type - ), + _ => None, } } - // If the cardinality cannot be calculated anyway, give an error. - _ => exec_err!("Cardinality cannot be calculated for {:?}", self), + _ => None, } } @@ -656,11 +654,13 @@ fn next_value(value: ScalarValue) -> ScalarValue { } /// This function computes the cardinality ratio of the given intervals. -pub fn cardinality_ratio( - initial_interval: &Interval, - final_interval: &Interval, -) -> Result { - Ok(final_interval.cardinality()? as f64 / initial_interval.cardinality()? as f64) +pub fn cardinality_ratio(initial_interval: &Interval, final_interval: &Interval) -> f64 { + match (final_interval.cardinality(), initial_interval.cardinality()) { + (Some(final_interval), Some(initial_interval)) => { + final_interval as f64 / initial_interval as f64 + } + _ => 1.0, + } } pub fn apply_operator(op: &Operator, lhs: &Interval, rhs: &Interval) -> Result { @@ -982,7 +982,6 @@ impl NullableInterval { mod tests { use super::next_value; use crate::intervals::{Interval, IntervalBound}; - use arrow_schema::DataType; use datafusion_common::{Result, ScalarValue}; @@ -1680,7 +1679,7 @@ mod tests { ), ]; for interval in intervals { - assert_eq!(interval.cardinality()?, distinct_f64); + assert_eq!(interval.cardinality().unwrap(), distinct_f64); } let intervals = [ @@ -1694,20 +1693,20 @@ mod tests { ), ]; for interval in intervals { - assert_eq!(interval.cardinality()?, distinct_f32); + assert_eq!(interval.cardinality().unwrap(), distinct_f32); } let interval = Interval::new( IntervalBound::new(ScalarValue::from(-0.0625), false), IntervalBound::new(ScalarValue::from(0.0625), true), ); - assert_eq!(interval.cardinality()?, distinct_f64 * 2_048); + assert_eq!(interval.cardinality().unwrap(), distinct_f64 * 2_048); let interval = Interval::new( IntervalBound::new(ScalarValue::from(-0.0625_f32), false), IntervalBound::new(ScalarValue::from(0.0625_f32), true), ); - assert_eq!(interval.cardinality()?, distinct_f32 * 256); + assert_eq!(interval.cardinality().unwrap(), distinct_f32 * 256); Ok(()) } From 62be1e227d3e323372404815f8e4e48e0fe574ea Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 14 Sep 2023 17:49:08 +0300 Subject: [PATCH 02/53] is_exact fix --- datafusion/common/src/stats.rs | 2 +- datafusion/core/src/physical_plan/filter.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 392d812eec7e..000fe6b1f478 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -43,7 +43,7 @@ pub struct Statistics { impl Statistics { /// Returns a [`Statistics`] instance corresponding to the given schema by assigning infinite /// bounds to each column in the schema. This is useful when the input statistics are not - /// known to given an opportunity to the current executor to shrink the bounds of some columns. + /// known to give an opportunity to the current executor to shrink the bounds of some columns. pub fn new_with_unbounded_columns(schema: SchemaRef) -> Self { let data_types = schema .fields() diff --git a/datafusion/core/src/physical_plan/filter.rs b/datafusion/core/src/physical_plan/filter.rs index 8d15401f74ad..99d90ded2e75 100644 --- a/datafusion/core/src/physical_plan/filter.rs +++ b/datafusion/core/src/physical_plan/filter.rs @@ -202,7 +202,7 @@ impl ExecutionPlan for FilterExec { input_stats.num_rows, input_stats.total_byte_size, &column_stats, - input_stats.is_exact, + false, analysis_ctx, ); } From 76da729518a633ce4ebb15f222929cb54f80c71b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 15 Sep 2023 12:56:48 +0300 Subject: [PATCH 03/53] Minor changes --- datafusion/common/src/stats.rs | 18 +++++++----------- 1 file changed, 7 insertions(+), 11 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 000fe6b1f478..fbc506c7e041 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -45,23 +45,19 @@ impl Statistics { /// bounds to each column in the schema. This is useful when the input statistics are not /// known to give an opportunity to the current executor to shrink the bounds of some columns. pub fn new_with_unbounded_columns(schema: SchemaRef) -> Self { - let data_types = schema - .fields() - .iter() - .map(|field| field.data_type()) - .collect::>(); Self { num_rows: None, total_byte_size: None, column_statistics: Some( - data_types - .into_iter() - .map(|data_type| { - let dt = ScalarValue::try_from(data_type.clone()).ok(); + schema + .fields() + .iter() + .map(|field| { + let inf = ScalarValue::try_from(field.data_type()).ok(); ColumnStatistics { null_count: None, - max_value: dt.clone(), - min_value: dt, + max_value: inf.clone(), + min_value: inf, distinct_count: None, } }) From 1bb940d81e556010e1a26ea466f58b0809b37b67 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 15 Sep 2023 13:31:19 +0300 Subject: [PATCH 04/53] minor changes --- datafusion/core/src/physical_plan/filter.rs | 30 +++++++-------------- 1 file changed, 10 insertions(+), 20 deletions(-) diff --git a/datafusion/core/src/physical_plan/filter.rs b/datafusion/core/src/physical_plan/filter.rs index 99d90ded2e75..ed93474f84e8 100644 --- a/datafusion/core/src/physical_plan/filter.rs +++ b/datafusion/core/src/physical_plan/filter.rs @@ -195,17 +195,16 @@ impl ExecutionPlan for FilterExec { let input_stats = self.input.statistics(); if let Some(column_stats) = input_stats.column_statistics { - if let Ok(analysis_ctx) = - analyze_with_context(&self.input.schema(), &column_stats, predicate) - { - return calculate_statistics( - input_stats.num_rows, - input_stats.total_byte_size, - &column_stats, - false, - analysis_ctx, - ); - } + let starter_ctx = AnalysisContext::from_statistics(&self.input.schema(), &column_stats); + let analysis_context = analyze(predicate, starter_ctx).unwrap(); + return calculate_statistics( + input_stats.num_rows, + input_stats.total_byte_size, + &column_stats, + false, + analysis_context, + ); + } } @@ -213,15 +212,6 @@ impl ExecutionPlan for FilterExec { } } -fn analyze_with_context( - input_schema: &Schema, - column_stats: &[ColumnStatistics], - predicate: &Arc, -) -> Result { - let starter_ctx = AnalysisContext::from_statistics(input_schema, column_stats); - analyze(predicate, starter_ctx) -} - fn calculate_statistics( num_rows: Option, total_byte_size: Option, From c380f4a24e6f87068ef2ea2df9ac4ca20743de6c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 15 Sep 2023 13:37:11 +0300 Subject: [PATCH 05/53] Minor changes --- datafusion/core/src/physical_plan/filter.rs | 70 +++++++++------------ 1 file changed, 29 insertions(+), 41 deletions(-) diff --git a/datafusion/core/src/physical_plan/filter.rs b/datafusion/core/src/physical_plan/filter.rs index ed93474f84e8..4b55f00f9786 100644 --- a/datafusion/core/src/physical_plan/filter.rs +++ b/datafusion/core/src/physical_plan/filter.rs @@ -36,7 +36,6 @@ use crate::physical_plan::{ use arrow::compute::filter_record_batch; use arrow::datatypes::{DataType, SchemaRef}; use arrow::record_batch::RecordBatch; -use arrow_schema::Schema; use datafusion_common::cast::as_boolean_array; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; @@ -195,16 +194,35 @@ impl ExecutionPlan for FilterExec { let input_stats = self.input.statistics(); if let Some(column_stats) = input_stats.column_statistics { - let starter_ctx = AnalysisContext::from_statistics(&self.input.schema(), &column_stats); - let analysis_context = analyze(predicate, starter_ctx).unwrap(); - return calculate_statistics( - input_stats.num_rows, - input_stats.total_byte_size, - &column_stats, - false, - analysis_context, - ); - + let num_rows = input_stats.num_rows; + let total_byte_size = input_stats.total_byte_size; + let input_analysis_ctx = + AnalysisContext::from_statistics(&self.input.schema(), &column_stats); + let analysis_ctx = analyze(predicate, input_analysis_ctx).unwrap(); + + let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); + let num_rows = + num_rows.map(|num| (num as f64 * selectivity).ceil() as usize); + let total_byte_size = total_byte_size + .map(|size| (size as f64 * selectivity).ceil() as usize); + + return if let Some(analysis_boundaries) = analysis_ctx.boundaries { + let column_statistics = + collect_new_statistics(&column_stats, analysis_boundaries); + Statistics { + num_rows, + total_byte_size, + column_statistics: Some(column_statistics), + is_exact: false, + } + } else { + Statistics { + num_rows, + total_byte_size, + column_statistics: Some(column_stats.to_vec()), + is_exact: false, + } + }; } } @@ -212,36 +230,6 @@ impl ExecutionPlan for FilterExec { } } -fn calculate_statistics( - num_rows: Option, - total_byte_size: Option, - input_stats: &[ColumnStatistics], - is_exact: bool, - analysis_ctx: AnalysisContext, -) -> Statistics { - let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); - let num_rows = num_rows.map(|num| (num as f64 * selectivity).ceil() as usize); - let total_byte_size = - total_byte_size.map(|size| (size as f64 * selectivity).ceil() as usize); - - if let Some(analysis_boundaries) = analysis_ctx.boundaries { - let column_statistics = collect_new_statistics(input_stats, analysis_boundaries); - return Statistics { - num_rows, - total_byte_size, - column_statistics: Some(column_statistics), - is_exact, - }; - } - - Statistics { - num_rows, - total_byte_size, - column_statistics: Some(input_stats.to_vec()), - is_exact, - } -} - /// This function ensures that all bounds in the `ExprBoundaries` vector are /// converted to closed bounds. If a lower/upper bound is initially open, it /// is adjusted by using the next/previous value for its data type to convert From b801e46c6c5cc16723a9773b88132e544cdb7e14 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 15 Sep 2023 14:52:38 +0300 Subject: [PATCH 06/53] Minor changes --- .../datasource/physical_plan/file_scan_config.rs | 13 ++++++++----- datafusion/core/src/physical_plan/filter.rs | 9 ++++++++- datafusion/core/src/physical_planner.rs | 8 +++++++- datafusion/physical-expr/src/analysis.rs | 7 +++++-- .../physical-expr/src/expressions/binary.rs | 3 ++- datafusion/physical-expr/src/expressions/cast.rs | 3 +++ .../physical-expr/src/intervals/cp_solver.rs | 15 +++++++++++++-- 7 files changed, 46 insertions(+), 12 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 e9ce5238c596..d4436e288971 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -130,21 +130,24 @@ impl FileScanConfig { let mut table_cols_stats = vec![]; for idx in proj_iter { if idx < self.file_schema.fields().len() { - table_fields.push(self.file_schema.field(idx).clone()); + let field = self.file_schema.field(idx).clone(); + table_fields.push(field.clone()); if let Some(file_cols_stats) = &self.statistics.column_statistics { table_cols_stats.push(file_cols_stats[idx].clone()) } else { - table_cols_stats.push(ColumnStatistics::default()) + table_cols_stats.push(ColumnStatistics::new_with_unbounded_column(field.data_type())) } } else { let partition_idx = idx - self.file_schema.fields().len(); + let name = &self.table_partition_cols[partition_idx].0; + let dtype = &self.table_partition_cols[partition_idx].1; table_fields.push(Field::new( - &self.table_partition_cols[partition_idx].0, - self.table_partition_cols[partition_idx].1.to_owned(), + name, + dtype.to_owned(), false, )); // TODO provide accurate stat for partition column (#1186) - table_cols_stats.push(ColumnStatistics::default()) + table_cols_stats.push(ColumnStatistics::new_with_unbounded_column(dtype)) } } diff --git a/datafusion/core/src/physical_plan/filter.rs b/datafusion/core/src/physical_plan/filter.rs index 4b55f00f9786..54ddbf6802fb 100644 --- a/datafusion/core/src/physical_plan/filter.rs +++ b/datafusion/core/src/physical_plan/filter.rs @@ -189,7 +189,8 @@ impl ExecutionPlan for FilterExec { /// predicate's selectivity value can be determined for the incoming data. fn statistics(&self) -> Statistics { let predicate = self.predicate(); - + print_plan(&self.input); + println!("self.schema.fields:{:?}", self.schema().fields); if check_support(predicate) { let input_stats = self.input.statistics(); @@ -262,6 +263,12 @@ fn collect_new_statistics( .collect() } +fn print_plan(plan: &Arc) -> () { + let formatted = crate::physical_plan::displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); +} + /// The FilterExec streams wraps the input iterator and applies the predicate expression to /// determine which rows to include in its output batches struct FilterExecStream { diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index def4d59873df..4a9c7a49b9ba 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2157,6 +2157,12 @@ mod tests { Ok(()) } + fn print_plan(plan: &Arc) -> () { + let formatted = crate::physical_plan::displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); + } + #[tokio::test] async fn test_with_csv_plan() -> Result<()> { let logical_plan = test_csv_scan() @@ -2166,7 +2172,7 @@ mod tests { .build()?; let plan = plan(&logical_plan).await?; - + print_plan(&plan); // c12 is f64, c7 is u8 -> cast c7 to f64 // the cast here is implicit so has CastOptions with safe=true let _expected = "predicate: BinaryExpr { left: TryCastExpr { expr: Column { name: \"c7\", index: 6 }, cast_type: Float64 }, op: Lt, right: Column { name: \"c12\", index: 11 } }"; diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index d0f386991473..2d54f1b624d6 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -125,8 +125,9 @@ pub fn analyze( DataFusionError::Internal("No column exists at the input to filter".to_string()) })?; + println!("start expr:{:?}", expr); let mut graph = ExprIntervalGraph::try_new(expr.clone())?; - + println!("created graph"); let columns: Vec> = collect_columns(expr) .into_iter() .map(|c| Arc::new(c) as Arc) @@ -148,7 +149,9 @@ pub fn analyze( }) .collect(); - match graph.update_ranges(&mut target_indices_and_boundaries)? { + let res = graph.update_ranges(&mut target_indices_and_boundaries)?; + println!("update range done"); + match res { PropagationResult::Success => Ok(shrink_boundaries( expr, graph, diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index f235ea9fcdaf..e3aee58dea20 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -375,12 +375,13 @@ impl PhysicalExpr for BinaryExpr { // changes accordingly. return Ok(vec![]); } else if self.op.is_comparison_operator() { - if interval == &Interval::CERTAINLY_FALSE { + if interval == &Interval::CERTAINLY_FALSE || self.op == Operator::Eq { // TODO: We will handle strictly false clauses by negating // the comparison operator (e.g. GT to LE, LT to GE) // once open/closed intervals are supported. return Ok(vec![]); } + println!("interval:{:?}", interval); // Propagate the comparison operator. propagate_comparison(&self.op, left_interval, right_interval)? } else { diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index 9390089063a0..ced885a43848 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -126,6 +126,9 @@ impl PhysicalExpr for CastExpr { children: &[&Interval], ) -> Result>> { let child_interval = children[0]; + println!("child_interval:{:?}", child_interval); + println!("self:{:?}", self); + println!("interval:{:?}", interval); // Get child's datatype: let cast_type = child_interval.get_datatype()?; Ok(vec![Some( diff --git a/datafusion/physical-expr/src/intervals/cp_solver.rs b/datafusion/physical-expr/src/intervals/cp_solver.rs index 0a090636dc4b..8da4eb6ed8a2 100644 --- a/datafusion/physical-expr/src/intervals/cp_solver.rs +++ b/datafusion/physical-expr/src/intervals/cp_solver.rs @@ -507,8 +507,10 @@ impl ExprIntervalGraph { if !children_intervals.is_empty() { // Reverse to align with [PhysicalExpr]'s children: children_intervals.reverse(); + println!("self.graph[node].expr: {:?}", self.graph[node].expr); self.graph[node].interval = self.graph[node].expr.evaluate_bounds(&children_intervals)?; + println!("completed"); } } Ok(&self.graph[self.root].interval) @@ -533,9 +535,13 @@ impl ExprIntervalGraph { .map(|child| self.graph[*child].interval()) .collect::>(); let node_interval = self.graph[node].interval(); + println!("self.graph[node].expr:{:?}", self.graph[node].expr); + println!("children_intervals: {:?}", children_intervals); + println!("node_interval: {:?}", node_interval); let propagated_intervals = self.graph[node] .expr .propagate_constraints(node_interval, &children_intervals)?; + println!("after expr propagate"); for (child, interval) in children.into_iter().zip(propagated_intervals) { if let Some(interval) = interval { self.graph[child].interval = interval; @@ -555,13 +561,18 @@ impl ExprIntervalGraph { leaf_bounds: &mut [(usize, Interval)], ) -> Result { self.assign_intervals(leaf_bounds); + println!("self: {:?}", self); let bounds = self.evaluate_bounds()?; + println!("bounds: {:?}", bounds); + println!("after evaluate"); if bounds == &Interval::CERTAINLY_FALSE { Ok(PropagationResult::Infeasible) } else if bounds == &Interval::UNCERTAIN { - let result = self.propagate_constraints(); + println!("before propagate"); + let result = self.propagate_constraints()?; + println!("after propagate"); self.update_intervals(leaf_bounds); - result + Ok(result) } else { Ok(PropagationResult::CannotPropagate) } From 96b97c42d41870b4f77ed2e745247ebe8f60ea25 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 15 Sep 2023 18:17:08 +0300 Subject: [PATCH 07/53] datatype check added, statistics default removed --- datafusion-examples/examples/csv_opener.rs | 3 +- .../examples/custom_datasource.rs | 2 +- datafusion-examples/examples/json_opener.rs | 4 +- datafusion/common/src/stats.rs | 2 +- .../core/src/datasource/file_format/arrow.rs | 4 +- .../core/src/datasource/file_format/avro.rs | 4 +- .../core/src/datasource/file_format/csv.rs | 4 +- .../core/src/datasource/file_format/json.rs | 4 +- .../core/src/datasource/listing/table.rs | 12 ++- .../core/src/datasource/physical_plan/avro.rs | 12 +-- .../physical_plan/file_scan_config.rs | 17 ++-- .../datasource/physical_plan/file_stream.rs | 5 +- .../core/src/datasource/physical_plan/json.rs | 16 ++-- .../core/src/datasource/physical_plan/mod.rs | 32 ++++++-- .../src/datasource/physical_plan/parquet.rs | 16 ++-- .../combine_partial_final_agg.rs | 2 +- .../enforce_distribution.rs | 10 +-- .../src/physical_optimizer/join_selection.rs | 79 +++++++++++++++---- .../physical_optimizer/pipeline_checker.rs | 2 +- .../replace_with_order_preserving_variants.rs | 2 +- .../core/src/physical_optimizer/test_utils.rs | 4 +- .../core/src/physical_plan/aggregates/mod.rs | 16 +++- datafusion/core/src/physical_plan/analyze.rs | 2 +- datafusion/core/src/physical_plan/common.rs | 6 +- datafusion/core/src/physical_plan/explain.rs | 2 +- datafusion/core/src/physical_plan/filter.rs | 34 +++----- datafusion/core/src/physical_plan/insert.rs | 2 +- .../src/physical_plan/joins/cross_join.rs | 73 +++++++++++++---- .../joins/symmetric_hash_join.rs | 2 +- .../core/src/physical_plan/joins/utils.rs | 2 +- datafusion/core/src/physical_plan/limit.rs | 29 +++++-- .../core/src/physical_plan/projection.rs | 7 +- .../core/src/physical_plan/streaming.rs | 2 +- datafusion/core/src/physical_plan/union.rs | 4 +- datafusion/core/src/physical_plan/unnest.rs | 2 +- .../windows/bounded_window_agg_exec.rs | 12 ++- .../physical_plan/windows/window_agg_exec.rs | 12 ++- datafusion/core/src/physical_planner.rs | 8 +- datafusion/core/src/test/exec.rs | 2 +- datafusion/core/src/test/mod.rs | 6 +- datafusion/core/src/test_util/mod.rs | 2 +- datafusion/core/src/test_util/parquet.rs | 3 +- .../provider_filter_pushdown.rs | 2 +- .../tests/custom_sources_cases/statistics.rs | 10 ++- .../core/tests/parquet/custom_reader.rs | 4 +- datafusion/core/tests/parquet/page_pruning.rs | 4 +- .../core/tests/parquet/schema_coercion.rs | 8 +- datafusion/core/tests/path_partition.rs | 28 ++++++- .../tests/user_defined/user_defined_plan.rs | 2 +- datafusion/physical-expr/src/analysis.rs | 8 +- .../physical-expr/src/expressions/binary.rs | 3 +- .../physical-expr/src/expressions/cast.rs | 3 - .../physical-expr/src/intervals/cp_solver.rs | 16 +--- .../physical-expr/src/intervals/utils.rs | 28 +++++-- .../substrait/src/physical_plan/consumer.rs | 6 +- .../tests/cases/roundtrip_physical_plan.rs | 4 +- 56 files changed, 382 insertions(+), 208 deletions(-) diff --git a/datafusion-examples/examples/csv_opener.rs b/datafusion-examples/examples/csv_opener.rs index 0587b515b2d8..bbd4fcb405a8 100644 --- a/datafusion-examples/examples/csv_opener.rs +++ b/datafusion-examples/examples/csv_opener.rs @@ -17,6 +17,7 @@ use std::{sync::Arc, vec}; +use datafusion::common::Statistics; use datafusion::{ assert_batches_eq, datasource::{ @@ -60,7 +61,7 @@ async fn main() -> Result<()> { object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new(path.display().to_string(), 10)]], - statistics: Default::default(), + statistics: Statistics::new_with_unbounded_columns(schema.clone()), projection: Some(vec![12, 0]), limit: Some(5), table_partition_cols: vec![], diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index a24573c860bb..7e2d1725a403 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -271,6 +271,6 @@ impl ExecutionPlan for CustomExec { } fn statistics(&self) -> Statistics { - Statistics::default() + Statistics::new_with_unbounded_columns(self.schema()) } } diff --git a/datafusion-examples/examples/json_opener.rs b/datafusion-examples/examples/json_opener.rs index 71bb0ae4fe48..6d54858a95ef 100644 --- a/datafusion-examples/examples/json_opener.rs +++ b/datafusion-examples/examples/json_opener.rs @@ -28,7 +28,7 @@ use datafusion::{ error::Result, physical_plan::metrics::ExecutionPlanMetricsSet, }; -use datafusion_common::FileCompressionType; +use datafusion_common::{FileCompressionType, Statistics}; use futures::StreamExt; use object_store::ObjectStore; @@ -63,7 +63,7 @@ async fn main() -> Result<()> { object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new(path.to_string(), 10)]], - statistics: Default::default(), + statistics: Statistics::new_with_unbounded_columns(schema.clone()), projection: Some(vec![1, 0]), limit: Some(5), table_partition_cols: vec![], diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index fbc506c7e041..6a70062585f1 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -26,7 +26,7 @@ use crate::ScalarValue; /// Fields are optional and can be inexact because the sources /// sometimes provide approximate estimates for performance reasons /// and the transformations output are not always predictable. -#[derive(Debug, Clone, Default, PartialEq, Eq)] +#[derive(Debug, Clone, PartialEq, Eq)] pub struct Statistics { /// The number of table rows pub num_rows: Option, diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 45ecdd6083e7..32c64b772f29 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -74,10 +74,10 @@ impl FileFormat for ArrowFormat { &self, _state: &SessionState, _store: &Arc, - _table_schema: SchemaRef, + table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::default()) + Ok(Statistics::new_with_unbounded_columns(table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index e68a4cad2207..6d2039c22e9f 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -74,10 +74,10 @@ impl FileFormat for AvroFormat { &self, _state: &SessionState, _store: &Arc, - _table_schema: SchemaRef, + table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::default()) + Ok(Statistics::new_with_unbounded_columns(table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 4578ab5a4324..c681c6fa550b 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -235,10 +235,10 @@ impl FileFormat for CsvFormat { &self, _state: &SessionState, _store: &Arc, - _table_schema: SchemaRef, + table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::default()) + Ok(Statistics::new_with_unbounded_columns(table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 0aa87a9a3228..97bf65404029 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -152,10 +152,10 @@ impl FileFormat for JsonFormat { &self, _state: &SessionState, _store: &Arc, - _table_schema: SchemaRef, + table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::default()) + Ok(Statistics::new_with_unbounded_columns(table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index e36252a99566..8b410801115d 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -941,7 +941,10 @@ impl ListingTable { let store = if let Some(url) = self.table_paths.get(0) { ctx.runtime_env().object_store(url)? } else { - return Ok((vec![], Statistics::default())); + return Ok(( + vec![], + Statistics::new_with_unbounded_columns(self.file_schema.clone()), + )); }; // list files (with partitions) let file_list = future::try_join_all(self.table_paths.iter().map(|table_path| { @@ -980,7 +983,7 @@ impl ListingTable { } } } else { - Statistics::default() + Statistics::new_with_unbounded_columns(self.file_schema.clone()) }; Ok((part_file, statistics)) as Result<(PartitionedFile, Statistics)> }); @@ -1582,7 +1585,10 @@ mod tests { let cache = StatisticsCache::default(); assert!(cache.get(&meta).is_none()); - cache.save(meta.clone(), Statistics::default()); + cache.save( + meta.clone(), + Statistics::new_with_unbounded_columns(Arc::new(Schema::empty())), + ); assert!(cache.get(&meta).is_some()); // file size changed diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 93655e8665f0..cc2fe9effa3e 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -272,8 +272,8 @@ mod tests { let avro_exec = AvroExec::new(FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![vec![meta.into()]], - file_schema, - statistics: Statistics::default(), + file_schema: file_schema.clone(), + statistics: Statistics::new_with_unbounded_columns(file_schema), projection: Some(vec![0, 1, 2]), limit: None, table_partition_cols: vec![], @@ -344,8 +344,8 @@ mod tests { let avro_exec = AvroExec::new(FileScanConfig { object_store_url, file_groups: vec![vec![meta.into()]], - file_schema, - statistics: Statistics::default(), + file_schema: file_schema.clone(), + statistics: Statistics::new_with_unbounded_columns(file_schema), projection, limit: None, table_partition_cols: vec![], @@ -417,8 +417,8 @@ mod tests { projection: Some(vec![0, 1, file_schema.fields().len(), 2]), object_store_url, file_groups: vec![vec![partitioned_file]], - file_schema, - statistics: Statistics::default(), + file_schema: file_schema.clone(), + statistics: Statistics::new_with_unbounded_columns(file_schema), limit: None, table_partition_cols: vec![("date".to_owned(), DataType::Utf8)], output_ordering: vec![], 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 d4436e288971..74a5351cd6ca 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -135,17 +135,15 @@ impl FileScanConfig { if let Some(file_cols_stats) = &self.statistics.column_statistics { table_cols_stats.push(file_cols_stats[idx].clone()) } else { - table_cols_stats.push(ColumnStatistics::new_with_unbounded_column(field.data_type())) + table_cols_stats.push(ColumnStatistics::new_with_unbounded_column( + field.data_type(), + )) } } else { let partition_idx = idx - self.file_schema.fields().len(); let name = &self.table_partition_cols[partition_idx].0; let dtype = &self.table_partition_cols[partition_idx].1; - table_fields.push(Field::new( - name, - dtype.to_owned(), - false, - )); + table_fields.push(Field::new(name, dtype.to_owned(), false)); // TODO provide accurate stat for partition column (#1186) table_cols_stats.push(ColumnStatistics::new_with_unbounded_column(dtype)) } @@ -511,7 +509,7 @@ mod tests { let conf = config_for_projection( Arc::clone(&file_schema), None, - Statistics::default(), + Statistics::new_with_unbounded_columns(file_schema.clone()), vec![( "date".to_owned(), wrap_partition_type_in_dict(DataType::Utf8), @@ -559,7 +557,8 @@ mod tests { }) .collect(), ), - ..Default::default() + total_byte_size: None, + is_exact: false, }, vec![( "date".to_owned(), @@ -619,7 +618,7 @@ mod tests { file_batch.schema().fields().len(), file_batch.schema().fields().len() + 2, ]), - Statistics::default(), + Statistics::new_with_unbounded_columns(file_batch.schema()), partition_cols.clone(), ); let (proj_schema, ..) = conf.project(); diff --git a/datafusion/core/src/datasource/physical_plan/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index 71b322de44f7..709444902c65 100644 --- a/datafusion/core/src/datasource/physical_plan/file_stream.rs +++ b/datafusion/core/src/datasource/physical_plan/file_stream.rs @@ -521,6 +521,7 @@ mod tests { use arrow_schema::Schema; use datafusion_common::internal_err; use datafusion_common::DataFusionError; + use datafusion_common::Statistics; use super::*; use crate::datasource::file_format::write::BatchSerializer; @@ -659,9 +660,9 @@ mod tests { let config = FileScanConfig { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), - file_schema, + file_schema: file_schema.clone(), file_groups: vec![file_group], - statistics: Default::default(), + statistics: Statistics::new_with_unbounded_columns(file_schema), projection: None, limit: self.limit, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index d5c3a5635a2b..95ea80a87342 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -456,8 +456,8 @@ mod tests { FileScanConfig { object_store_url, file_groups, - file_schema, - statistics: Statistics::default(), + file_schema: file_schema.clone(), + statistics: Statistics::new_with_unbounded_columns(file_schema), projection: None, limit: Some(3), table_partition_cols: vec![], @@ -535,8 +535,8 @@ mod tests { FileScanConfig { object_store_url, file_groups, - file_schema, - statistics: Statistics::default(), + file_schema: file_schema.clone(), + statistics: Statistics::new_with_unbounded_columns(file_schema), projection: None, limit: Some(3), table_partition_cols: vec![], @@ -583,8 +583,8 @@ mod tests { FileScanConfig { object_store_url, file_groups, - file_schema, - statistics: Statistics::default(), + file_schema: file_schema.clone(), + statistics: Statistics::new_with_unbounded_columns(file_schema), projection: Some(vec![0, 2]), limit: None, table_partition_cols: vec![], @@ -636,8 +636,8 @@ mod tests { FileScanConfig { object_store_url, file_groups, - file_schema, - statistics: Statistics::default(), + file_schema: file_schema.clone(), + statistics: Statistics::new_with_unbounded_columns(file_schema), projection: Some(vec![3, 0, 2]), limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 4a9c2c2bc259..7d03ce09211f 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -799,7 +799,9 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: file_group, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(Arc::new( + Schema::empty(), + )), projection: None, limit: None, table_partition_cols: vec![], @@ -862,7 +864,9 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: fg.clone(), file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(Arc::new( + Schema::empty(), + )), projection: None, limit: None, table_partition_cols: vec![], @@ -896,7 +900,9 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: single_partition, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(Arc::new( + Schema::empty(), + )), projection: None, limit: None, table_partition_cols: vec![], @@ -933,7 +939,9 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: single_partition, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(Arc::new( + Schema::empty(), + )), projection: None, limit: None, table_partition_cols: vec![], @@ -976,7 +984,9 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: source_partitions, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(Arc::new( + Schema::empty(), + )), projection: None, limit: None, table_partition_cols: vec![], @@ -1015,7 +1025,9 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: source_partitions, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(Arc::new( + Schema::empty(), + )), projection: None, limit: None, table_partition_cols: vec![], @@ -1055,7 +1067,9 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: source_partitions, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(Arc::new( + Schema::empty(), + )), projection: None, limit: None, table_partition_cols: vec![], @@ -1084,7 +1098,9 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: single_partition, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(Arc::new( + Schema::empty(), + )), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index d16c79a9692c..7b8158444274 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -860,8 +860,8 @@ mod tests { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], - file_schema, - statistics: Statistics::default(), + file_schema: file_schema.clone(), + statistics: Statistics::new_with_unbounded_columns(file_schema), projection, limit: None, table_partition_cols: vec![], @@ -1517,8 +1517,8 @@ mod tests { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups, - file_schema, - statistics: Statistics::default(), + file_schema: file_schema.clone(), + statistics: Statistics::new_with_unbounded_columns(file_schema), projection: None, limit: None, table_partition_cols: vec![], @@ -1620,8 +1620,8 @@ mod tests { FileScanConfig { object_store_url, file_groups: vec![vec![partitioned_file]], - file_schema: schema, - statistics: Statistics::default(), + file_schema: schema.clone(), + statistics: Statistics::new_with_unbounded_columns(schema), // file has 10 cols so index 12 should be month and 13 should be day projection: Some(vec![0, 1, 2, 12, 13]), limit: None, @@ -1695,7 +1695,9 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![vec![partitioned_file]], file_schema: Arc::new(Schema::empty()), - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(Arc::new( + Schema::empty(), + )), projection: None, limit: None, table_partition_cols: vec![], 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 de47f3fbee73..74789a8be323 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -263,7 +263,7 @@ mod tests { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(schema.clone()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 3bfe9ae3df5d..88d0704f4273 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1816,7 +1816,7 @@ mod tests { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -1844,7 +1844,7 @@ mod tests { vec![PartitionedFile::new("x".to_string(), 100)], vec![PartitionedFile::new("y".to_string(), 100)], ], - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -1866,7 +1866,7 @@ mod tests { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -1897,7 +1897,7 @@ mod tests { vec![PartitionedFile::new("x".to_string(), 100)], vec![PartitionedFile::new("y".to_string(), 100)], ], - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -3876,7 +3876,7 @@ mod tests { "x".to_string(), 100, )]], - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(schema()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 628cc1da3b1e..043c1a35235e 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -599,7 +599,10 @@ mod tests_statistical { Statistics { num_rows: Some(10), total_byte_size: Some(100000), - ..Default::default() + column_statistics: Some(vec![ + ColumnStatistics::new_with_unbounded_column(&DataType::Int32), + ]), + is_exact: false, }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), )); @@ -608,7 +611,10 @@ mod tests_statistical { Statistics { num_rows: Some(100000), total_byte_size: Some(10), - ..Default::default() + column_statistics: Some(vec![ + ColumnStatistics::new_with_unbounded_column(&DataType::Int32), + ]), + is_exact: false, }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), )); @@ -650,7 +656,8 @@ mod tests_statistical { Some(50_000), Some(50_000), ), - ..Default::default() + total_byte_size: None, + is_exact: false, }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), )); @@ -663,7 +670,8 @@ mod tests_statistical { Some(5000), Some(1000), ), - ..Default::default() + total_byte_size: None, + is_exact: false, }, Schema::new(vec![Field::new("medium_col", DataType::Int32, false)]), )); @@ -676,7 +684,8 @@ mod tests_statistical { Some(100_000), Some(1000), ), - ..Default::default() + total_byte_size: None, + is_exact: false, }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), )); @@ -975,7 +984,10 @@ mod tests_statistical { Statistics { num_rows: Some(10000000), total_byte_size: Some(10000000), - ..Default::default() + column_statistics: Some(vec![ + ColumnStatistics::new_with_unbounded_column(&DataType::Int32), + ]), + is_exact: false, }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), )); @@ -984,7 +996,10 @@ mod tests_statistical { Statistics { num_rows: Some(10), total_byte_size: Some(10), - ..Default::default() + column_statistics: Some(vec![ + ColumnStatistics::new_with_unbounded_column(&DataType::Int32), + ]), + is_exact: false, }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), )); @@ -993,7 +1008,10 @@ mod tests_statistical { Statistics { num_rows: None, total_byte_size: None, - ..Default::default() + column_statistics: Some(vec![ + ColumnStatistics::new_with_unbounded_column(&DataType::Int32), + ]), + is_exact: false, }, Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), )); @@ -1053,7 +1071,10 @@ mod tests_statistical { Statistics { num_rows: Some(10000000), total_byte_size: Some(10000000), - ..Default::default() + column_statistics: Some(vec![ + ColumnStatistics::new_with_unbounded_column(&DataType::Int32), + ]), + is_exact: false, }, Schema::new(vec![Field::new("big_col1", DataType::Int32, false)]), )); @@ -1062,7 +1083,10 @@ mod tests_statistical { Statistics { num_rows: Some(20000000), total_byte_size: Some(20000000), - ..Default::default() + column_statistics: Some(vec![ + ColumnStatistics::new_with_unbounded_column(&DataType::Int32), + ]), + is_exact: false, }, Schema::new(vec![Field::new("big_col2", DataType::Int32, false)]), )); @@ -1071,7 +1095,10 @@ mod tests_statistical { Statistics { num_rows: None, total_byte_size: None, - ..Default::default() + column_statistics: Some(vec![ + ColumnStatistics::new_with_unbounded_column(&DataType::Int32), + ]), + is_exact: false, }, Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), )); @@ -1173,34 +1200,54 @@ mod tests_statistical { #[cfg(test)] mod util_tests { + use arrow_schema::{DataType, Field, Schema}; + use datafusion_common::ScalarValue; use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::{BinaryExpr, Column, NegativeExpr}; + use datafusion_physical_expr::expressions::{ + BinaryExpr, Column, Literal, NegativeExpr, + }; use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::PhysicalExpr; use std::sync::Arc; #[test] fn check_expr_supported() { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Utf8, false), + ])); let supported_expr = Arc::new(BinaryExpr::new( Arc::new(Column::new("a", 0)), Operator::Plus, Arc::new(Column::new("a", 0)), )) as Arc; - assert!(check_support(&supported_expr)); + assert!(check_support(&supported_expr, schema.clone())); let supported_expr_2 = Arc::new(Column::new("a", 0)) as Arc; - assert!(check_support(&supported_expr_2)); + assert!(check_support(&supported_expr_2, schema.clone())); let unsupported_expr = Arc::new(BinaryExpr::new( Arc::new(Column::new("a", 0)), Operator::Or, Arc::new(Column::new("a", 0)), )) as Arc; - assert!(!check_support(&unsupported_expr)); + assert!(!check_support(&unsupported_expr, schema.clone())); let unsupported_expr_2 = Arc::new(BinaryExpr::new( Arc::new(Column::new("a", 0)), Operator::Or, Arc::new(NegativeExpr::new(Arc::new(Column::new("a", 0)))), )) as Arc; - assert!(!check_support(&unsupported_expr_2)); + assert!(!check_support(&unsupported_expr_2, schema.clone())); + let unsupported_expr_3 = Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Eq, + Arc::new(Literal::new(ScalarValue::Int16(Some(-1)))), + )) as Arc; + assert!(!check_support(&unsupported_expr_3, schema.clone())); + let unsupported_expr_4 = Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Eq, + Arc::new(Literal::new(ScalarValue::Int16(Some(-1)))), + )) as Arc; + assert!(!check_support(&unsupported_expr_4, schema.clone())); } } diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index 44679647b5b2..e79f1b3b61a6 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -163,7 +163,7 @@ pub fn check_finiteness_requirements( /// [`Operator`]: datafusion_expr::Operator fn is_prunable(join: &SymmetricHashJoinExec) -> bool { join.filter().map_or(false, |filter| { - check_support(filter.expression()) + check_support(filter.expression(), join.schema()) && filter .schema() .fields() diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index b406a541054d..593061210b95 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -897,7 +897,7 @@ mod tests { "file_path".to_string(), 100, )]], - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(schema.clone()), projection: Some(projection), limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index e021cda2c868..10b55d5f798d 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -269,7 +269,7 @@ pub fn parquet_exec(schema: &SchemaRef) -> Arc { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(schema.clone()), projection: None, limit: None, table_partition_cols: vec![], @@ -293,7 +293,7 @@ pub fn parquet_exec_sorted( object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(schema.clone()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index c31c6badd32e..198d40f8ae6b 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -30,7 +30,9 @@ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::utils::longest_consecutive_prefix; -use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; +use datafusion_common::{ + not_impl_err, plan_err, ColumnStatistics, DataFusionError, Result, +}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ @@ -992,6 +994,12 @@ impl ExecutionPlan for AggregateExec { // - case where we group by on a column for which with have the `distinct` stat // TODO stats: aggr expression: // - aggregations somtimes also preserve invariants such as min, max... + let column_statistics = self + .schema() + .fields() + .iter() + .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) + .collect::>(); match self.mode { AggregateMode::Final | AggregateMode::FinalPartitioned if self.group_by.expr.is_empty() => @@ -999,14 +1007,16 @@ impl ExecutionPlan for AggregateExec { Statistics { num_rows: Some(1), is_exact: true, - ..Default::default() + column_statistics: Some(column_statistics), + total_byte_size: None, } } _ => Statistics { // the output row count is surely not larger than its input row count num_rows: self.input.statistics().num_rows, is_exact: false, - ..Default::default() + column_statistics: Some(column_statistics), + total_byte_size: None, }, } } diff --git a/datafusion/core/src/physical_plan/analyze.rs b/datafusion/core/src/physical_plan/analyze.rs index 98fce19a1dd7..5c38766613fb 100644 --- a/datafusion/core/src/physical_plan/analyze.rs +++ b/datafusion/core/src/physical_plan/analyze.rs @@ -183,7 +183,7 @@ impl ExecutionPlan for AnalyzeExec { fn statistics(&self) -> Statistics { // Statistics an an ANALYZE plan are not relevant - Statistics::default() + Statistics::new_with_unbounded_columns(self.schema()) } } diff --git a/datafusion/core/src/physical_plan/common.rs b/datafusion/core/src/physical_plan/common.rs index 787f3eed2673..15051357b1f6 100644 --- a/datafusion/core/src/physical_plan/common.rs +++ b/datafusion/core/src/physical_plan/common.rs @@ -146,7 +146,11 @@ pub fn compute_record_batch_statistics( None => (0..schema.fields().len()).collect(), }; - let mut column_statistics = vec![ColumnStatistics::default(); projection.len()]; + let mut column_statistics = schema + .fields() + .iter() + .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) + .collect::>(); for partition in batches.iter() { for batch in partition { diff --git a/datafusion/core/src/physical_plan/explain.rs b/datafusion/core/src/physical_plan/explain.rs index 7ee6f268d815..60ae9feb3edf 100644 --- a/datafusion/core/src/physical_plan/explain.rs +++ b/datafusion/core/src/physical_plan/explain.rs @@ -171,7 +171,7 @@ impl ExecutionPlan for ExplainExec { fn statistics(&self) -> Statistics { // Statistics an EXPLAIN plan are not relevant - Statistics::default() + Statistics::new_with_unbounded_columns(self.schema()) } } diff --git a/datafusion/core/src/physical_plan/filter.rs b/datafusion/core/src/physical_plan/filter.rs index 54ddbf6802fb..8c40c1c4f9cd 100644 --- a/datafusion/core/src/physical_plan/filter.rs +++ b/datafusion/core/src/physical_plan/filter.rs @@ -189,9 +189,8 @@ impl ExecutionPlan for FilterExec { /// predicate's selectivity value can be determined for the incoming data. fn statistics(&self) -> Statistics { let predicate = self.predicate(); - print_plan(&self.input); - println!("self.schema.fields:{:?}", self.schema().fields); - if check_support(predicate) { + + if check_support(predicate, self.schema()) { let input_stats = self.input.statistics(); if let Some(column_stats) = input_stats.column_statistics { @@ -263,12 +262,6 @@ fn collect_new_statistics( .collect() } -fn print_plan(plan: &Arc) -> () { - let formatted = crate::physical_plan::displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); -} - /// The FilterExec streams wraps the input iterator and applies the predicate expression to /// determine which rows to include in its output batches struct FilterExecStream { @@ -506,7 +499,7 @@ mod tests { max_value: Some(ScalarValue::Int32(Some(100))), ..Default::default() }]), - ..Default::default() + is_exact: false, }, schema.clone(), )); @@ -547,7 +540,8 @@ mod tests { max_value: Some(ScalarValue::Int32(Some(100))), ..Default::default() }]), - ..Default::default() + is_exact: false, + total_byte_size: None, }, schema.clone(), )); @@ -604,7 +598,8 @@ mod tests { ..Default::default() }, ]), - ..Default::default() + is_exact: false, + total_byte_size: None, }, schema.clone(), )); @@ -659,12 +654,7 @@ mod tests { // a: min=???, max=??? (missing) let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let input = Arc::new(StatisticsExec::new( - Statistics { - column_statistics: Some(vec![ColumnStatistics { - ..Default::default() - }]), - ..Default::default() - }, + Statistics::new_with_unbounded_columns(Arc::new(schema.clone())), schema.clone(), )); @@ -714,7 +704,7 @@ mod tests { ..Default::default() }, ]), - ..Default::default() + is_exact: false, }, schema, )); @@ -824,7 +814,7 @@ mod tests { ..Default::default() }, ]), - ..Default::default() + is_exact: false, }, schema, )); @@ -880,7 +870,7 @@ mod tests { ..Default::default() }, ]), - ..Default::default() + is_exact: false, }, schema, )); @@ -945,7 +935,7 @@ mod tests { ..Default::default() }, ]), - ..Default::default() + is_exact: false, }, schema, )); diff --git a/datafusion/core/src/physical_plan/insert.rs b/datafusion/core/src/physical_plan/insert.rs index 8c03fb543f5b..867ddb792f4d 100644 --- a/datafusion/core/src/physical_plan/insert.rs +++ b/datafusion/core/src/physical_plan/insert.rs @@ -251,7 +251,7 @@ impl ExecutionPlan for FileSinkExec { } fn statistics(&self) -> Statistics { - Statistics::default() + Statistics::new_with_unbounded_columns(self.schema()) } } diff --git a/datafusion/core/src/physical_plan/joins/cross_join.rs b/datafusion/core/src/physical_plan/joins/cross_join.rs index 256942754350..9f7f20a4b185 100644 --- a/datafusion/core/src/physical_plan/joins/cross_join.rs +++ b/datafusion/core/src/physical_plan/joins/cross_join.rs @@ -260,9 +260,9 @@ impl ExecutionPlan for CrossJoinExec { fn statistics(&self) -> Statistics { stats_cartesian_product( self.left.statistics(), - self.left.schema().fields().len(), + self.left.schema(), self.right.statistics(), - self.right.schema().fields().len(), + self.right.schema(), ) } } @@ -270,9 +270,9 @@ impl ExecutionPlan for CrossJoinExec { /// [left/right]_col_count are required in case the column statistics are None fn stats_cartesian_product( left_stats: Statistics, - left_col_count: usize, + left_schema: SchemaRef, right_stats: Statistics, - right_col_count: usize, + right_schema: SchemaRef, ) -> Statistics { let left_row_count = left_stats.num_rows; let right_row_count = right_stats.num_rows; @@ -289,19 +289,37 @@ fn stats_cartesian_product( .zip(right_stats.total_byte_size) .map(|(a, b)| 2 * a * b); + let left_col_count = left_schema.fields().len(); + let right_col_count = right_schema.fields().len(); // calculate column stats let column_statistics = // complete the column statistics if they are missing only on one side match (left_stats.column_statistics, right_stats.column_statistics) { - (None, None) => None, - (None, Some(right_col_stat)) => Some(( - vec![ColumnStatistics::default(); left_col_count], - right_col_stat, - )), - (Some(left_col_stat), None) => Some(( - left_col_stat, - vec![ColumnStatistics::default(); right_col_count], - )), + (None, None) => { + let mut left_stats= vec![]; + let mut right_stats= vec![]; + for index in 0..left_col_count{ + left_stats.push(ColumnStatistics::new_with_unbounded_column(left_schema.field(index).data_type())); + } + for index in 0..right_col_count{ + right_stats.push(ColumnStatistics::new_with_unbounded_column(right_schema.field(index).data_type())); + } + Some((left_stats,right_stats)) + }, + (None, Some(right_col_stat)) => { + let mut left_stats= vec![]; + for index in 0..left_col_count{ + left_stats.push(ColumnStatistics::new_with_unbounded_column(left_schema.field(index).data_type())); + } + Some((left_stats, right_col_stat)) + }, + (Some(left_col_stat), None) => { + let mut right_stats= vec![]; + for index in 0..right_col_count{ + right_stats.push(ColumnStatistics::new_with_unbounded_column(right_schema.field(index).data_type())); + } + Some((left_col_stat, right_stats)) + }, (Some(left_col_stat), Some(right_col_stat)) => { Some((left_col_stat, right_col_stat)) } @@ -461,6 +479,7 @@ mod tests { use crate::common::assert_contains; use crate::physical_plan::common; use crate::test::{build_table_scan_i32, columns}; + use arrow_schema::{DataType, Field}; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; async fn join_collect( @@ -516,7 +535,19 @@ mod tests { }]), }; - let result = stats_cartesian_product(left, 3, right, 2); + let result = stats_cartesian_product( + left, + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + Field::new("c", DataType::Int32, false), + ])), + right, + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + ])), + ); let expected = Statistics { is_exact: true, @@ -583,7 +614,19 @@ mod tests { }]), }; - let result = stats_cartesian_product(left, 3, right, 2); + let result = stats_cartesian_product( + left, + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + Field::new("c", DataType::Int32, false), + ])), + right, + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + ])), + ); let expected = Statistics { is_exact: true, diff --git a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs index 1c664adfbb71..28c3609e56b3 100644 --- a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs +++ b/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs @@ -481,7 +481,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { fn statistics(&self) -> Statistics { // TODO stats: it is not possible in general to know the output size of joins - Statistics::default() + Statistics::new_with_unbounded_columns(self.schema()) } fn execute( diff --git a/datafusion/core/src/physical_plan/joins/utils.rs b/datafusion/core/src/physical_plan/joins/utils.rs index bd3de1acbf0f..5d73a5187ec3 100644 --- a/datafusion/core/src/physical_plan/joins/utils.rs +++ b/datafusion/core/src/physical_plan/joins/utils.rs @@ -1545,7 +1545,7 @@ mod tests { num_rows, column_statistics: column_stats, is_exact, - ..Default::default() + total_byte_size: None, } } diff --git a/datafusion/core/src/physical_plan/limit.rs b/datafusion/core/src/physical_plan/limit.rs index 87a07f8d46fe..f72229154250 100644 --- a/datafusion/core/src/physical_plan/limit.rs +++ b/datafusion/core/src/physical_plan/limit.rs @@ -33,7 +33,7 @@ use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics} use arrow::array::ArrayRef; use arrow::datatypes::SchemaRef; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; -use datafusion_common::{internal_err, DataFusionError, Result}; +use datafusion_common::{internal_err, ColumnStatistics, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::OrderingEquivalenceProperties; @@ -201,6 +201,12 @@ impl ExecutionPlan for GlobalLimitExec { } }) .unwrap_or(usize::MAX); + let col_stats = self + .schema() + .fields() + .iter() + .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) + .collect::>(); match input_stats { Statistics { num_rows: Some(nr), .. @@ -210,7 +216,8 @@ impl ExecutionPlan for GlobalLimitExec { Statistics { num_rows: Some(0), is_exact: input_stats.is_exact, - ..Default::default() + column_statistics: Some(col_stats), + total_byte_size: None, } } else if nr <= max_row_num { // if the input does not reach the "fetch" globally, return input stats @@ -221,7 +228,8 @@ impl ExecutionPlan for GlobalLimitExec { Statistics { num_rows: Some(max_row_num), is_exact: input_stats.is_exact, - ..Default::default() + column_statistics: Some(col_stats), + total_byte_size: None, } } } @@ -229,7 +237,8 @@ impl ExecutionPlan for GlobalLimitExec { // the result output row number will always be no greater than the limit number num_rows: Some(max_row_num), is_exact: false, - ..Default::default() + column_statistics: Some(col_stats), + total_byte_size: None, }, } } @@ -355,6 +364,12 @@ impl ExecutionPlan for LocalLimitExec { fn statistics(&self) -> Statistics { let input_stats = self.input.statistics(); + let col_stats = self + .schema() + .fields() + .iter() + .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) + .collect::>(); match input_stats { // if the input does not reach the limit globally, return input stats Statistics { @@ -370,13 +385,15 @@ impl ExecutionPlan for LocalLimitExec { // this is not actually exact, but will be when GlobalLimit is applied // TODO stats: find a more explicit way to vehiculate this information is_exact: input_stats.is_exact, - ..Default::default() + column_statistics: Some(col_stats), + total_byte_size: None, }, _ => Statistics { // the result output row number will always be no greater than the limit number num_rows: Some(self.fetch * self.output_partitioning().partition_count()), is_exact: false, - ..Default::default() + column_statistics: Some(col_stats), + total_byte_size: None, }, } } diff --git a/datafusion/core/src/physical_plan/projection.rs b/datafusion/core/src/physical_plan/projection.rs index 12c89eee1931..ee3ac964e614 100644 --- a/datafusion/core/src/physical_plan/projection.rs +++ b/datafusion/core/src/physical_plan/projection.rs @@ -403,13 +403,16 @@ fn stats_projection( inner_exprs .clone() .into_iter() - .map(|e| { + .enumerate() + .map(|(index, e)| { if let Some(col) = e.as_any().downcast_ref::() { input_col_stats[col.index()].clone() } else { // TODO stats: estimate more statistics from expressions // (expressions should compute their statistics themselves) - ColumnStatistics::default() + ColumnStatistics::new_with_unbounded_column( + schema.field(index).data_type(), + ) } }) .collect() diff --git a/datafusion/core/src/physical_plan/streaming.rs b/datafusion/core/src/physical_plan/streaming.rs index 6c33f88a3991..b14b8214a2e1 100644 --- a/datafusion/core/src/physical_plan/streaming.rs +++ b/datafusion/core/src/physical_plan/streaming.rs @@ -186,6 +186,6 @@ impl ExecutionPlan for StreamingTableExec { } fn statistics(&self) -> Statistics { - Default::default() + Statistics::new_with_unbounded_columns(self.schema()) } } diff --git a/datafusion/core/src/physical_plan/union.rs b/datafusion/core/src/physical_plan/union.rs index 491d24c2897b..ac31004186d5 100644 --- a/datafusion/core/src/physical_plan/union.rs +++ b/datafusion/core/src/physical_plan/union.rs @@ -269,7 +269,7 @@ impl ExecutionPlan for UnionExec { .iter() .map(|ep| ep.statistics()) .reduce(stats_union) - .unwrap_or_default() + .unwrap_or(Statistics::new_with_unbounded_columns(self.schema())) } fn benefits_from_input_partitioning(&self) -> Vec { @@ -443,7 +443,7 @@ impl ExecutionPlan for InterleaveExec { .iter() .map(|ep| ep.statistics()) .reduce(stats_union) - .unwrap_or_default() + .unwrap_or(Statistics::new_with_unbounded_columns(self.schema())) } fn benefits_from_input_partitioning(&self) -> Vec { diff --git a/datafusion/core/src/physical_plan/unnest.rs b/datafusion/core/src/physical_plan/unnest.rs index 40c4edc95330..943557c20310 100644 --- a/datafusion/core/src/physical_plan/unnest.rs +++ b/datafusion/core/src/physical_plan/unnest.rs @@ -161,7 +161,7 @@ impl ExecutionPlan for UnnestExec { } fn statistics(&self) -> Statistics { - Default::default() + Statistics::new_with_unbounded_columns(self.schema()) } } diff --git a/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs b/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs index 92e66783dc57..55813d9776ee 100644 --- a/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs +++ b/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs @@ -341,9 +341,17 @@ impl ExecutionPlan for BoundedWindowAggExec { if let Some(input_col_stats) = input_stat.column_statistics { column_statistics.extend(input_col_stats); } else { - column_statistics.extend(vec![ColumnStatistics::default(); input_cols]); + for index in 0..input_cols { + column_statistics.push(ColumnStatistics::new_with_unbounded_column( + self.schema().field(index).data_type(), + )) + } + } + for index in 0..win_cols { + column_statistics.push(ColumnStatistics::new_with_unbounded_column( + self.schema().field(index + input_cols).data_type(), + )) } - column_statistics.extend(vec![ColumnStatistics::default(); win_cols]); Statistics { is_exact: input_stat.is_exact, num_rows: input_stat.num_rows, diff --git a/datafusion/core/src/physical_plan/windows/window_agg_exec.rs b/datafusion/core/src/physical_plan/windows/window_agg_exec.rs index 68d1f33fcde4..8098cb2ebcc9 100644 --- a/datafusion/core/src/physical_plan/windows/window_agg_exec.rs +++ b/datafusion/core/src/physical_plan/windows/window_agg_exec.rs @@ -265,9 +265,17 @@ impl ExecutionPlan for WindowAggExec { if let Some(input_col_stats) = input_stat.column_statistics { column_statistics.extend(input_col_stats); } else { - column_statistics.extend(vec![ColumnStatistics::default(); input_cols]); + for index in 0..input_cols { + column_statistics.push(ColumnStatistics::new_with_unbounded_column( + self.schema().field(index).data_type(), + )) + } + } + for index in 0..win_cols { + column_statistics.push(ColumnStatistics::new_with_unbounded_column( + self.schema().field(index + input_cols).data_type(), + )) } - column_statistics.extend(vec![ColumnStatistics::default(); win_cols]); Statistics { is_exact: input_stat.is_exact, num_rows: input_stat.num_rows, diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 4a9c7a49b9ba..def4d59873df 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2157,12 +2157,6 @@ mod tests { Ok(()) } - fn print_plan(plan: &Arc) -> () { - let formatted = crate::physical_plan::displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); - } - #[tokio::test] async fn test_with_csv_plan() -> Result<()> { let logical_plan = test_csv_scan() @@ -2172,7 +2166,7 @@ mod tests { .build()?; let plan = plan(&logical_plan).await?; - print_plan(&plan); + // c12 is f64, c7 is u8 -> cast c7 to f64 // the cast here is implicit so has CastOptions with safe=true let _expected = "predicate: BinaryExpr { left: TryCastExpr { expr: Column { name: \"c7\", index: 6 }, cast_type: Float64 }, op: Lt, right: Column { name: \"c12\", index: 11 } }"; diff --git a/datafusion/core/src/test/exec.rs b/datafusion/core/src/test/exec.rs index 44ce5cf3282b..88322146ef8e 100644 --- a/datafusion/core/src/test/exec.rs +++ b/datafusion/core/src/test/exec.rs @@ -454,7 +454,7 @@ impl ExecutionPlan for ErrorExec { } fn statistics(&self) -> Statistics { - Statistics::default() + Statistics::new_with_unbounded_columns(self.schema()) } } diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 011d2e64281a..d51de4bb5569 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -193,9 +193,9 @@ pub fn partitioned_csv_config( ) -> Result { Ok(FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), - file_schema: schema, + file_schema: schema.clone(), file_groups, - statistics: Default::default(), + statistics: Statistics::new_with_unbounded_columns(schema), projection: None, limit: None, table_partition_cols: vec![], @@ -344,7 +344,7 @@ pub fn csv_exec_sorted( object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::default(), + statistics: Statistics::new_with_unbounded_columns(schema.clone()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index ab29cecbb8ea..e21835cc9321 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -315,7 +315,7 @@ impl ExecutionPlan for UnboundedExec { } fn statistics(&self) -> Statistics { - Statistics::default() + Statistics::new_with_unbounded_columns(self.schema()) } } diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index d3a1f9c1ef7c..fdd89569b137 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -35,6 +35,7 @@ use crate::physical_plan::filter::FilterExec; use crate::physical_plan::metrics::MetricsSet; use crate::physical_plan::ExecutionPlan; use crate::prelude::{Expr, SessionConfig}; +use datafusion_common::Statistics; use object_store::path::Path; use object_store::ObjectMeta; use parquet::arrow::ArrowWriter; @@ -147,7 +148,7 @@ impl TestParquetFile { range: None, extensions: None, }]], - statistics: Default::default(), + statistics: Statistics::new_with_unbounded_columns(self.schema.clone()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index 79214092fa57..f6af3a3041e8 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -115,7 +115,7 @@ impl ExecutionPlan for CustomPlan { fn statistics(&self) -> Statistics { // here we could provide more accurate statistics // but we want to test the filter pushdown not the CBOs - Statistics::default() + Statistics::new_with_unbounded_columns(self.schema()) } } diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index 43e6c8851ec4..b446708f0e7f 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -243,17 +243,23 @@ async fn sql_filter() -> Result<()> { #[tokio::test] async fn sql_limit() -> Result<()> { let (stats, schema) = fully_defined(); - let ctx = init_ctx(stats.clone(), schema)?; + let ctx = init_ctx(stats.clone(), schema.clone())?; let df = ctx.sql("SELECT * FROM stats_table LIMIT 5").await.unwrap(); let physical_plan = df.create_physical_plan().await.unwrap(); // when the limit is smaller than the original number of lines // we loose all statistics except the for number of rows which becomes the limit + let col_stats = schema + .fields() + .iter() + .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) + .collect::>(); assert_eq!( Statistics { num_rows: Some(5), is_exact: true, - ..Default::default() + column_statistics: Some(col_stats), + total_byte_size: None }, physical_plan.statistics() ); diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index 75ff56a26508..7a61b6ceb1d7 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -77,8 +77,8 @@ async fn route_data_access_ops_to_parquet_file_reader_factory() { // just any url that doesn't point to in memory object store object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], - file_schema, - statistics: Statistics::default(), + file_schema: file_schema.clone(), + statistics: Statistics::new_with_unbounded_columns(file_schema), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 4337259c1e62..0bdad7ef4492 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -71,8 +71,8 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { FileScanConfig { object_store_url, file_groups: vec![vec![partitioned_file]], - file_schema: schema, - statistics: Statistics::default(), + file_schema: schema.clone(), + statistics: Statistics::new_with_unbounded_columns(schema), // file has 10 cols so index 12 should be month projection: None, limit: None, diff --git a/datafusion/core/tests/parquet/schema_coercion.rs b/datafusion/core/tests/parquet/schema_coercion.rs index f7dace993091..e3925d48e12f 100644 --- a/datafusion/core/tests/parquet/schema_coercion.rs +++ b/datafusion/core/tests/parquet/schema_coercion.rs @@ -62,8 +62,8 @@ async fn multi_parquet_coercion() { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], - file_schema, - statistics: Statistics::default(), + file_schema: file_schema.clone(), + statistics: Statistics::new_with_unbounded_columns(file_schema), projection: None, limit: None, table_partition_cols: vec![], @@ -126,8 +126,8 @@ async fn multi_parquet_coercion_projection() { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], - file_schema, - statistics: Statistics::default(), + file_schema: file_schema.clone(), + statistics: Statistics::new_with_unbounded_columns(file_schema), projection: Some(vec![1, 0, 2]), limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index 1c08733e3df6..a9ecc296b8be 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -468,9 +468,24 @@ async fn parquet_statistics() -> Result<()> { // stats for the first col are read from the parquet file assert_eq!(stat_cols[0].null_count, Some(3)); // TODO assert partition column (1,2,3) stats once implemented (#1186) - assert_eq!(stat_cols[1], ColumnStatistics::default()); - assert_eq!(stat_cols[2], ColumnStatistics::default()); - assert_eq!(stat_cols[3], ColumnStatistics::default()); + assert_eq!( + stat_cols[1], + ColumnStatistics::new_with_unbounded_column( + physical_plan.schema().field(1).data_type() + ), + ); + assert_eq!( + stat_cols[2], + ColumnStatistics::new_with_unbounded_column( + physical_plan.schema().field(2).data_type() + ), + ); + assert_eq!( + stat_cols[3], + ColumnStatistics::new_with_unbounded_column( + physical_plan.schema().field(3).data_type() + ), + ); //// WITH PROJECTION //// let dataframe = ctx.sql("SELECT mycol, day FROM t WHERE day='28'").await?; @@ -485,7 +500,12 @@ async fn parquet_statistics() -> Result<()> { // stats for the first col are read from the parquet file assert_eq!(stat_cols[0].null_count, Some(1)); // TODO assert partition column stats once implemented (#1186) - assert_eq!(stat_cols[1], ColumnStatistics::default()); + assert_eq!( + stat_cols[1], + ColumnStatistics::new_with_unbounded_column( + physical_plan.schema().field(1).data_type() + ), + ); Ok(()) } diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 21ec20f0d4d6..6fcc078489b5 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -493,7 +493,7 @@ impl ExecutionPlan for TopKExec { fn statistics(&self) -> Statistics { // to improve the optimizability of this plan // better statistics inference could be provided - Statistics::default() + Statistics::new_with_unbounded_columns(self.schema()) } } diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index 2d54f1b624d6..1349d5ab116b 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -125,9 +125,8 @@ pub fn analyze( DataFusionError::Internal("No column exists at the input to filter".to_string()) })?; - println!("start expr:{:?}", expr); let mut graph = ExprIntervalGraph::try_new(expr.clone())?; - println!("created graph"); + let columns: Vec> = collect_columns(expr) .into_iter() .map(|c| Arc::new(c) as Arc) @@ -148,10 +147,7 @@ pub fn analyze( }) }) .collect(); - - let res = graph.update_ranges(&mut target_indices_and_boundaries)?; - println!("update range done"); - match res { + match graph.update_ranges(&mut target_indices_and_boundaries)? { PropagationResult::Success => Ok(shrink_boundaries( expr, graph, diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index e3aee58dea20..f235ea9fcdaf 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -375,13 +375,12 @@ impl PhysicalExpr for BinaryExpr { // changes accordingly. return Ok(vec![]); } else if self.op.is_comparison_operator() { - if interval == &Interval::CERTAINLY_FALSE || self.op == Operator::Eq { + if interval == &Interval::CERTAINLY_FALSE { // TODO: We will handle strictly false clauses by negating // the comparison operator (e.g. GT to LE, LT to GE) // once open/closed intervals are supported. return Ok(vec![]); } - println!("interval:{:?}", interval); // Propagate the comparison operator. propagate_comparison(&self.op, left_interval, right_interval)? } else { diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index ced885a43848..9390089063a0 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -126,9 +126,6 @@ impl PhysicalExpr for CastExpr { children: &[&Interval], ) -> Result>> { let child_interval = children[0]; - println!("child_interval:{:?}", child_interval); - println!("self:{:?}", self); - println!("interval:{:?}", interval); // Get child's datatype: let cast_type = child_interval.get_datatype()?; Ok(vec![Some( diff --git a/datafusion/physical-expr/src/intervals/cp_solver.rs b/datafusion/physical-expr/src/intervals/cp_solver.rs index 8da4eb6ed8a2..7e6c71db4739 100644 --- a/datafusion/physical-expr/src/intervals/cp_solver.rs +++ b/datafusion/physical-expr/src/intervals/cp_solver.rs @@ -507,10 +507,8 @@ impl ExprIntervalGraph { if !children_intervals.is_empty() { // Reverse to align with [PhysicalExpr]'s children: children_intervals.reverse(); - println!("self.graph[node].expr: {:?}", self.graph[node].expr); self.graph[node].interval = self.graph[node].expr.evaluate_bounds(&children_intervals)?; - println!("completed"); } } Ok(&self.graph[self.root].interval) @@ -535,13 +533,10 @@ impl ExprIntervalGraph { .map(|child| self.graph[*child].interval()) .collect::>(); let node_interval = self.graph[node].interval(); - println!("self.graph[node].expr:{:?}", self.graph[node].expr); - println!("children_intervals: {:?}", children_intervals); - println!("node_interval: {:?}", node_interval); let propagated_intervals = self.graph[node] .expr .propagate_constraints(node_interval, &children_intervals)?; - println!("after expr propagate"); + for (child, interval) in children.into_iter().zip(propagated_intervals) { if let Some(interval) = interval { self.graph[child].interval = interval; @@ -561,18 +556,13 @@ impl ExprIntervalGraph { leaf_bounds: &mut [(usize, Interval)], ) -> Result { self.assign_intervals(leaf_bounds); - println!("self: {:?}", self); let bounds = self.evaluate_bounds()?; - println!("bounds: {:?}", bounds); - println!("after evaluate"); if bounds == &Interval::CERTAINLY_FALSE { Ok(PropagationResult::Infeasible) } else if bounds == &Interval::UNCERTAIN { - println!("before propagate"); - let result = self.propagate_constraints()?; - println!("after propagate"); + let result = self.propagate_constraints(); self.update_intervals(leaf_bounds); - Ok(result) + result } else { Ok(PropagationResult::CannotPropagate) } diff --git a/datafusion/physical-expr/src/intervals/utils.rs b/datafusion/physical-expr/src/intervals/utils.rs index be3b17771303..131cc8e6faaa 100644 --- a/datafusion/physical-expr/src/intervals/utils.rs +++ b/datafusion/physical-expr/src/intervals/utils.rs @@ -25,7 +25,7 @@ use crate::{ PhysicalExpr, }; -use arrow_schema::DataType; +use arrow_schema::{DataType, SchemaRef}; use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::Operator; @@ -38,15 +38,29 @@ const DT_MS_MASK: i64 = 0xFFFF_FFFF; /// We do not support every type of [`Operator`]s either. Over time, this check /// will relax as more types of `PhysicalExpr`s and `Operator`s are supported. /// Currently, [`CastExpr`], [`BinaryExpr`], [`Column`] and [`Literal`] are supported. -pub fn check_support(expr: &Arc) -> bool { +pub fn check_support(expr: &Arc, schema: SchemaRef) -> bool { let expr_any = expr.as_any(); - let expr_supported = if let Some(binary_expr) = expr_any.downcast_ref::() - { + if let Some(binary_expr) = expr_any.downcast_ref::() { is_operator_supported(binary_expr.op()) + && check_support(binary_expr.left(), schema.clone()) + && check_support(binary_expr.right(), schema) + } else if let Some(column) = expr_any.downcast_ref::() { + if let Ok(field) = schema.field_with_name(column.name()) { + is_datatype_supported(field.data_type()) + } else { + return false; + } + } else if let Some(literal) = expr_any.downcast_ref::() { + if let Ok(dt) = literal.data_type(&schema) { + is_datatype_supported(&dt) + } else { + return false; + } + } else if let Some(cast) = expr_any.downcast_ref::() { + check_support(&cast.children()[0], schema) } else { - expr_any.is::() || expr_any.is::() || expr_any.is::() - }; - expr_supported && expr.children().iter().all(check_support) + false + } } // This function returns the inverse operator of the given operator. diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index 8d25626a3bfe..f9ff671a5e06 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -23,7 +23,7 @@ use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion::error::{DataFusionError, Result}; -use datafusion::physical_plan::ExecutionPlan; +use datafusion::physical_plan::{ExecutionPlan, Statistics}; use datafusion::prelude::SessionContext; use object_store::ObjectMeta; use std::collections::HashMap; @@ -104,7 +104,9 @@ pub async fn from_substrait_rel( object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: Arc::new(Schema::empty()), file_groups, - statistics: Default::default(), + statistics: Statistics::new_with_unbounded_columns(Arc::new( + Schema::empty(), + )), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs index 25d60471a9cd..85a40585253b 100644 --- a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs @@ -20,7 +20,7 @@ use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion::error::Result; -use datafusion::physical_plan::{displayable, ExecutionPlan}; +use datafusion::physical_plan::{displayable, ExecutionPlan, Statistics}; use datafusion::prelude::SessionContext; use datafusion_substrait::physical_plan::{consumer, producer}; use std::collections::HashMap; @@ -42,7 +42,7 @@ async fn parquet_exec() -> Result<()> { 123, )], ], - statistics: Default::default(), + statistics: Statistics::new_with_unbounded_columns(Arc::new(Schema::empty())), projection: None, limit: None, table_partition_cols: vec![], From dd5e4c94dd2f0e2861c0a9d3a95b1b7d331312e5 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 18 Sep 2023 13:13:31 +0300 Subject: [PATCH 08/53] MemExec uses the stats of projections, agg optimize excludes unbounded cases --- .../aggregate_statistics.rs | 10 ++++++-- datafusion/core/src/physical_plan/common.rs | 16 ++++++------ .../sqllogictest/test_files/subquery.slt | 25 ++++++++++--------- 3 files changed, 30 insertions(+), 21 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 396e66972f30..e7db9610fd60 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -221,7 +221,10 @@ fn take_optimizable_min( .. } = &col_stats[col_expr.index()] { - return Some((val.clone(), casted_expr.name().to_string())); + // Exclude the unbounded case + if !val.is_null() { + return Some((val.clone(), casted_expr.name().to_string())); + } } } } @@ -249,7 +252,10 @@ fn take_optimizable_max( .. } = &col_stats[col_expr.index()] { - return Some((val.clone(), casted_expr.name().to_string())); + // Exclude the unbounded case + if !val.is_null() { + return Some((val.clone(), casted_expr.name().to_string())); + } } } } diff --git a/datafusion/core/src/physical_plan/common.rs b/datafusion/core/src/physical_plan/common.rs index 15051357b1f6..249122f7b32a 100644 --- a/datafusion/core/src/physical_plan/common.rs +++ b/datafusion/core/src/physical_plan/common.rs @@ -146,10 +146,10 @@ pub fn compute_record_batch_statistics( None => (0..schema.fields().len()).collect(), }; - let mut column_statistics = schema - .fields() + let fields = schema.fields(); + let mut column_statistics = projection .iter() - .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) + .map(|&idx| ColumnStatistics::new_with_unbounded_column(fields[idx].data_type())) .collect::>(); for partition in batches.iter() { @@ -383,11 +383,13 @@ mod tests { use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::union::UnionExec; use arrow::compute::SortOptions; + use arrow::ipc::Null; use arrow::{ array::{Float32Array, Float64Array}, datatypes::{DataType, Field, Schema}, record_batch::RecordBatch, }; + use datafusion_common::ScalarValue; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{col, Column}; @@ -704,14 +706,14 @@ mod tests { column_statistics: Some(vec![ ColumnStatistics { distinct_count: None, - max_value: None, - min_value: None, + max_value: Some(ScalarValue::Float32(None)), + min_value: Some(ScalarValue::Float32(None)), null_count: Some(0), }, ColumnStatistics { distinct_count: None, - max_value: None, - min_value: None, + max_value: Some(ScalarValue::Float64(None)), + min_value: Some(ScalarValue::Float64(None)), null_count: Some(0), }, ]), diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index fe074da1bba0..2eccb60aad3e 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -284,19 +284,20 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum ------------TableScan: t2 projection=[t2_id, t2_int] physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum] ---CoalesceBatchesExec: target_batch_size=8192 -----HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)] -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] +--ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int)@0 as SUM(t2.t2_int), t2_id@1 as t2_id] +----CoalesceBatchesExec: target_batch_size=8192 +------HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] +--------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] +----------CoalesceBatchesExec: target_batch_size=8192 +------------FilterExec: SUM(t2.t2_int)@1 < 3 +--------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +--------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +----------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] --------CoalesceBatchesExec: target_batch_size=8192 -----------FilterExec: SUM(t2.t2_int)@1 < 3 -------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] ---------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query II rowsort SELECT t1_id, (SELECT sum(t2_int) FROM t2 WHERE t2.t2_id = t1.t1_id having sum(t2_int) < 3) as t2_sum from t1 From c891ea93051844816e7761106f9eea54bbbb2238 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 18 Sep 2023 17:12:29 +0300 Subject: [PATCH 09/53] fix after merge --- datafusion/common/src/stats.rs | 2 -- datafusion/core/src/datasource/listing/table.rs | 7 +++---- datafusion/execution/src/cache/cache_unit.rs | 17 +++++++++++++++-- datafusion/physical-expr/src/analysis.rs | 5 +---- datafusion/physical-plan/src/common.rs | 1 - .../physical-plan/src/joins/cross_join.rs | 1 + 6 files changed, 20 insertions(+), 13 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 0b2d8af7c28c..e7ca8e4d7d6f 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -20,8 +20,6 @@ use arrow::datatypes::{DataType, SchemaRef}; use std::fmt::Display; -use arrow::datatypes::DataType; - use crate::ScalarValue; /// Statistics for a relation diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index f85ffa9766ba..8cc5129698bf 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -939,14 +939,13 @@ impl ListingTable { // collect the statistics if required by the config let files = file_list.then(|part_file| async { let part_file = part_file?; - let mut statistics_result = Statistics::default(); - if self.options.collect_stat { + let statistics_result = if self.options.collect_stat { let statistics_cache = self.collected_statistics.clone(); match statistics_cache.get_with_extra( &part_file.object_meta.location, &part_file.object_meta, ) { - Some(statistics) => statistics_result = statistics.as_ref().clone(), + Some(statistics) => statistics.as_ref().clone(), None => { let statistics = self .options @@ -963,7 +962,7 @@ impl ListingTable { statistics.clone().into(), &part_file.object_meta, ); - statistics_result = statistics; + statistics } } } else { diff --git a/datafusion/execution/src/cache/cache_unit.rs b/datafusion/execution/src/cache/cache_unit.rs index 3ef699ac2360..2c40c17f0df9 100644 --- a/datafusion/execution/src/cache/cache_unit.rs +++ b/datafusion/execution/src/cache/cache_unit.rs @@ -96,8 +96,11 @@ impl CacheAccessor> for DefaultFileStatisticsCache { #[cfg(test)] mod tests { + use std::sync::Arc; + use crate::cache::cache_unit::DefaultFileStatisticsCache; use crate::cache::CacheAccessor; + use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use chrono::DateTime; use datafusion_common::Statistics; use object_store::path::Path; @@ -113,11 +116,21 @@ mod tests { size: 1024, e_tag: None, }; - let cache = DefaultFileStatisticsCache::default(); assert!(cache.get_with_extra(&meta.location, &meta).is_none()); - cache.put_with_extra(&meta.location, Statistics::default().into(), &meta); + cache.put_with_extra( + &meta.location, + Statistics::new_with_unbounded_columns(Arc::new(Schema::new(vec![ + Field::new( + "test_column", + DataType::Timestamp(TimeUnit::Second, None), + false, + ), + ]))) + .into(), + &meta, + ); assert!(cache.get_with_extra(&meta.location, &meta).is_some()); // file size changed diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index f47cd9518921..1349d5ab116b 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -190,15 +190,12 @@ fn shrink_boundaries( let (_, root_index) = graph_nodes[0]; let final_result = graph.get_interval(root_index); - // If during selectivity calculation we encounter an error, use 1.0 as cardinality estimate - // safest estimate(e.q largest possible value). let selectivity = calculate_selectivity( &final_result.lower.value, &final_result.upper.value, &target_boundaries, &initial_boundaries, - ) - .unwrap_or(1.0); + ); AnalysisContext::new(target_boundaries).with_selectivity(selectivity) } diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index 9893e2b9cf0c..bf16fe2e8d39 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -383,7 +383,6 @@ mod tests { use crate::sorts::sort::SortExec; use crate::union::UnionExec; use arrow::compute::SortOptions; - use arrow::ipc::Null; use arrow::{ array::{Float32Array, Float64Array}, datatypes::{DataType, Field, Schema}, diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 671b0b128721..4171677325a6 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -477,6 +477,7 @@ mod tests { use super::*; use crate::common; use crate::test::build_table_scan_i32; + use arrow_schema::{DataType, Field}; use datafusion_common::{assert_batches_sorted_eq, assert_contains}; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; From 2450ef255784c086f5c1f997515cc7925e17ef8d Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 19 Sep 2023 11:19:55 +0300 Subject: [PATCH 10/53] proto fix --- datafusion/proto/proto/datafusion.proto | 4 ++-- datafusion/proto/src/generated/pbjson.rs | 14 ++++++++------ datafusion/proto/src/generated/prost.rs | 8 ++++---- datafusion/proto/src/physical_plan/from_proto.rs | 12 ++++++++++-- datafusion/proto/src/physical_plan/to_proto.rs | 4 ++-- 5 files changed, 26 insertions(+), 16 deletions(-) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index a0148c8af077..218abd944a43 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1513,8 +1513,8 @@ message Statistics { message ColumnStats { ScalarValue min_value = 1; ScalarValue max_value = 2; - uint32 null_count = 3; - uint32 distinct_count = 4; + repeated uint32 null_count = 3; + repeated uint32 distinct_count = 4; } message NamedStructFieldExpr { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 948516117428..47280c7008ad 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -3152,10 +3152,10 @@ impl serde::Serialize for ColumnStats { if self.max_value.is_some() { len += 1; } - if self.null_count != 0 { + if !self.null_count.is_empty() { len += 1; } - if self.distinct_count != 0 { + if !self.distinct_count.is_empty() { len += 1; } let mut struct_ser = serializer.serialize_struct("datafusion.ColumnStats", len)?; @@ -3165,10 +3165,10 @@ impl serde::Serialize for ColumnStats { if let Some(v) = self.max_value.as_ref() { struct_ser.serialize_field("maxValue", v)?; } - if self.null_count != 0 { + if !self.null_count.is_empty() { struct_ser.serialize_field("nullCount", &self.null_count)?; } - if self.distinct_count != 0 { + if !self.distinct_count.is_empty() { struct_ser.serialize_field("distinctCount", &self.distinct_count)?; } struct_ser.end() @@ -3264,7 +3264,8 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { return Err(serde::de::Error::duplicate_field("nullCount")); } null_count__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map.next_value::>>()? + .into_iter().map(|x| x.0).collect()) ; } GeneratedField::DistinctCount => { @@ -3272,7 +3273,8 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { return Err(serde::de::Error::duplicate_field("distinctCount")); } distinct_count__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map.next_value::>>()? + .into_iter().map(|x| x.0).collect()) ; } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 22a4beadb826..90409ac6320b 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2143,10 +2143,10 @@ pub struct ColumnStats { pub min_value: ::core::option::Option, #[prost(message, optional, tag = "2")] pub max_value: ::core::option::Option, - #[prost(uint32, tag = "3")] - pub null_count: u32, - #[prost(uint32, tag = "4")] - pub distinct_count: u32, + #[prost(uint32, repeated, tag = "3")] + pub null_count: ::prost::alloc::vec::Vec, + #[prost(uint32, repeated, tag = "4")] + pub distinct_count: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index e1d70634d884..48c335e1cdef 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -525,10 +525,18 @@ impl TryFrom<&protobuf::FileGroup> for Vec { impl From<&protobuf::ColumnStats> for ColumnStatistics { fn from(cs: &protobuf::ColumnStats) -> ColumnStatistics { ColumnStatistics { - null_count: Some(cs.null_count as usize), + null_count: if cs.null_count.is_empty() { + None + } else { + Some(cs.null_count[0] as usize) + }, max_value: cs.max_value.as_ref().map(|m| m.try_into().unwrap()), min_value: cs.min_value.as_ref().map(|m| m.try_into().unwrap()), - distinct_count: Some(cs.distinct_count as usize), + distinct_count: if cs.distinct_count.is_empty() { + None + } else { + Some(cs.distinct_count[0] as usize) + }, } } } diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index a5b1300360fe..cec12a64295f 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -471,8 +471,8 @@ impl From<&ColumnStatistics> for protobuf::ColumnStats { protobuf::ColumnStats { min_value: cs.min_value.as_ref().map(|m| m.try_into().unwrap()), max_value: cs.max_value.as_ref().map(|m| m.try_into().unwrap()), - null_count: cs.null_count.map(|n| n as u32).unwrap_or(0), - distinct_count: cs.distinct_count.map(|n| n as u32).unwrap_or(0), + null_count: cs.null_count.map(|n| vec![n as u32]).unwrap_or(vec![]), + distinct_count: cs.distinct_count.map(|n| vec![n as u32]).unwrap_or(vec![]), } } } From a2d1f2ec9bdd8a0d6100a62c3829ea099e8bf81d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 19 Sep 2023 15:41:32 +0300 Subject: [PATCH 11/53] Simplifications --- datafusion-examples/examples/csv_opener.rs | 2 +- .../examples/custom_datasource.rs | 2 +- datafusion-examples/examples/json_opener.rs | 2 +- datafusion/common/src/stats.rs | 37 +++--- .../core/src/datasource/file_format/arrow.rs | 2 +- .../core/src/datasource/file_format/avro.rs | 2 +- .../core/src/datasource/file_format/csv.rs | 2 +- .../core/src/datasource/file_format/json.rs | 2 +- .../core/src/datasource/listing/table.rs | 4 +- .../core/src/datasource/physical_plan/avro.rs | 6 +- .../physical_plan/file_scan_config.rs | 4 +- .../datasource/physical_plan/file_stream.rs | 2 +- .../core/src/datasource/physical_plan/json.rs | 8 +- .../core/src/datasource/physical_plan/mod.rs | 34 ++--- .../src/datasource/physical_plan/parquet.rs | 10 +- .../combine_partial_final_agg.rs | 2 +- .../enforce_distribution.rs | 10 +- .../replace_with_order_preserving_variants.rs | 2 +- .../core/src/physical_optimizer/test_utils.rs | 4 +- datafusion/core/src/test/mod.rs | 4 +- datafusion/core/src/test_util/mod.rs | 2 +- datafusion/core/src/test_util/parquet.rs | 2 +- .../provider_filter_pushdown.rs | 2 +- .../tests/custom_sources_cases/statistics.rs | 6 +- .../core/tests/parquet/custom_reader.rs | 2 +- datafusion/core/tests/parquet/page_pruning.rs | 2 +- .../core/tests/parquet/schema_coercion.rs | 4 +- datafusion/core/tests/path_partition.rs | 22 ++-- .../tests/user_defined/user_defined_plan.rs | 2 +- datafusion/execution/src/cache/cache_unit.rs | 13 +- datafusion/physical-expr/src/analysis.rs | 48 +++---- .../physical-expr/src/intervals/cp_solver.rs | 1 - .../src/intervals/interval_aritmetic.rs | 121 +++++++++--------- .../physical-plan/src/aggregates/mod.rs | 7 +- datafusion/physical-plan/src/analyze.rs | 2 +- datafusion/physical-plan/src/explain.rs | 2 +- datafusion/physical-plan/src/filter.rs | 6 +- datafusion/physical-plan/src/insert.rs | 2 +- .../src/joins/symmetric_hash_join.rs | 2 +- datafusion/physical-plan/src/streaming.rs | 2 +- datafusion/physical-plan/src/test/exec.rs | 2 +- datafusion/physical-plan/src/union.rs | 4 +- datafusion/physical-plan/src/unnest.rs | 2 +- .../substrait/src/physical_plan/consumer.rs | 6 +- .../tests/cases/roundtrip_physical_plan.rs | 2 +- 45 files changed, 192 insertions(+), 215 deletions(-) diff --git a/datafusion-examples/examples/csv_opener.rs b/datafusion-examples/examples/csv_opener.rs index bbd4fcb405a8..d749c3768f0b 100644 --- a/datafusion-examples/examples/csv_opener.rs +++ b/datafusion-examples/examples/csv_opener.rs @@ -61,7 +61,7 @@ async fn main() -> Result<()> { object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new(path.display().to_string(), 10)]], - statistics: Statistics::new_with_unbounded_columns(schema.clone()), + statistics: Statistics::new_with_unbounded_columns(&schema), projection: Some(vec![12, 0]), limit: Some(5), table_partition_cols: vec![], diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index 7e2d1725a403..065075a2cd76 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -271,6 +271,6 @@ impl ExecutionPlan for CustomExec { } fn statistics(&self) -> Statistics { - Statistics::new_with_unbounded_columns(self.schema()) + Statistics::new_with_unbounded_columns(&self.schema()) } } diff --git a/datafusion-examples/examples/json_opener.rs b/datafusion-examples/examples/json_opener.rs index 6d54858a95ef..27a5d35ef415 100644 --- a/datafusion-examples/examples/json_opener.rs +++ b/datafusion-examples/examples/json_opener.rs @@ -63,7 +63,7 @@ async fn main() -> Result<()> { object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new(path.to_string(), 10)]], - statistics: Statistics::new_with_unbounded_columns(schema.clone()), + statistics: Statistics::new_with_unbounded_columns(&schema), projection: Some(vec![1, 0]), limit: Some(5), table_partition_cols: vec![], diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index e7ca8e4d7d6f..a677ba6879c4 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -17,7 +17,7 @@ //! This module provides data structures to represent statistics -use arrow::datatypes::{DataType, SchemaRef}; +use arrow::datatypes::{DataType, Schema}; use std::fmt::Display; use crate::ScalarValue; @@ -44,28 +44,31 @@ impl Statistics { /// Returns a [`Statistics`] instance corresponding to the given schema by assigning infinite /// bounds to each column in the schema. This is useful when the input statistics are not /// known to give an opportunity to the current executor to shrink the bounds of some columns. - pub fn new_with_unbounded_columns(schema: SchemaRef) -> Self { + pub fn new_with_unbounded_columns(schema: &Schema) -> Self { Self { num_rows: None, total_byte_size: None, - column_statistics: Some( - schema - .fields() - .iter() - .map(|field| { - let inf = ScalarValue::try_from(field.data_type()).ok(); - ColumnStatistics { - null_count: None, - max_value: inf.clone(), - min_value: inf, - distinct_count: None, - } - }) - .collect(), - ), + column_statistics: Some(Statistics::unbounded_column_statistics(schema)), is_exact: false, } } + + /// Returns an unbounded ColumnStatistics for each field in the schema. + pub fn unbounded_column_statistics(schema: &Schema) -> Vec { + schema + .fields() + .iter() + .map(|field| { + let inf = ScalarValue::try_from(field.data_type()).ok(); + ColumnStatistics { + null_count: None, + max_value: inf.clone(), + min_value: inf, + distinct_count: None, + } + }) + .collect() + } } impl Display for Statistics { diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 32c64b772f29..04937660d65a 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -77,7 +77,7 @@ impl FileFormat for ArrowFormat { table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::new_with_unbounded_columns(table_schema)) + Ok(Statistics::new_with_unbounded_columns(&table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index 6d2039c22e9f..1920c85baf06 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -77,7 +77,7 @@ impl FileFormat for AvroFormat { table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::new_with_unbounded_columns(table_schema)) + Ok(Statistics::new_with_unbounded_columns(&table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index c681c6fa550b..efbe810974c2 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -238,7 +238,7 @@ impl FileFormat for CsvFormat { table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::new_with_unbounded_columns(table_schema)) + Ok(Statistics::new_with_unbounded_columns(&table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 97bf65404029..d770b2faa8ed 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -155,7 +155,7 @@ impl FileFormat for JsonFormat { table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::new_with_unbounded_columns(table_schema)) + Ok(Statistics::new_with_unbounded_columns(&table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 8cc5129698bf..233077ca2a2a 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -919,7 +919,7 @@ impl ListingTable { } else { return Ok(( vec![], - Statistics::new_with_unbounded_columns(self.file_schema.clone()), + Statistics::new_with_unbounded_columns(&self.file_schema), )); }; // list files (with partitions) @@ -966,7 +966,7 @@ impl ListingTable { } } } else { - Statistics::new_with_unbounded_columns(self.file_schema.clone()) + Statistics::new_with_unbounded_columns(&self.file_schema) }; Ok((part_file, statistics_result)) as Result<(PartitionedFile, Statistics)> }); diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index cc2fe9effa3e..620cf8cdc659 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -273,7 +273,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![vec![meta.into()]], file_schema: file_schema.clone(), - statistics: Statistics::new_with_unbounded_columns(file_schema), + statistics: Statistics::new_with_unbounded_columns(&file_schema), projection: Some(vec![0, 1, 2]), limit: None, table_partition_cols: vec![], @@ -345,7 +345,7 @@ mod tests { object_store_url, file_groups: vec![vec![meta.into()]], file_schema: file_schema.clone(), - statistics: Statistics::new_with_unbounded_columns(file_schema), + statistics: Statistics::new_with_unbounded_columns(&file_schema), projection, limit: None, table_partition_cols: vec![], @@ -418,7 +418,7 @@ mod tests { object_store_url, file_groups: vec![vec![partitioned_file]], file_schema: file_schema.clone(), - statistics: Statistics::new_with_unbounded_columns(file_schema), + statistics: Statistics::new_with_unbounded_columns(&file_schema), limit: None, table_partition_cols: vec![("date".to_owned(), DataType::Utf8)], output_ordering: vec![], 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 51f482bcc0b5..d82da71aa247 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -508,7 +508,7 @@ mod tests { let conf = config_for_projection( Arc::clone(&file_schema), None, - Statistics::new_with_unbounded_columns(file_schema.clone()), + Statistics::new_with_unbounded_columns(&file_schema), vec![( "date".to_owned(), wrap_partition_type_in_dict(DataType::Utf8), @@ -617,7 +617,7 @@ mod tests { file_batch.schema().fields().len(), file_batch.schema().fields().len() + 2, ]), - Statistics::new_with_unbounded_columns(file_batch.schema()), + Statistics::new_with_unbounded_columns(&file_batch.schema()), partition_cols.clone(), ); let (proj_schema, ..) = conf.project(); diff --git a/datafusion/core/src/datasource/physical_plan/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index 709444902c65..774ff26f3b77 100644 --- a/datafusion/core/src/datasource/physical_plan/file_stream.rs +++ b/datafusion/core/src/datasource/physical_plan/file_stream.rs @@ -662,7 +662,7 @@ mod tests { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: file_schema.clone(), file_groups: vec![file_group], - statistics: Statistics::new_with_unbounded_columns(file_schema), + statistics: Statistics::new_with_unbounded_columns(&file_schema), projection: None, limit: self.limit, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 95ea80a87342..2224e0c7774b 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -457,7 +457,7 @@ mod tests { object_store_url, file_groups, file_schema: file_schema.clone(), - statistics: Statistics::new_with_unbounded_columns(file_schema), + statistics: Statistics::new_with_unbounded_columns(&file_schema), projection: None, limit: Some(3), table_partition_cols: vec![], @@ -536,7 +536,7 @@ mod tests { object_store_url, file_groups, file_schema: file_schema.clone(), - statistics: Statistics::new_with_unbounded_columns(file_schema), + statistics: Statistics::new_with_unbounded_columns(&file_schema), projection: None, limit: Some(3), table_partition_cols: vec![], @@ -584,7 +584,7 @@ mod tests { object_store_url, file_groups, file_schema: file_schema.clone(), - statistics: Statistics::new_with_unbounded_columns(file_schema), + statistics: Statistics::new_with_unbounded_columns(&file_schema), projection: Some(vec![0, 2]), limit: None, table_partition_cols: vec![], @@ -637,7 +637,7 @@ mod tests { object_store_url, file_groups, file_schema: file_schema.clone(), - statistics: Statistics::new_with_unbounded_columns(file_schema), + statistics: Statistics::new_with_unbounded_columns(&file_schema), projection: Some(vec![3, 0, 2]), limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 7d03ce09211f..53bf06abbe12 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -799,9 +799,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: file_group, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(Arc::new( - Schema::empty(), - )), + statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -864,9 +862,9 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: fg.clone(), file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(Arc::new( - Schema::empty(), - )), + statistics: Statistics::new_with_unbounded_columns( + &Arc::new(Schema::empty()), + ), projection: None, limit: None, table_partition_cols: vec![], @@ -900,9 +898,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: single_partition, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(Arc::new( - Schema::empty(), - )), + statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -939,9 +935,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: single_partition, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(Arc::new( - Schema::empty(), - )), + statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -984,9 +978,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: source_partitions, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(Arc::new( - Schema::empty(), - )), + statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -1025,9 +1017,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: source_partitions, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(Arc::new( - Schema::empty(), - )), + statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -1067,9 +1057,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: source_partitions, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(Arc::new( - Schema::empty(), - )), + statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -1098,9 +1086,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: single_partition, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(Arc::new( - Schema::empty(), - )), + statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 7b8158444274..5e9607afcbde 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -861,7 +861,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], file_schema: file_schema.clone(), - statistics: Statistics::new_with_unbounded_columns(file_schema), + statistics: Statistics::new_with_unbounded_columns(&file_schema), projection, limit: None, table_partition_cols: vec![], @@ -1518,7 +1518,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups, file_schema: file_schema.clone(), - statistics: Statistics::new_with_unbounded_columns(file_schema), + statistics: Statistics::new_with_unbounded_columns(&file_schema), projection: None, limit: None, table_partition_cols: vec![], @@ -1621,7 +1621,7 @@ mod tests { object_store_url, file_groups: vec![vec![partitioned_file]], file_schema: schema.clone(), - statistics: Statistics::new_with_unbounded_columns(schema), + statistics: Statistics::new_with_unbounded_columns(&schema), // file has 10 cols so index 12 should be month and 13 should be day projection: Some(vec![0, 1, 2, 12, 13]), limit: None, @@ -1695,9 +1695,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![vec![partitioned_file]], file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(Arc::new( - Schema::empty(), - )), + statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], 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 2c63ad10f55d..2a54df3e8f35 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -248,7 +248,7 @@ mod tests { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_with_unbounded_columns(schema.clone()), + statistics: Statistics::new_with_unbounded_columns(schema), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 803a42617816..549b1bfdb267 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1802,7 +1802,7 @@ mod tests { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_with_unbounded_columns(schema()), + statistics: Statistics::new_with_unbounded_columns(&schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -1830,7 +1830,7 @@ mod tests { vec![PartitionedFile::new("x".to_string(), 100)], vec![PartitionedFile::new("y".to_string(), 100)], ], - statistics: Statistics::new_with_unbounded_columns(schema()), + statistics: Statistics::new_with_unbounded_columns(&schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -1852,7 +1852,7 @@ mod tests { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_with_unbounded_columns(schema()), + statistics: Statistics::new_with_unbounded_columns(&schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -1883,7 +1883,7 @@ mod tests { vec![PartitionedFile::new("x".to_string(), 100)], vec![PartitionedFile::new("y".to_string(), 100)], ], - statistics: Statistics::new_with_unbounded_columns(schema()), + statistics: Statistics::new_with_unbounded_columns(&schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -3862,7 +3862,7 @@ mod tests { "x".to_string(), 100, )]], - statistics: Statistics::new_with_unbounded_columns(schema()), + statistics: Statistics::new_with_unbounded_columns(&schema()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 593061210b95..cdf2439017de 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -897,7 +897,7 @@ mod tests { "file_path".to_string(), 100, )]], - statistics: Statistics::new_with_unbounded_columns(schema.clone()), + statistics: Statistics::new_with_unbounded_columns(schema), projection: Some(projection), limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 10b55d5f798d..04db5f804de0 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -269,7 +269,7 @@ pub fn parquet_exec(schema: &SchemaRef) -> Arc { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_with_unbounded_columns(schema.clone()), + statistics: Statistics::new_with_unbounded_columns(schema), projection: None, limit: None, table_partition_cols: vec![], @@ -293,7 +293,7 @@ pub fn parquet_exec_sorted( object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_with_unbounded_columns(schema.clone()), + statistics: Statistics::new_with_unbounded_columns(schema), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 41726a7c9f09..16d0ad6e0707 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -194,7 +194,7 @@ pub fn partitioned_csv_config( object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: schema.clone(), file_groups, - statistics: Statistics::new_with_unbounded_columns(schema), + statistics: Statistics::new_with_unbounded_columns(&schema), projection: None, limit: None, table_partition_cols: vec![], @@ -282,7 +282,7 @@ pub fn csv_exec_sorted( object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_with_unbounded_columns(schema.clone()), + statistics: Statistics::new_with_unbounded_columns(schema), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 101be0ae4c07..eb381c8d79cf 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -239,7 +239,7 @@ impl ExecutionPlan for UnboundedExec { } fn statistics(&self) -> Statistics { - Statistics::new_with_unbounded_columns(self.schema()) + Statistics::new_with_unbounded_columns(&self.schema()) } } diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index fdd89569b137..f06c8506ea52 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -148,7 +148,7 @@ impl TestParquetFile { range: None, extensions: None, }]], - statistics: Statistics::new_with_unbounded_columns(self.schema.clone()), + statistics: Statistics::new_with_unbounded_columns(&self.schema), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index f6af3a3041e8..095a82932b89 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -115,7 +115,7 @@ impl ExecutionPlan for CustomPlan { fn statistics(&self) -> Statistics { // here we could provide more accurate statistics // but we want to test the filter pushdown not the CBOs - Statistics::new_with_unbounded_columns(self.schema()) + Statistics::new_with_unbounded_columns(&self.schema()) } } diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index b446708f0e7f..0a1c18f842c2 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -249,11 +249,7 @@ async fn sql_limit() -> Result<()> { let physical_plan = df.create_physical_plan().await.unwrap(); // when the limit is smaller than the original number of lines // we loose all statistics except the for number of rows which becomes the limit - let col_stats = schema - .fields() - .iter() - .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) - .collect::>(); + let col_stats = Statistics::unbounded_column_statistics(&schema); assert_eq!( Statistics { num_rows: Some(5), diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index 7a61b6ceb1d7..d755e5fd574d 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -78,7 +78,7 @@ async fn route_data_access_ops_to_parquet_file_reader_factory() { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], file_schema: file_schema.clone(), - statistics: Statistics::new_with_unbounded_columns(file_schema), + statistics: Statistics::new_with_unbounded_columns(&file_schema), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 0bdad7ef4492..6b5a2e4f1010 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -72,7 +72,7 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { object_store_url, file_groups: vec![vec![partitioned_file]], file_schema: schema.clone(), - statistics: Statistics::new_with_unbounded_columns(schema), + statistics: Statistics::new_with_unbounded_columns(&schema), // file has 10 cols so index 12 should be month projection: None, limit: None, diff --git a/datafusion/core/tests/parquet/schema_coercion.rs b/datafusion/core/tests/parquet/schema_coercion.rs index e3925d48e12f..614e2e3bb180 100644 --- a/datafusion/core/tests/parquet/schema_coercion.rs +++ b/datafusion/core/tests/parquet/schema_coercion.rs @@ -63,7 +63,7 @@ async fn multi_parquet_coercion() { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], file_schema: file_schema.clone(), - statistics: Statistics::new_with_unbounded_columns(file_schema), + statistics: Statistics::new_with_unbounded_columns(&file_schema), projection: None, limit: None, table_partition_cols: vec![], @@ -127,7 +127,7 @@ async fn multi_parquet_coercion_projection() { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], file_schema: file_schema.clone(), - statistics: Statistics::new_with_unbounded_columns(file_schema), + statistics: Statistics::new_with_unbounded_columns(&file_schema), projection: Some(vec![1, 0, 2]), limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index a9ecc296b8be..bd50a41dcc23 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -458,7 +458,8 @@ async fn parquet_statistics() -> Result<()> { //// NO PROJECTION //// let dataframe = ctx.sql("SELECT * FROM t").await?; let physical_plan = dataframe.create_physical_plan().await?; - assert_eq!(physical_plan.schema().fields().len(), 4); + let schema = physical_plan.schema(); + assert_eq!(schema.fields().len(), 4); let stat_cols = physical_plan .statistics() @@ -470,27 +471,22 @@ async fn parquet_statistics() -> Result<()> { // TODO assert partition column (1,2,3) stats once implemented (#1186) assert_eq!( stat_cols[1], - ColumnStatistics::new_with_unbounded_column( - physical_plan.schema().field(1).data_type() - ), + ColumnStatistics::new_with_unbounded_column(schema.field(1).data_type()), ); assert_eq!( stat_cols[2], - ColumnStatistics::new_with_unbounded_column( - physical_plan.schema().field(2).data_type() - ), + ColumnStatistics::new_with_unbounded_column(schema.field(2).data_type()), ); assert_eq!( stat_cols[3], - ColumnStatistics::new_with_unbounded_column( - physical_plan.schema().field(3).data_type() - ), + ColumnStatistics::new_with_unbounded_column(schema.field(3).data_type()), ); //// WITH PROJECTION //// let dataframe = ctx.sql("SELECT mycol, day FROM t WHERE day='28'").await?; let physical_plan = dataframe.create_physical_plan().await?; - assert_eq!(physical_plan.schema().fields().len(), 2); + let schema = physical_plan.schema(); + assert_eq!(schema.fields().len(), 2); let stat_cols = physical_plan .statistics() @@ -502,9 +498,7 @@ async fn parquet_statistics() -> Result<()> { // TODO assert partition column stats once implemented (#1186) assert_eq!( stat_cols[1], - ColumnStatistics::new_with_unbounded_column( - physical_plan.schema().field(1).data_type() - ), + ColumnStatistics::new_with_unbounded_column(schema.field(1).data_type()), ); Ok(()) diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 6fcc078489b5..61aa2a2645d2 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -493,7 +493,7 @@ impl ExecutionPlan for TopKExec { fn statistics(&self) -> Statistics { // to improve the optimizability of this plan // better statistics inference could be provided - Statistics::new_with_unbounded_columns(self.schema()) + Statistics::new_with_unbounded_columns(&self.schema()) } } diff --git a/datafusion/execution/src/cache/cache_unit.rs b/datafusion/execution/src/cache/cache_unit.rs index 2c40c17f0df9..c0a94f39b8f3 100644 --- a/datafusion/execution/src/cache/cache_unit.rs +++ b/datafusion/execution/src/cache/cache_unit.rs @@ -96,7 +96,6 @@ impl CacheAccessor> for DefaultFileStatisticsCache { #[cfg(test)] mod tests { - use std::sync::Arc; use crate::cache::cache_unit::DefaultFileStatisticsCache; use crate::cache::CacheAccessor; @@ -121,13 +120,11 @@ mod tests { cache.put_with_extra( &meta.location, - Statistics::new_with_unbounded_columns(Arc::new(Schema::new(vec![ - Field::new( - "test_column", - DataType::Timestamp(TimeUnit::Second, None), - false, - ), - ]))) + Statistics::new_with_unbounded_columns(&Schema::new(vec![Field::new( + "test_column", + DataType::Timestamp(TimeUnit::Second, None), + false, + )])) .into(), &meta, ); diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index 1349d5ab116b..d7c91b61da8b 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -147,20 +147,22 @@ pub fn analyze( }) }) .collect(); - match graph.update_ranges(&mut target_indices_and_boundaries)? { - PropagationResult::Success => Ok(shrink_boundaries( - expr, - graph, - target_boundaries, - target_expr_and_indices, - )), - PropagationResult::Infeasible => { - Ok(AnalysisContext::new(target_boundaries).with_selectivity(0.0)) - } - PropagationResult::CannotPropagate => { - Ok(AnalysisContext::new(target_boundaries).with_selectivity(1.0)) - } - } + Ok( + match graph.update_ranges(&mut target_indices_and_boundaries)? { + PropagationResult::Success => shrink_boundaries( + expr, + graph, + target_boundaries, + target_expr_and_indices, + )?, + PropagationResult::Infeasible => { + AnalysisContext::new(target_boundaries).with_selectivity(0.0) + } + PropagationResult::CannotPropagate => { + AnalysisContext::new(target_boundaries).with_selectivity(1.0) + } + }, + ) } /// If the `PropagationResult` indicates success, this function calculates the @@ -172,7 +174,7 @@ fn shrink_boundaries( mut graph: ExprIntervalGraph, mut target_boundaries: Vec, target_expr_and_indices: Vec<(Arc, usize)>, -) -> AnalysisContext { +) -> Result { let initial_boundaries = target_boundaries.clone(); target_expr_and_indices.iter().for_each(|(expr, i)| { if let Some(column) = expr.as_any().downcast_ref::() { @@ -195,9 +197,9 @@ fn shrink_boundaries( &final_result.upper.value, &target_boundaries, &initial_boundaries, - ); + )?; - AnalysisContext::new(target_boundaries).with_selectivity(selectivity) + Ok(AnalysisContext::new(target_boundaries).with_selectivity(selectivity)) } /// This function calculates the filter predicate's selectivity by comparing @@ -214,20 +216,20 @@ fn calculate_selectivity( upper_value: &ScalarValue, target_boundaries: &[ExprBoundaries], initial_boundaries: &[ExprBoundaries], -) -> f64 { +) -> Result { match (lower_value, upper_value) { - (ScalarValue::Boolean(Some(true)), ScalarValue::Boolean(Some(true))) => 1.0, - (ScalarValue::Boolean(Some(false)), ScalarValue::Boolean(Some(false))) => 0.0, + (ScalarValue::Boolean(Some(true)), ScalarValue::Boolean(Some(true))) => Ok(1.0), + (ScalarValue::Boolean(Some(false)), ScalarValue::Boolean(Some(false))) => Ok(0.0), _ => { // Since the intervals are assumed uniform and the values // are not correlated, we need to multiply the selectivities // of multiple columns to get the overall selectivity. - target_boundaries.iter().enumerate().fold( + target_boundaries.iter().enumerate().try_fold( 1.0, |acc, (i, ExprBoundaries { interval, .. })| { let temp = - cardinality_ratio(&initial_boundaries[i].interval, interval); - acc * temp + cardinality_ratio(&initial_boundaries[i].interval, interval)?; + Ok(acc * temp) }, ) } diff --git a/datafusion/physical-expr/src/intervals/cp_solver.rs b/datafusion/physical-expr/src/intervals/cp_solver.rs index 7e6c71db4739..0a090636dc4b 100644 --- a/datafusion/physical-expr/src/intervals/cp_solver.rs +++ b/datafusion/physical-expr/src/intervals/cp_solver.rs @@ -536,7 +536,6 @@ impl ExprIntervalGraph { let propagated_intervals = self.graph[node] .expr .propagate_constraints(node_interval, &children_intervals)?; - for (child, interval) in children.into_iter().zip(propagated_intervals) { if let Some(interval) = interval { self.graph[child].interval = interval; diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index 682939730532..5b9a5886a990 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -511,53 +511,52 @@ impl Interval { /// - If any of the bounds is an infinite bound /// - If the type is not implemented yet /// - If there is an overflow during a computation - pub fn cardinality(&self) -> Option { - match self.get_datatype() { - Ok(data_type) if data_type.is_integer() => { - self.upper.value.distance(&self.lower.value).map(|diff| { - calculate_cardinality_based_on_bounds( - self.lower.open, - self.upper.open, - diff as u64, - ) - }) - } - // Ordering floating-point numbers according to their binary representations - // coincide with their natural ordering. Therefore, we can consider their - // binary representations as "indices" and subtract them. For details, see: - // https://stackoverflow.com/questions/8875064/how-many-distinct-floating-point-numbers-in-a-specific-range - Ok(data_type) if data_type.is_floating() => { - // If the minimum value is a negative number, we need to - // switch sides to ensure an unsigned result. - let new_zero = - ScalarValue::new_zero(&self.lower.value.data_type()).ok()?; - let (min, max) = if self.lower.value < new_zero { - (self.upper.value.clone(), self.lower.value.clone()) - } else { - (self.lower.value.clone(), self.upper.value.clone()) - }; + pub fn cardinality(&self) -> Result> { + let data_type = self.get_datatype()?; + if data_type.is_integer() { + Ok(self.upper.value.distance(&self.lower.value).map(|diff| { + calculate_cardinality_based_on_bounds( + self.lower.open, + self.upper.open, + diff as u64, + ) + })) + } + // Ordering floating-point numbers according to their binary representations + // coincide with their natural ordering. Therefore, we can consider their + // binary representations as "indices" and subtract them. For details, see: + // https://stackoverflow.com/questions/8875064/how-many-distinct-floating-point-numbers-in-a-specific-range + else if data_type.is_floating() { + // If the minimum value is a negative number, we need to + // switch sides to ensure an unsigned result. + let new_zero = ScalarValue::new_zero(&self.lower.value.data_type())?; + let (min, max) = if self.lower.value < new_zero { + (self.upper.value.clone(), self.lower.value.clone()) + } else { + (self.lower.value.clone(), self.upper.value.clone()) + }; - match (min, max) { - ( - ScalarValue::Float32(Some(lower)), - ScalarValue::Float32(Some(upper)), - ) => Some(calculate_cardinality_based_on_bounds( - self.lower.open, - self.upper.open, - (upper.to_bits().sub_checked(lower.to_bits()).ok()?) as u64, - )), - ( - ScalarValue::Float64(Some(lower)), - ScalarValue::Float64(Some(upper)), - ) => Some(calculate_cardinality_based_on_bounds( - self.lower.open, - self.upper.open, - upper.to_bits().sub_checked(lower.to_bits()).ok()?, - )), - _ => None, - } - } - _ => None, + Ok(match (min, max) { + ( + ScalarValue::Float32(Some(lower)), + ScalarValue::Float32(Some(upper)), + ) => Some(calculate_cardinality_based_on_bounds( + self.lower.open, + self.upper.open, + (upper.to_bits().sub_checked(lower.to_bits())?) as u64, + )), + ( + ScalarValue::Float64(Some(lower)), + ScalarValue::Float64(Some(upper)), + ) => Some(calculate_cardinality_based_on_bounds( + self.lower.open, + self.upper.open, + upper.to_bits().sub_checked(lower.to_bits())?, + )), + _ => None, + }) + } else { + Ok(None) } } @@ -661,13 +660,21 @@ fn next_value(value: ScalarValue) -> ScalarValue { } /// This function computes the cardinality ratio of the given intervals. -pub fn cardinality_ratio(initial_interval: &Interval, final_interval: &Interval) -> f64 { - match (final_interval.cardinality(), initial_interval.cardinality()) { - (Some(final_interval), Some(initial_interval)) => { - final_interval as f64 / initial_interval as f64 - } - _ => 1.0, - } +pub fn cardinality_ratio( + initial_interval: &Interval, + final_interval: &Interval, +) -> Result { + Ok( + match ( + final_interval.cardinality()?, + initial_interval.cardinality()?, + ) { + (Some(final_interval), Some(initial_interval)) => { + final_interval as f64 / initial_interval as f64 + } + _ => 1.0, + }, + ) } pub fn apply_operator(op: &Operator, lhs: &Interval, rhs: &Interval) -> Result { @@ -1686,7 +1693,7 @@ mod tests { ), ]; for interval in intervals { - assert_eq!(interval.cardinality().unwrap(), distinct_f64); + assert_eq!(interval.cardinality()?.unwrap(), distinct_f64); } let intervals = [ @@ -1700,20 +1707,20 @@ mod tests { ), ]; for interval in intervals { - assert_eq!(interval.cardinality().unwrap(), distinct_f32); + assert_eq!(interval.cardinality()?.unwrap(), distinct_f32); } let interval = Interval::new( IntervalBound::new(ScalarValue::from(-0.0625), false), IntervalBound::new(ScalarValue::from(0.0625), true), ); - assert_eq!(interval.cardinality().unwrap(), distinct_f64 * 2_048); + assert_eq!(interval.cardinality()?.unwrap(), distinct_f64 * 2_048); let interval = Interval::new( IntervalBound::new(ScalarValue::from(-0.0625_f32), false), IntervalBound::new(ScalarValue::from(0.0625_f32), true), ); - assert_eq!(interval.cardinality().unwrap(), distinct_f32 * 256); + assert_eq!(interval.cardinality()?.unwrap(), distinct_f32 * 256); Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 99a4ee0e38d8..2cedf3923790 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1008,12 +1008,7 @@ impl ExecutionPlan for AggregateExec { // - case where we group by on a column for which with have the `distinct` stat // TODO stats: aggr expression: // - aggregations somtimes also preserve invariants such as min, max... - let column_statistics = self - .schema() - .fields() - .iter() - .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) - .collect::>(); + let column_statistics = Statistics::unbounded_column_statistics(&self.schema()); match self.mode { AggregateMode::Final | AggregateMode::FinalPartitioned if self.group_by.expr.is_empty() => diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index 95a9ac3f0c4c..9d7df458d157 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -183,7 +183,7 @@ impl ExecutionPlan for AnalyzeExec { fn statistics(&self) -> Statistics { // Statistics an an ANALYZE plan are not relevant - Statistics::new_with_unbounded_columns(self.schema()) + Statistics::new_with_unbounded_columns(&self.schema()) } } diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index 16e326d0a49a..91a0bb8f9948 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -171,7 +171,7 @@ impl ExecutionPlan for ExplainExec { fn statistics(&self) -> Statistics { // Statistics an EXPLAIN plan are not relevant - Statistics::new_with_unbounded_columns(self.schema()) + Statistics::new_with_unbounded_columns(&self.schema()) } } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 8739e8b2be3f..421cd5e23e03 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -238,7 +238,7 @@ impl ExecutionPlan for FilterExec { }; } } - Statistics::new_with_unbounded_columns(self.schema()) + Statistics::new_with_unbounded_columns(&self.schema()) } } @@ -609,7 +609,7 @@ mod tests { // a: min=???, max=??? (missing) let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let input = Arc::new(StatisticsExec::new( - Statistics::new_with_unbounded_columns(Arc::new(schema.clone())), + Statistics::new_with_unbounded_columns(&schema), schema.clone(), )); @@ -929,7 +929,7 @@ mod tests { async fn test_empty_input_statistics() -> Result<()> { let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let input = Arc::new(StatisticsExec::new( - Statistics::new_with_unbounded_columns(Arc::new(schema.clone())), + Statistics::new_with_unbounded_columns(&schema), schema, )); // WHERE a <= 10 AND 0 <= a - 5 diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index d220d25d34ae..c5fec688dc16 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -251,7 +251,7 @@ impl ExecutionPlan for FileSinkExec { } fn statistics(&self) -> Statistics { - Statistics::new_with_unbounded_columns(self.schema()) + Statistics::new_with_unbounded_columns(&self.schema()) } } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 0b80a7c197a1..651f0a3ae51d 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -479,7 +479,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { fn statistics(&self) -> Statistics { // TODO stats: it is not possible in general to know the output size of joins - Statistics::new_with_unbounded_columns(self.schema()) + Statistics::new_with_unbounded_columns(&self.schema()) } fn execute( diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 980579a986f7..19516061317b 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -186,6 +186,6 @@ impl ExecutionPlan for StreamingTableExec { } fn statistics(&self) -> Statistics { - Statistics::new_with_unbounded_columns(self.schema()) + Statistics::new_with_unbounded_columns(&self.schema()) } } diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index 2476ae1dbfcd..05a6ae6191f2 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -449,7 +449,7 @@ impl ExecutionPlan for ErrorExec { } fn statistics(&self) -> Statistics { - Statistics::new_with_unbounded_columns(self.schema()) + Statistics::new_with_unbounded_columns(&self.schema()) } } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 6046807fa3da..84de40bdca57 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -269,7 +269,7 @@ impl ExecutionPlan for UnionExec { .iter() .map(|ep| ep.statistics()) .reduce(stats_union) - .unwrap_or(Statistics::new_with_unbounded_columns(self.schema())) + .unwrap_or(Statistics::new_with_unbounded_columns(&self.schema())) } fn benefits_from_input_partitioning(&self) -> Vec { @@ -443,7 +443,7 @@ impl ExecutionPlan for InterleaveExec { .iter() .map(|ep| ep.statistics()) .reduce(stats_union) - .unwrap_or(Statistics::new_with_unbounded_columns(self.schema())) + .unwrap_or(Statistics::new_with_unbounded_columns(&self.schema())) } fn benefits_from_input_partitioning(&self) -> Vec { diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index e5b13a9890b7..1a011a1141af 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -161,7 +161,7 @@ impl ExecutionPlan for UnnestExec { } fn statistics(&self) -> Statistics { - Statistics::new_with_unbounded_columns(self.schema()) + Statistics::new_with_unbounded_columns(&self.schema()) } } diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index f9ff671a5e06..4e25d4952a9a 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -104,9 +104,9 @@ pub async fn from_substrait_rel( object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: Arc::new(Schema::empty()), file_groups, - statistics: Statistics::new_with_unbounded_columns(Arc::new( - Schema::empty(), - )), + statistics: Statistics::new_with_unbounded_columns( + &Schema::empty(), + ), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs index 85a40585253b..49b20cc17877 100644 --- a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs @@ -42,7 +42,7 @@ async fn parquet_exec() -> Result<()> { 123, )], ], - statistics: Statistics::new_with_unbounded_columns(Arc::new(Schema::empty())), + statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], From fa72d41aa9edd55b9e92ae830a20c8e10636c790 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 20 Sep 2023 10:23:37 +0300 Subject: [PATCH 12/53] statistics() returns result --- .../examples/custom_datasource.rs | 4 +- .../core/src/datasource/file_format/csv.rs | 4 +- .../core/src/datasource/file_format/json.rs | 4 +- .../src/datasource/file_format/parquet.rs | 10 ++-- .../core/src/datasource/listing/table.rs | 12 ++--- .../datasource/physical_plan/arrow_file.rs | 4 +- .../core/src/datasource/physical_plan/avro.rs | 4 +- .../core/src/datasource/physical_plan/csv.rs | 4 +- .../core/src/datasource/physical_plan/json.rs | 4 +- .../src/datasource/physical_plan/parquet.rs | 4 +- .../aggregate_statistics.rs | 8 +-- .../enforce_distribution.rs | 4 +- .../src/physical_optimizer/join_selection.rs | 51 +++++++++++++------ datafusion/core/src/physical_planner.rs | 2 +- datafusion/core/src/test/mod.rs | 4 +- datafusion/core/src/test_util/mod.rs | 4 +- datafusion/core/tests/custom_sources.rs | 6 +-- .../provider_filter_pushdown.rs | 4 +- .../tests/custom_sources_cases/statistics.rs | 14 ++--- .../core/tests/parquet/file_statistics.rs | 12 ++--- datafusion/core/tests/path_partition.rs | 4 +- .../tests/user_defined/user_defined_plan.rs | 4 +- .../physical-plan/src/aggregates/mod.rs | 24 +++++---- datafusion/physical-plan/src/analyze.rs | 4 +- .../physical-plan/src/coalesce_batches.rs | 2 +- .../physical-plan/src/coalesce_partitions.rs | 2 +- datafusion/physical-plan/src/display.rs | 8 +-- datafusion/physical-plan/src/empty.rs | 8 ++- datafusion/physical-plan/src/explain.rs | 4 +- datafusion/physical-plan/src/filter.rs | 40 ++++++++------- datafusion/physical-plan/src/insert.rs | 4 +- .../physical-plan/src/joins/cross_join.rs | 10 ++-- .../physical-plan/src/joins/hash_join.rs | 2 +- .../src/joins/nested_loop_join.rs | 2 +- .../src/joins/sort_merge_join.rs | 2 +- .../src/joins/symmetric_hash_join.rs | 4 +- datafusion/physical-plan/src/joins/utils.rs | 10 ++-- datafusion/physical-plan/src/lib.rs | 2 +- datafusion/physical-plan/src/limit.rs | 22 ++++---- datafusion/physical-plan/src/memory.rs | 6 +-- datafusion/physical-plan/src/projection.rs | 8 +-- .../physical-plan/src/repartition/mod.rs | 2 +- datafusion/physical-plan/src/sorts/sort.rs | 2 +- .../src/sorts/sort_preserving_merge.rs | 2 +- datafusion/physical-plan/src/streaming.rs | 4 +- datafusion/physical-plan/src/test/exec.rs | 30 +++++++---- datafusion/physical-plan/src/union.rs | 26 +++++++--- datafusion/physical-plan/src/unnest.rs | 4 +- datafusion/physical-plan/src/values.rs | 8 ++- .../src/windows/bounded_window_agg_exec.rs | 8 +-- .../src/windows/window_agg_exec.rs | 8 +-- 51 files changed, 241 insertions(+), 188 deletions(-) diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index 065075a2cd76..c97f48bd5987 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -270,7 +270,7 @@ impl ExecutionPlan for CustomExec { )?)) } - fn statistics(&self) -> Statistics { - Statistics::new_with_unbounded_columns(&self.schema()) + fn statistics(&self) -> Result { + Ok(Statistics::new_with_unbounded_columns(&self.schema())) } } diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index efbe810974c2..e60e95514ef3 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -642,8 +642,8 @@ mod tests { assert_eq!(tt_batches, 50 /* 100/2 */); // test metadata - assert_eq!(exec.statistics().num_rows, None); - assert_eq!(exec.statistics().total_byte_size, None); + assert_eq!(exec.statistics()?.num_rows, None); + assert_eq!(exec.statistics()?.total_byte_size, None); Ok(()) } diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index d770b2faa8ed..74c291cff2dd 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -408,8 +408,8 @@ mod tests { assert_eq!(tt_batches, 6 /* 12/2 */); // test metadata - assert_eq!(exec.statistics().num_rows, None); - assert_eq!(exec.statistics().total_byte_size, None); + assert_eq!(exec.statistics()?.num_rows, None); + assert_eq!(exec.statistics()?.total_byte_size, None); Ok(()) } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 0200238c585b..06bcb03567fc 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -1122,8 +1122,8 @@ mod tests { assert_eq!(tt_batches, 4 /* 8/2 */); // test metadata - assert_eq!(exec.statistics().num_rows, Some(8)); - assert_eq!(exec.statistics().total_byte_size, Some(671)); + assert_eq!(exec.statistics()?.num_rows, Some(8)); + assert_eq!(exec.statistics()?.total_byte_size, Some(671)); Ok(()) } @@ -1164,9 +1164,9 @@ mod tests { get_exec(&state, "alltypes_plain.parquet", projection, Some(1)).await?; // note: even if the limit is set, the executor rounds up to the batch size - assert_eq!(exec.statistics().num_rows, Some(8)); - assert_eq!(exec.statistics().total_byte_size, Some(671)); - assert!(exec.statistics().is_exact); + assert_eq!(exec.statistics()?.num_rows, Some(8)); + assert_eq!(exec.statistics()?.total_byte_size, Some(671)); + assert!(exec.statistics()?.is_exact); let batches = collect(exec, task_ctx).await?; assert_eq!(1, batches.len()); assert_eq!(11, batches[0].num_columns()); diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 233077ca2a2a..6f61db4a4a29 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1048,8 +1048,8 @@ mod tests { assert_eq!(exec.output_partitioning().partition_count(), 1); // test metadata - assert_eq!(exec.statistics().num_rows, Some(8)); - assert_eq!(exec.statistics().total_byte_size, Some(671)); + assert_eq!(exec.statistics()?.num_rows, Some(8)); + assert_eq!(exec.statistics()?.total_byte_size, Some(671)); Ok(()) } @@ -1071,8 +1071,8 @@ mod tests { let table = ListingTable::try_new(config)?; let exec = table.scan(&state, None, &[], None).await?; - assert_eq!(exec.statistics().num_rows, Some(8)); - assert_eq!(exec.statistics().total_byte_size, Some(671)); + assert_eq!(exec.statistics()?.num_rows, Some(8)); + assert_eq!(exec.statistics()?.total_byte_size, Some(671)); Ok(()) } @@ -1095,8 +1095,8 @@ mod tests { let table = ListingTable::try_new(config)?; let exec = table.scan(&state, None, &[], None).await?; - assert_eq!(exec.statistics().num_rows, None); - assert_eq!(exec.statistics().total_byte_size, None); + assert_eq!(exec.statistics()?.num_rows, None); + assert_eq!(exec.statistics()?.total_byte_size, None); Ok(()) } diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index a47376248ed3..ef4e3d418ef7 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -143,8 +143,8 @@ impl ExecutionPlan for ArrowExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - self.projected_statistics.clone() + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) } } diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 620cf8cdc659..75470d83ce54 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -154,8 +154,8 @@ impl ExecutionPlan for AvroExec { Ok(Box::pin(stream)) } - fn statistics(&self) -> Statistics { - self.projected_statistics.clone() + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) } fn metrics(&self) -> Option { diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index aedc9198a88d..2a4d9bc017dd 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -234,8 +234,8 @@ impl ExecutionPlan for CsvExec { Ok(Box::pin(stream) as SendableRecordBatchStream) } - fn statistics(&self) -> Statistics { - self.projected_statistics.clone() + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) } fn metrics(&self) -> Option { diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 2224e0c7774b..da94ba9adc5e 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -163,8 +163,8 @@ impl ExecutionPlan for NdJsonExec { Ok(Box::pin(stream) as SendableRecordBatchStream) } - fn statistics(&self) -> Statistics { - self.projected_statistics.clone() + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) } fn metrics(&self) -> Option { diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 5e9607afcbde..22bf4ecd7ec7 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -387,8 +387,8 @@ impl ExecutionPlan for ParquetExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - self.projected_statistics.clone() + fn statistics(&self) -> Result { + Ok(self.projected_statistics.clone()) } } diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index e7db9610fd60..ce68f4f39c21 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -58,7 +58,7 @@ impl PhysicalOptimizerRule for AggregateStatistics { .as_any() .downcast_ref::() .expect("take_optimizable() ensures that this is a AggregateExec"); - let stats = partial_agg_exec.input().statistics(); + let stats = partial_agg_exec.input().statistics()?; let mut projections = vec![]; for expr in partial_agg_exec.aggr_expr() { if let Some((non_null_rows, name)) = @@ -126,8 +126,10 @@ fn take_optimizable(node: &dyn ExecutionPlan) -> Option> && partial_agg_exec.filter_expr().iter().all(|e| e.is_none()) { let stats = partial_agg_exec.input().statistics(); - if stats.is_exact { - return Some(child); + if let Ok(stats) = stats { + if stats.is_exact { + return Some(child); + } } } } diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 549b1bfdb267..322ac7bcd809 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1252,7 +1252,7 @@ fn ensure_distribution( return Ok(Transformed::No(dist_context)); } // Don't need to apply when the returned row count is not greater than 1: - let stats = dist_context.plan.statistics(); + let stats = dist_context.plan.statistics()?; let mut repartition_beneficial_stat = true; if stats.is_exact { repartition_beneficial_stat = @@ -1775,7 +1775,7 @@ mod tests { unreachable!(); } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 344cdc7e896e..d04d3253f3a8 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -64,13 +64,15 @@ fn should_swap_join_order(left: &dyn ExecutionPlan, right: &dyn ExecutionPlan) - // Get the left and right table's total bytes // If both the left and right tables contain total_byte_size statistics, // use `total_byte_size` to determine `should_swap_join_order`, else use `num_rows` - let (left_size, right_size) = match ( - left.statistics().total_byte_size, - right.statistics().total_byte_size, - ) { - (Some(l), Some(r)) => (Some(l), Some(r)), - _ => (left.statistics().num_rows, right.statistics().num_rows), + let (left_stats, right_stats) = match (left.statistics(), right.statistics()) { + (Ok(left), Ok(right)) => (left, right), + _ => return false, }; + let (left_size, right_size) = + match (left_stats.total_byte_size, right_stats.total_byte_size) { + (Some(l), Some(r)) => (Some(l), Some(r)), + _ => (left_stats.num_rows, right_stats.num_rows), + }; match (left_size, right_size) { (Some(l), Some(r)) => l > r, @@ -84,9 +86,14 @@ fn supports_collect_by_size( ) -> bool { // Currently we do not trust the 0 value from stats, due to stats collection might have bug // TODO check the logic in datasource::get_statistics_with_limit() - if let Some(size) = plan.statistics().total_byte_size { + let stats = if let Ok(stats) = plan.statistics() { + stats + } else { + return false; + }; + if let Some(size) = stats.total_byte_size { size != 0 && size < collection_size_threshold - } else if let Some(row_count) = plan.statistics().num_rows { + } else if let Some(row_count) = stats.num_rows { row_count != 0 && row_count < collection_size_threshold } else { false @@ -734,9 +741,12 @@ mod tests_statistical { .downcast_ref::() .expect("The type of the plan should not be changed"); - assert_eq!(swapped_join.left().statistics().total_byte_size, Some(10)); assert_eq!( - swapped_join.right().statistics().total_byte_size, + swapped_join.left().statistics().unwrap().total_byte_size, + Some(10) + ); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, Some(100000) ); } @@ -783,10 +793,13 @@ mod tests_statistical { .expect("The type of the plan should not be changed"); assert_eq!( - swapped_join.left().statistics().total_byte_size, + swapped_join.left().statistics().unwrap().total_byte_size, Some(100000) ); - assert_eq!(swapped_join.right().statistics().total_byte_size, Some(10)); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, + Some(10) + ); } #[tokio::test] @@ -824,9 +837,12 @@ mod tests_statistical { assert_eq!(swapped_join.schema().fields().len(), 1); - assert_eq!(swapped_join.left().statistics().total_byte_size, Some(10)); assert_eq!( - swapped_join.right().statistics().total_byte_size, + swapped_join.left().statistics().unwrap().total_byte_size, + Some(10) + ); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, Some(100000) ); @@ -936,9 +952,12 @@ mod tests_statistical { .downcast_ref::() .expect("The type of the plan should not be changed"); - assert_eq!(swapped_join.left().statistics().total_byte_size, Some(10)); assert_eq!( - swapped_join.right().statistics().total_byte_size, + swapped_join.left().statistics().unwrap().total_byte_size, + Some(10) + ); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, Some(100000) ); } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 2328ffce235d..bd7d1d8acfc9 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2672,7 +2672,7 @@ mod tests { unimplemented!("NoOpExecutionPlan::execute"); } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { unimplemented!("NoOpExecutionPlan::statistics"); } } diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 16d0ad6e0707..53a40dc00df3 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -375,8 +375,8 @@ impl ExecutionPlan for StatisticsExec { unimplemented!("This plan only serves for testing statistics") } - fn statistics(&self) -> Statistics { - self.stats.clone() + fn statistics(&self) -> Result { + Ok(self.stats.clone()) } } diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index eb381c8d79cf..54d9332b5bbc 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -238,8 +238,8 @@ impl ExecutionPlan for UnboundedExec { })) } - fn statistics(&self) -> Statistics { - Statistics::new_with_unbounded_columns(&self.schema()) + fn statistics(&self) -> Result { + Ok(Statistics::new_with_unbounded_columns(&self.schema())) } } diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index 771da80aa6e7..bc20d0ea092a 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -153,9 +153,9 @@ impl ExecutionPlan for CustomExecutionPlan { Ok(Box::pin(TestCustomRecordBatchStream { nb_batch: 1 })) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { let batch = TEST_CUSTOM_RECORD_BATCH!().unwrap(); - Statistics { + Ok(Statistics { is_exact: true, num_rows: Some(batch.num_rows()), total_byte_size: None, @@ -176,7 +176,7 @@ impl ExecutionPlan for CustomExecutionPlan { }) .collect(), ), - } + }) } } diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index 095a82932b89..c879dd630cec 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -112,10 +112,10 @@ impl ExecutionPlan for CustomPlan { ))) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // here we could provide more accurate statistics // but we want to test the filter pushdown not the CBOs - Statistics::new_with_unbounded_columns(&self.schema()) + Ok(Statistics::new_with_unbounded_columns(&self.schema())) } } diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index 0a1c18f842c2..6d1aaeef5678 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -166,8 +166,8 @@ impl ExecutionPlan for StatisticsValidation { unimplemented!("This plan only serves for testing statistics") } - fn statistics(&self) -> Statistics { - self.stats.clone() + fn statistics(&self) -> Result { + Ok(self.stats.clone()) } } @@ -216,7 +216,7 @@ async fn sql_basic() -> Result<()> { let physical_plan = df.create_physical_plan().await.unwrap(); // the statistics should be those of the source - assert_eq!(stats, physical_plan.statistics()); + assert_eq!(stats, physical_plan.statistics()?); Ok(()) } @@ -233,7 +233,7 @@ async fn sql_filter() -> Result<()> { let physical_plan = df.create_physical_plan().await.unwrap(); - let stats = physical_plan.statistics(); + let stats = physical_plan.statistics()?; assert!(!stats.is_exact); assert_eq!(stats.num_rows, Some(1)); @@ -257,7 +257,7 @@ async fn sql_limit() -> Result<()> { column_statistics: Some(col_stats), total_byte_size: None }, - physical_plan.statistics() + physical_plan.statistics()? ); let df = ctx @@ -266,7 +266,7 @@ async fn sql_limit() -> Result<()> { .unwrap(); let physical_plan = df.create_physical_plan().await.unwrap(); // when the limit is larger than the original number of lines, statistics remain unchanged - assert_eq!(stats, physical_plan.statistics()); + assert_eq!(stats, physical_plan.statistics()?); Ok(()) } @@ -283,7 +283,7 @@ async fn sql_window() -> Result<()> { let physical_plan = df.create_physical_plan().await.unwrap(); - let result = physical_plan.statistics(); + let result = physical_plan.statistics()?; assert_eq!(stats.num_rows, result.num_rows); assert!(result.column_statistics.is_some()); diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index 90abbe9e2128..7f01fcafb21b 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -49,8 +49,8 @@ async fn load_table_stats_with_session_level_cache() { assert_eq!(get_cache_size(&state1), 0); let exec1 = table1.scan(&state1, None, &[], None).await.unwrap(); - assert_eq!(exec1.statistics().num_rows, Some(8)); - assert_eq!(exec1.statistics().total_byte_size, Some(671)); + assert_eq!(exec1.statistics().unwrap().num_rows, Some(8)); + assert_eq!(exec1.statistics().unwrap().total_byte_size, Some(671)); assert_eq!(get_cache_size(&state1), 1); //Session 2 first time list files @@ -65,16 +65,16 @@ async fn load_table_stats_with_session_level_cache() { 0 ); let exec2 = table2.scan(&state2, None, &[], None).await.unwrap(); - assert_eq!(exec2.statistics().num_rows, Some(8)); - assert_eq!(exec2.statistics().total_byte_size, Some(671)); + assert_eq!(exec2.statistics().unwrap().num_rows, Some(8)); + assert_eq!(exec2.statistics().unwrap().total_byte_size, Some(671)); assert_eq!(get_cache_size(&state2), 1); //Session 1 second time list files //check session 1 cache result not show in session 2 assert_eq!(get_cache_size(&state1), 1); let exec3 = table1.scan(&state1, None, &[], None).await.unwrap(); - assert_eq!(exec3.statistics().num_rows, Some(8)); - assert_eq!(exec3.statistics().total_byte_size, Some(671)); + assert_eq!(exec3.statistics().unwrap().num_rows, Some(8)); + assert_eq!(exec3.statistics().unwrap().total_byte_size, Some(671)); // List same file no increase assert_eq!(get_cache_size(&state1), 1); } diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index bd50a41dcc23..a0f689968ced 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -462,7 +462,7 @@ async fn parquet_statistics() -> Result<()> { assert_eq!(schema.fields().len(), 4); let stat_cols = physical_plan - .statistics() + .statistics()? .column_statistics .expect("col stats should be defined"); assert_eq!(stat_cols.len(), 4); @@ -489,7 +489,7 @@ async fn parquet_statistics() -> Result<()> { assert_eq!(schema.fields().len(), 2); let stat_cols = physical_plan - .statistics() + .statistics()? .column_statistics .expect("col stats should be defined"); assert_eq!(stat_cols.len(), 2); diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index 61aa2a2645d2..1029450e5bf5 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -490,10 +490,10 @@ impl ExecutionPlan for TopKExec { })) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // to improve the optimizability of this plan // better statistics inference could be provided - Statistics::new_with_unbounded_columns(&self.schema()) + Ok(Statistics::new_with_unbounded_columns(&self.schema())) } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2cedf3923790..0c1739979f59 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -30,9 +30,7 @@ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::utils::longest_consecutive_prefix; -use datafusion_common::{ - not_impl_err, plan_err, ColumnStatistics, DataFusionError, Result, -}; +use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ @@ -1002,7 +1000,7 @@ impl ExecutionPlan for AggregateExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // TODO stats: group expressions: // - once expressions will be able to compute their own stats, use it here // - case where we group by on a column for which with have the `distinct` stat @@ -1013,20 +1011,20 @@ impl ExecutionPlan for AggregateExec { AggregateMode::Final | AggregateMode::FinalPartitioned if self.group_by.expr.is_empty() => { - Statistics { + Ok(Statistics { num_rows: Some(1), is_exact: true, column_statistics: Some(column_statistics), total_byte_size: None, - } + }) } - _ => Statistics { + _ => Ok(Statistics { // the output row count is surely not larger than its input row count - num_rows: self.input.statistics().num_rows, + num_rows: self.input.statistics()?.num_rows, is_exact: false, column_statistics: Some(column_statistics), total_byte_size: None, - }, + }), } } } @@ -1826,9 +1824,13 @@ mod tests { Ok(Box::pin(stream)) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { let (_, batches) = some_data(); - common::compute_record_batch_statistics(&[batches], &self.schema(), None) + Ok(common::compute_record_batch_statistics( + &[batches], + &self.schema(), + None, + )) } } diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index 9d7df458d157..5f856f920633 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -181,9 +181,9 @@ impl ExecutionPlan for AnalyzeExec { ))) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // Statistics an an ANALYZE plan are not relevant - Statistics::new_with_unbounded_columns(&self.schema()) + Ok(Statistics::new_with_unbounded_columns(&self.schema())) } } diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index f46a228064fe..c80b743cf917 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -174,7 +174,7 @@ impl ExecutionPlan for CoalesceBatchesExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 8eddf57ae551..daef6d146d3c 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -159,7 +159,7 @@ impl ExecutionPlan for CoalescePartitionsExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index e4a4e113eb07..2cf515925dcc 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -261,8 +261,8 @@ impl<'a, 'b> ExecutionPlanVisitor for IndentVisitor<'a, 'b> { } } } - if self.show_statistics { - write!(self.f, ", statistics=[{}]", plan.statistics())?; + if self.show_statistics && plan.statistics().is_ok() { + write!(self.f, ", statistics=[{}]", plan.statistics().unwrap())?; } writeln!(self.f)?; self.indent += 1; @@ -341,8 +341,8 @@ impl ExecutionPlanVisitor for GraphvizVisitor<'_, '_> { } }; - let statistics = if self.show_statistics { - format!("statistics=[{}]", plan.statistics()) + let statistics = if self.show_statistics && plan.statistics().is_ok() { + format!("statistics=[{}]", plan.statistics().unwrap()) } else { "".to_string() }; diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index 675dac9ad265..82314905b588 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -161,11 +161,15 @@ impl ExecutionPlan for EmptyExec { )?)) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { let batch = self .data() .expect("Create empty RecordBatch should not fail"); - common::compute_record_batch_statistics(&[batch], &self.schema, None) + Ok(common::compute_record_batch_statistics( + &[batch], + &self.schema, + None, + )) } } diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index 91a0bb8f9948..7dffc8ec668e 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -169,9 +169,9 @@ impl ExecutionPlan for ExplainExec { ))) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // Statistics an EXPLAIN plan are not relevant - Statistics::new_with_unbounded_columns(&self.schema()) + Ok(Statistics::new_with_unbounded_columns(&self.schema())) } } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 421cd5e23e03..df25009daa50 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -154,7 +154,9 @@ impl ExecutionPlan for FilterExec { } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let stats = self.statistics(); + let stats = self + .statistics() + .expect("Ordering equivalences need to handle the error case of statistics"); // Add the columns that have only one value (singleton) after filtering to constants. if let Some(col_stats) = stats.column_statistics { let constants = collect_columns(self.predicate()) @@ -200,18 +202,18 @@ impl ExecutionPlan for FilterExec { /// The output statistics of a filtering operation can be estimated if the /// predicate's selectivity value can be determined for the incoming data. - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { let predicate = self.predicate(); if check_support(predicate, self.schema()) { - let input_stats = self.input.statistics(); + let input_stats = self.input.statistics()?; if let Some(column_stats) = input_stats.column_statistics { let num_rows = input_stats.num_rows; let total_byte_size = input_stats.total_byte_size; let input_analysis_ctx = AnalysisContext::from_statistics(&self.input.schema(), &column_stats); - let analysis_ctx = analyze(predicate, input_analysis_ctx).unwrap(); + let analysis_ctx = analyze(predicate, input_analysis_ctx)?; let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); let num_rows = @@ -222,23 +224,23 @@ impl ExecutionPlan for FilterExec { return if let Some(analysis_boundaries) = analysis_ctx.boundaries { let column_statistics = collect_new_statistics(&column_stats, analysis_boundaries); - Statistics { + Ok(Statistics { num_rows, total_byte_size, column_statistics: Some(column_statistics), is_exact: false, - } + }) } else { - Statistics { + Ok(Statistics { num_rows, total_byte_size, column_statistics: Some(column_stats.to_vec()), is_exact: false, - } + }) }; } } - Statistics::new_with_unbounded_columns(&self.schema()) + Ok(Statistics::new_with_unbounded_columns(&self.schema())) } } @@ -467,7 +469,7 @@ mod tests { let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics(); + let statistics = filter.statistics()?; assert_eq!(statistics.num_rows, Some(25)); assert_eq!(statistics.total_byte_size, Some(25 * bytes_per_row)); assert_eq!( @@ -515,7 +517,7 @@ mod tests { sub_filter, )?); - let statistics = filter.statistics(); + let statistics = filter.statistics()?; assert_eq!(statistics.num_rows, Some(16)); assert_eq!( statistics.column_statistics, @@ -576,7 +578,7 @@ mod tests { binary(col("a", &schema)?, Operator::GtEq, lit(10i32), &schema)?, b_gt_5, )?); - let statistics = filter.statistics(); + let statistics = filter.statistics()?; // On a uniform distribution, only fifteen rows will satisfy the // filter that 'a' proposed (a >= 10 AND a <= 25) (15/100) and only // 5 rows will satisfy the filter that 'b' proposed (b > 45) (5/50). @@ -621,7 +623,7 @@ mod tests { let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics(); + let statistics = filter.statistics()?; assert_eq!(statistics.num_rows, None); Ok(()) @@ -695,7 +697,7 @@ mod tests { )); let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics(); + let statistics = filter.statistics()?; // 0.5 (from a) * 0.333333... (from b) * 0.798387... (from c) ≈ 0.1330... // num_rows after ceil => 133.0... => 134 // total_byte_size after ceil => 532.0... => 533 @@ -788,10 +790,10 @@ mod tests { )), )); // Since filter predicate passes all entries, statistics after filter shouldn't change. - let expected = input.statistics().column_statistics; + let expected = input.statistics()?.column_statistics; let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics(); + let statistics = filter.statistics()?; assert_eq!(statistics.num_rows, Some(1000)); assert_eq!(statistics.total_byte_size, Some(4000)); @@ -845,7 +847,7 @@ mod tests { )); let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics(); + let statistics = filter.statistics()?; assert_eq!(statistics.num_rows, Some(0)); assert_eq!(statistics.total_byte_size, Some(0)); @@ -902,7 +904,7 @@ mod tests { )); let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let statistics = filter.statistics(); + let statistics = filter.statistics()?; assert_eq!(statistics.num_rows, Some(490)); assert_eq!(statistics.total_byte_size, Some(1960)); @@ -952,7 +954,7 @@ mod tests { )); let filter: Arc = Arc::new(FilterExec::try_new(predicate, input)?); - let filter_statistics = filter.statistics(); + let filter_statistics = filter.statistics()?; let expected_filter_statistics = Statistics { num_rows: None, diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index c5fec688dc16..5c1e337beadf 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -250,8 +250,8 @@ impl ExecutionPlan for FileSinkExec { ))) } - fn statistics(&self) -> Statistics { - Statistics::new_with_unbounded_columns(&self.schema()) + fn statistics(&self) -> Result { + Ok(Statistics::new_with_unbounded_columns(&self.schema())) } } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 4171677325a6..4ab06d3323da 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -257,13 +257,13 @@ impl ExecutionPlan for CrossJoinExec { })) } - fn statistics(&self) -> Statistics { - stats_cartesian_product( - self.left.statistics(), + fn statistics(&self) -> Result { + Ok(stats_cartesian_product( + self.left.statistics()?, self.left.schema(), - self.right.statistics(), + self.right.statistics()?, self.right.schema(), - ) + )) } } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 8e204634f3d9..edb31821af1d 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -493,7 +493,7 @@ impl ExecutionPlan for HashJoinExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // TODO stats: it is not possible in general to know the output size of joins // There are some special cases though, for example: // - `A LEFT JOIN B ON A.col=B.col` with `COUNT_DISTINCT(B.col)=COUNT(B.col)` diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index c49c16dba313..59429186d606 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -282,7 +282,7 @@ impl ExecutionPlan for NestedLoopJoinExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { estimate_join_statistics( self.left.clone(), self.right.clone(), diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 4de723ab73ea..85a761869800 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -381,7 +381,7 @@ impl ExecutionPlan for SortMergeJoinExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // TODO stats: it is not possible in general to know the output size of joins // There are some special cases though, for example: // - `A LEFT JOIN B ON A.col=B.col` with `COUNT_DISTINCT(B.col)=COUNT(B.col)` diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 651f0a3ae51d..365e4a19a926 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -477,9 +477,9 @@ impl ExecutionPlan for SymmetricHashJoinExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { // TODO stats: it is not possible in general to know the output size of joins - Statistics::new_with_unbounded_columns(&self.schema()) + Ok(Statistics::new_with_unbounded_columns(&self.schema())) } fn execute( diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index ea72bb80b993..b39a4ad36dcd 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -724,21 +724,21 @@ pub(crate) fn estimate_join_statistics( right: Arc, on: JoinOn, join_type: &JoinType, -) -> Statistics { - let left_stats = left.statistics(); - let right_stats = right.statistics(); +) -> Result { + let left_stats = left.statistics()?; + let right_stats = right.statistics()?; let join_stats = estimate_join_cardinality(join_type, left_stats, right_stats, &on); let (num_rows, column_statistics) = match join_stats { Some(stats) => (Some(stats.num_rows), Some(stats.column_statistics)), None => (None, None), }; - Statistics { + Ok(Statistics { num_rows, total_byte_size: None, column_statistics, is_exact: false, - } + }) } // Estimate the cardinality for the given join with input statistics. diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 9c116e73ead7..f4685c056687 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -187,7 +187,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { } /// Returns the global output statistics for this `ExecutionPlan` node. - fn statistics(&self) -> Statistics; + fn statistics(&self) -> Result; } /// Indicate whether a data exchange is needed for the input of `plan`, which will be very helpful diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 718c198d2acb..de85bdb32d31 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -187,8 +187,8 @@ impl ExecutionPlan for GlobalLimitExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - let input_stats = self.input.statistics(); + fn statistics(&self) -> Result { + let input_stats = self.input.statistics()?; let skip = self.skip; // the maximum row number needs to be fetched let max_row_num = self @@ -207,7 +207,7 @@ impl ExecutionPlan for GlobalLimitExec { .iter() .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) .collect::>(); - match input_stats { + let stats = match input_stats { Statistics { num_rows: Some(nr), .. } => { @@ -240,7 +240,8 @@ impl ExecutionPlan for GlobalLimitExec { column_statistics: Some(col_stats), total_byte_size: None, }, - } + }; + Ok(stats) } } @@ -362,15 +363,15 @@ impl ExecutionPlan for LocalLimitExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - let input_stats = self.input.statistics(); + fn statistics(&self) -> Result { + let input_stats = self.input.statistics()?; let col_stats = self .schema() .fields() .iter() .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) .collect::>(); - match input_stats { + let stats = match input_stats { // if the input does not reach the limit globally, return input stats Statistics { num_rows: Some(nr), .. @@ -395,7 +396,8 @@ impl ExecutionPlan for LocalLimitExec { column_statistics: Some(col_stats), total_byte_size: None, }, - } + }; + Ok(stats) } } @@ -765,7 +767,7 @@ mod tests { let offset = GlobalLimitExec::new(Arc::new(CoalescePartitionsExec::new(csv)), skip, fetch); - Ok(offset.statistics().num_rows) + Ok(offset.statistics()?.num_rows) } async fn row_number_statistics_for_local_limit( @@ -778,7 +780,7 @@ mod tests { let offset = LocalLimitExec::new(csv, fetch); - Ok(offset.statistics().num_rows) + Ok(offset.statistics()?.num_rows) } /// Return a RecordBatch with a single array with row_count sz diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index b29c8e9c7bd9..2e1ee9a76694 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -144,12 +144,12 @@ impl ExecutionPlan for MemoryExec { } /// We recompute the statistics dynamically from the arrow metadata as it is pretty cheap to do so - fn statistics(&self) -> Statistics { - common::compute_record_batch_statistics( + fn statistics(&self) -> Result { + Ok(common::compute_record_batch_statistics( &self.partitions, &self.schema, self.projection.clone(), - ) + )) } } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index c899f9812d3b..ebed44977f33 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -332,12 +332,12 @@ impl ExecutionPlan for ProjectionExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - stats_projection( - self.input.statistics(), + fn statistics(&self) -> Result { + Ok(stats_projection( + self.input.statistics()?, self.expr.iter().map(|(e, _)| Arc::clone(e)), self.schema.clone(), - ) + )) } } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 14b54dc0614d..b0c60ee1a475 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -610,7 +610,7 @@ impl ExecutionPlan for RepartitionExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 7d260d42d9cd..7df0d07be392 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -882,7 +882,7 @@ impl ExecutionPlan for SortExec { Some(self.metrics_set.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 5b485e0b68e4..a72277367e3d 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -261,7 +261,7 @@ impl ExecutionPlan for SortPreservingMergeExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 19516061317b..3c0422065cde 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -185,7 +185,7 @@ impl ExecutionPlan for StreamingTableExec { }) } - fn statistics(&self) -> Statistics { - Statistics::new_with_unbounded_columns(&self.schema()) + fn statistics(&self) -> Result { + Ok(Statistics::new_with_unbounded_columns(&self.schema())) } } diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index 05a6ae6191f2..a25e21b59254 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -239,7 +239,7 @@ impl ExecutionPlan for MockExec { } // Panics if one of the batches is an error - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { let data: Result> = self .data .iter() @@ -249,9 +249,13 @@ impl ExecutionPlan for MockExec { }) .collect(); - let data = data.unwrap(); + let data = data?; - common::compute_record_batch_statistics(&[data], &self.schema, None) + Ok(common::compute_record_batch_statistics( + &[data], + &self.schema, + None, + )) } } @@ -369,8 +373,12 @@ impl ExecutionPlan for BarrierExec { Ok(builder.build()) } - fn statistics(&self) -> Statistics { - common::compute_record_batch_statistics(&self.data, &self.schema, None) + fn statistics(&self) -> Result { + Ok(common::compute_record_batch_statistics( + &self.data, + &self.schema, + None, + )) } } @@ -448,8 +456,8 @@ impl ExecutionPlan for ErrorExec { internal_err!("ErrorExec, unsurprisingly, errored in partition {partition}") } - fn statistics(&self) -> Statistics { - Statistics::new_with_unbounded_columns(&self.schema()) + fn statistics(&self) -> Result { + Ok(Statistics::new_with_unbounded_columns(&self.schema())) } } @@ -531,8 +539,8 @@ impl ExecutionPlan for StatisticsExec { unimplemented!("This plan only serves for testing statistics") } - fn statistics(&self) -> Statistics { - self.stats.clone() + fn statistics(&self) -> Result { + Ok(self.stats.clone()) } } @@ -625,7 +633,7 @@ impl ExecutionPlan for BlockingExec { })) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { unimplemented!() } } @@ -762,7 +770,7 @@ impl ExecutionPlan for PanicExec { })) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { unimplemented!() } } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 84de40bdca57..7f2f7b9a1809 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -264,12 +264,17 @@ impl ExecutionPlan for UnionExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - self.inputs + fn statistics(&self) -> Result { + let stats = self + .inputs .iter() - .map(|ep| ep.statistics()) + .map(|stat| stat.statistics()) + .collect::>>()?; + + Ok(stats + .into_iter() .reduce(stats_union) - .unwrap_or(Statistics::new_with_unbounded_columns(&self.schema())) + .unwrap_or_else(|| Statistics::new_with_unbounded_columns(&self.schema()))) } fn benefits_from_input_partitioning(&self) -> Vec { @@ -438,12 +443,17 @@ impl ExecutionPlan for InterleaveExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - self.inputs + fn statistics(&self) -> Result { + let stats = self + .inputs .iter() - .map(|ep| ep.statistics()) + .map(|stat| stat.statistics()) + .collect::>>()?; + + Ok(stats + .into_iter() .reduce(stats_union) - .unwrap_or(Statistics::new_with_unbounded_columns(&self.schema())) + .unwrap_or_else(|| Statistics::new_with_unbounded_columns(&self.schema()))) } fn benefits_from_input_partitioning(&self) -> Vec { diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 1a011a1141af..5c6f1b5b1463 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -160,8 +160,8 @@ impl ExecutionPlan for UnnestExec { })) } - fn statistics(&self) -> Statistics { - Statistics::new_with_unbounded_columns(&self.schema()) + fn statistics(&self) -> Result { + Ok(Statistics::new_with_unbounded_columns(&self.schema())) } } diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index 2cf341d1fe60..e6c71bef76b3 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -186,9 +186,13 @@ impl ExecutionPlan for ValuesExec { )?)) } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { let batch = self.data(); - common::compute_record_batch_statistics(&[batch], &self.schema, None) + Ok(common::compute_record_batch_statistics( + &[batch], + &self.schema, + None, + )) } } 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 c4b2ee222cd6..1648926d7e98 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -330,8 +330,8 @@ impl ExecutionPlan for BoundedWindowAggExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - let input_stat = self.input.statistics(); + fn statistics(&self) -> Result { + let input_stat = self.input.statistics()?; let win_cols = self.window_expr.len(); let input_cols = self.input_schema.fields().len(); // TODO stats: some windowing function will maintain invariants such as min, max... @@ -350,12 +350,12 @@ impl ExecutionPlan for BoundedWindowAggExec { self.schema().field(index + input_cols).data_type(), )) } - Statistics { + Ok(Statistics { is_exact: input_stat.is_exact, num_rows: input_stat.num_rows, column_statistics: Some(column_statistics), total_byte_size: None, - } + }) } } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 023a1dfa90a9..67f4cfd34623 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -254,8 +254,8 @@ impl ExecutionPlan for WindowAggExec { Some(self.metrics.clone_inner()) } - fn statistics(&self) -> Statistics { - let input_stat = self.input.statistics(); + fn statistics(&self) -> Result { + let input_stat = self.input.statistics()?; let win_cols = self.window_expr.len(); let input_cols = self.input_schema.fields().len(); // TODO stats: some windowing function will maintain invariants such as min, max... @@ -274,12 +274,12 @@ impl ExecutionPlan for WindowAggExec { self.schema().field(index + input_cols).data_type(), )) } - Statistics { + Ok(Statistics { is_exact: input_stat.is_exact, num_rows: input_stat.num_rows, column_statistics: Some(column_statistics), total_byte_size: None, - } + }) } } From f2e99f95a68fd7dd4c9fb199b03039e2f54a8294 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 21 Sep 2023 14:26:24 +0300 Subject: [PATCH 13/53] fix after merge --- datafusion-examples/examples/json_opener.rs | 2 +- datafusion/core/src/datasource/listing/table.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion-examples/examples/json_opener.rs b/datafusion-examples/examples/json_opener.rs index ce03ddf0efaf..ee5bd6b50faf 100644 --- a/datafusion-examples/examples/json_opener.rs +++ b/datafusion-examples/examples/json_opener.rs @@ -29,7 +29,7 @@ use datafusion::{ error::Result, physical_plan::metrics::ExecutionPlanMetricsSet, }; -use datafusion_common::{FileCompressionType, Statistics}; +use datafusion_common::Statistics; use futures::StreamExt; use object_store::ObjectStore; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index bd2c475da21c..7ab911ab995e 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -943,7 +943,7 @@ impl ListingTable { let files = file_list .map(|part_file| async { let part_file = part_file?; - let mut statistics_result = if self.options.collect_stat { + let statistics_result = if self.options.collect_stat { let statistics_cache = self.collected_statistics.clone(); match statistics_cache.get_with_extra( &part_file.object_meta.location, From 571654ffdd52d75b6fd74a7d944f225abf12477e Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 22 Sep 2023 15:22:10 +0300 Subject: [PATCH 14/53] Simplifications --- .../aggregate_statistics.rs | 17 ++-- .../src/physical_optimizer/join_selection.rs | 36 ++++----- .../physical-expr/src/intervals/utils.rs | 2 +- datafusion/physical-plan/src/display.rs | 10 ++- datafusion/physical-plan/src/filter.rs | 73 +++++++++-------- .../physical-plan/src/joins/cross_join.rs | 81 +++++++------------ datafusion/physical-plan/src/limit.rs | 16 +--- 7 files changed, 104 insertions(+), 131 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index ce68f4f39c21..02337c7f7654 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -53,7 +53,7 @@ impl PhysicalOptimizerRule for AggregateStatistics { plan: Arc, _config: &ConfigOptions, ) -> Result> { - if let Some(partial_agg_exec) = take_optimizable(&*plan) { + if let Some(partial_agg_exec) = take_optimizable(&*plan)? { let partial_agg_exec = partial_agg_exec .as_any() .downcast_ref::() @@ -111,7 +111,7 @@ impl PhysicalOptimizerRule for AggregateStatistics { /// If this is the case, return a ref to the partial `AggregateExec`, else `None`. /// We would have preferred to return a casted ref to AggregateExec but the recursion requires /// the `ExecutionPlan.children()` method that returns an owned reference. -fn take_optimizable(node: &dyn ExecutionPlan) -> Option> { +fn take_optimizable(node: &dyn ExecutionPlan) -> Result>> { if let Some(final_agg_exec) = node.as_any().downcast_ref::() { if final_agg_exec.mode() == &AggregateMode::Final && final_agg_exec.group_expr().is_empty() @@ -125,11 +125,9 @@ fn take_optimizable(node: &dyn ExecutionPlan) -> Option> && partial_agg_exec.group_expr().is_empty() && partial_agg_exec.filter_expr().iter().all(|e| e.is_none()) { - let stats = partial_agg_exec.input().statistics(); - if let Ok(stats) = stats { - if stats.is_exact { - return Some(child); - } + let stats = partial_agg_exec.input().statistics()?; + if stats.is_exact { + return Ok(Some(child)); } } } @@ -141,7 +139,7 @@ fn take_optimizable(node: &dyn ExecutionPlan) -> Option> } } } - None + Ok(None) } /// If this agg_expr is a count that is defined in the statistics, return it @@ -224,6 +222,9 @@ fn take_optimizable_min( } = &col_stats[col_expr.index()] { // Exclude the unbounded case + // As safest estimate, -inf, and + inf is used as bound of the column + // If minimum is -inf, it is not exact. Hence we shouldn't do optimization + // based on this statistic if !val.is_null() { return Some((val.clone(), casted_expr.name().to_string())); } diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index d04d3253f3a8..7081f0c7be53 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -60,23 +60,23 @@ impl JoinSelection { // TODO: We need some performance test for Right Semi/Right Join swap to Left Semi/Left Join in case that the right side is smaller but not much smaller. // TODO: In PrestoSQL, the optimizer flips join sides only if one side is much smaller than the other by more than SIZE_DIFFERENCE_THRESHOLD times, by default is is 8 times. /// Checks statistics for join swap. -fn should_swap_join_order(left: &dyn ExecutionPlan, right: &dyn ExecutionPlan) -> bool { +fn should_swap_join_order( + left: &dyn ExecutionPlan, + right: &dyn ExecutionPlan, +) -> Result { // Get the left and right table's total bytes // If both the left and right tables contain total_byte_size statistics, // use `total_byte_size` to determine `should_swap_join_order`, else use `num_rows` - let (left_stats, right_stats) = match (left.statistics(), right.statistics()) { - (Ok(left), Ok(right)) => (left, right), - _ => return false, - }; - let (left_size, right_size) = - match (left_stats.total_byte_size, right_stats.total_byte_size) { - (Some(l), Some(r)) => (Some(l), Some(r)), - _ => (left_stats.num_rows, right_stats.num_rows), - }; - - match (left_size, right_size) { - (Some(l), Some(r)) => l > r, - _ => false, + let left_stats = left.statistics()?; + let right_stats = right.statistics()?; + // First compare `total_byte_size` of left and right side, + // if information in this field is insufficient fallback to the `num_rows` + match (left_stats.total_byte_size, right_stats.total_byte_size) { + (Some(l), Some(r)) => Ok(l > r), + _ => match (left_stats.num_rows, right_stats.num_rows) { + (Some(l), Some(r)) => Ok(l > r), + _ => Ok(false), + }, } } @@ -301,7 +301,7 @@ fn try_collect_left( }; match (left_can_collect, right_can_collect) { (true, true) => { - if should_swap_join_order(&**left, &**right) + if should_swap_join_order(&**left, &**right)? && supports_swap(*hash_join.join_type()) { Ok(Some(swap_hash_join(hash_join, PartitionMode::CollectLeft)?)) @@ -340,7 +340,7 @@ fn try_collect_left( fn partitioned_hash_join(hash_join: &HashJoinExec) -> Result> { let left = hash_join.left(); let right = hash_join.right(); - if should_swap_join_order(&**left, &**right) && supports_swap(*hash_join.join_type()) + if should_swap_join_order(&**left, &**right)? && supports_swap(*hash_join.join_type()) { swap_hash_join(hash_join, PartitionMode::Partitioned) } else { @@ -380,7 +380,7 @@ fn statistical_join_selection_subrule( PartitionMode::Partitioned => { let left = hash_join.left(); let right = hash_join.right(); - if should_swap_join_order(&**left, &**right) + if should_swap_join_order(&**left, &**right)? && supports_swap(*hash_join.join_type()) { swap_hash_join(hash_join, PartitionMode::Partitioned).map(Some)? @@ -392,7 +392,7 @@ fn statistical_join_selection_subrule( } else if let Some(cross_join) = plan.as_any().downcast_ref::() { let left = cross_join.left(); let right = cross_join.right(); - if should_swap_join_order(&**left, &**right) { + if should_swap_join_order(&**left, &**right)? { let new_join = CrossJoinExec::new(Arc::clone(right), Arc::clone(left)); // TODO avoid adding ProjectionExec again and again, only adding Final Projection let proj: Arc = Arc::new(ProjectionExec::try_new( diff --git a/datafusion/physical-expr/src/intervals/utils.rs b/datafusion/physical-expr/src/intervals/utils.rs index 131cc8e6faaa..6d8f8e407e5a 100644 --- a/datafusion/physical-expr/src/intervals/utils.rs +++ b/datafusion/physical-expr/src/intervals/utils.rs @@ -57,7 +57,7 @@ pub fn check_support(expr: &Arc, schema: SchemaRef) -> bool { return false; } } else if let Some(cast) = expr_any.downcast_ref::() { - check_support(&cast.children()[0], schema) + check_support(cast.expr(), schema) } else { false } diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index 2cf515925dcc..010e19f59990 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -261,8 +261,9 @@ impl<'a, 'b> ExecutionPlanVisitor for IndentVisitor<'a, 'b> { } } } - if self.show_statistics && plan.statistics().is_ok() { - write!(self.f, ", statistics=[{}]", plan.statistics().unwrap())?; + let stats = plan.statistics().map_err(|_e| fmt::Error)?; + if self.show_statistics { + write!(self.f, ", statistics=[{}]", stats)?; } writeln!(self.f)?; self.indent += 1; @@ -341,8 +342,9 @@ impl ExecutionPlanVisitor for GraphvizVisitor<'_, '_> { } }; - let statistics = if self.show_statistics && plan.statistics().is_ok() { - format!("statistics=[{}]", plan.statistics().unwrap()) + let stats = plan.statistics().map_err(|_e| fmt::Error)?; + let statistics = if self.show_statistics { + format!("statistics=[{}]", stats) } else { "".to_string() }; diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index df25009daa50..5c77e51568be 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -205,42 +205,45 @@ impl ExecutionPlan for FilterExec { fn statistics(&self) -> Result { let predicate = self.predicate(); - if check_support(predicate, self.schema()) { - let input_stats = self.input.statistics()?; - - if let Some(column_stats) = input_stats.column_statistics { - let num_rows = input_stats.num_rows; - let total_byte_size = input_stats.total_byte_size; - let input_analysis_ctx = - AnalysisContext::from_statistics(&self.input.schema(), &column_stats); - let analysis_ctx = analyze(predicate, input_analysis_ctx)?; - - let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); - let num_rows = - num_rows.map(|num| (num as f64 * selectivity).ceil() as usize); - let total_byte_size = total_byte_size - .map(|size| (size as f64 * selectivity).ceil() as usize); - - return if let Some(analysis_boundaries) = analysis_ctx.boundaries { - let column_statistics = - collect_new_statistics(&column_stats, analysis_boundaries); - Ok(Statistics { - num_rows, - total_byte_size, - column_statistics: Some(column_statistics), - is_exact: false, - }) - } else { - Ok(Statistics { - num_rows, - total_byte_size, - column_statistics: Some(column_stats.to_vec()), - is_exact: false, - }) - }; - } + if !check_support(predicate, self.schema()) { + return Ok(Statistics::new_with_unbounded_columns(&self.schema())); + } + let input_stats = self.input.statistics()?; + + let column_stats = if let Some(column_stats) = input_stats.column_statistics { + column_stats + } else { + Statistics::unbounded_column_statistics(&self.input.schema()) + }; + + let num_rows = input_stats.num_rows; + let total_byte_size = input_stats.total_byte_size; + let input_analysis_ctx = + AnalysisContext::from_statistics(&self.input.schema(), &column_stats); + let analysis_ctx = analyze(predicate, input_analysis_ctx)?; + + let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); + let num_rows = num_rows.map(|num| (num as f64 * selectivity).ceil() as usize); + let total_byte_size = + total_byte_size.map(|size| (size as f64 * selectivity).ceil() as usize); + + if let Some(analysis_boundaries) = analysis_ctx.boundaries { + let column_statistics = + collect_new_statistics(&column_stats, analysis_boundaries); + Ok(Statistics { + num_rows, + total_byte_size, + column_statistics: Some(column_statistics), + is_exact: false, + }) + } else { + Ok(Statistics { + num_rows, + total_byte_size, + column_statistics: Some(column_stats.to_vec()), + is_exact: false, + }) } - Ok(Statistics::new_with_unbounded_columns(&self.schema())) } } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 4ab06d3323da..6dba4efa48ce 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -289,63 +289,40 @@ fn stats_cartesian_product( .zip(right_stats.total_byte_size) .map(|(a, b)| 2 * a * b); - let left_col_count = left_schema.fields().len(); - let right_col_count = right_schema.fields().len(); - // calculate column stats - let column_statistics = - // complete the column statistics if they are missing only on one side - match (left_stats.column_statistics, right_stats.column_statistics) { - (None, None) => { - let mut left_stats= vec![]; - let mut right_stats= vec![]; - for index in 0..left_col_count{ - left_stats.push(ColumnStatistics::new_with_unbounded_column(left_schema.field(index).data_type())); - } - for index in 0..right_col_count{ - right_stats.push(ColumnStatistics::new_with_unbounded_column(right_schema.field(index).data_type())); - } - Some((left_stats,right_stats)) - }, - (None, Some(right_col_stat)) => { - let mut left_stats= vec![]; - for index in 0..left_col_count{ - left_stats.push(ColumnStatistics::new_with_unbounded_column(left_schema.field(index).data_type())); - } - Some((left_stats, right_col_stat)) - }, - (Some(left_col_stat), None) => { - let mut right_stats= vec![]; - for index in 0..right_col_count{ - right_stats.push(ColumnStatistics::new_with_unbounded_column(right_schema.field(index).data_type())); - } - Some((left_col_stat, right_stats)) - }, - (Some(left_col_stat), Some(right_col_stat)) => { - Some((left_col_stat, right_col_stat)) - } - } - .map(|(left_col_stats, right_col_stats)| { - // the null counts must be multiplied by the row counts of the other side (if defined) - // Min, max and distinct_count on the other hand are invariants. - left_col_stats.into_iter().map(|s| ColumnStatistics{ - null_count: s.null_count.zip(right_row_count).map(|(a, b)| a * b), - distinct_count: s.distinct_count, - min_value: s.min_value, - max_value: s.max_value, - }).chain( - right_col_stats.into_iter().map(|s| ColumnStatistics{ - null_count: s.null_count.zip(left_row_count).map(|(a, b)| a * b), - distinct_count: s.distinct_count, - min_value: s.min_value, - max_value: s.max_value, - })).collect() - }); + let left_col_stats = if let Some(left_col_stats) = left_stats.column_statistics { + left_col_stats + } else { + Statistics::unbounded_column_statistics(&left_schema) + }; + let right_col_stats = if let Some(right_col_stats) = right_stats.column_statistics { + right_col_stats + } else { + Statistics::unbounded_column_statistics(&right_schema) + }; + + // the null counts must be multiplied by the row counts of the other side (if defined) + // Min, max and distinct_count on the other hand are invariants. + let cross_join_stats = left_col_stats + .into_iter() + .map(|s| ColumnStatistics { + null_count: s.null_count.zip(right_row_count).map(|(a, b)| a * b), + distinct_count: s.distinct_count, + min_value: s.min_value, + max_value: s.max_value, + }) + .chain(right_col_stats.into_iter().map(|s| ColumnStatistics { + null_count: s.null_count.zip(left_row_count).map(|(a, b)| a * b), + distinct_count: s.distinct_count, + min_value: s.min_value, + max_value: s.max_value, + })) + .collect(); Statistics { is_exact, num_rows, total_byte_size, - column_statistics, + column_statistics: Some(cross_join_stats), } } diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 00f6a3246e02..2fe2e678f60f 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -33,7 +33,7 @@ use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics} use arrow::array::ArrayRef; use arrow::datatypes::SchemaRef; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; -use datafusion_common::{internal_err, ColumnStatistics, DataFusionError, Result}; +use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::OrderingEquivalenceProperties; @@ -205,12 +205,7 @@ impl ExecutionPlan for GlobalLimitExec { } }) .unwrap_or(usize::MAX); - let col_stats = self - .schema() - .fields() - .iter() - .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) - .collect::>(); + let col_stats = Statistics::unbounded_column_statistics(&self.schema()); let stats = match input_stats { Statistics { num_rows: Some(nr), .. @@ -373,12 +368,7 @@ impl ExecutionPlan for LocalLimitExec { fn statistics(&self) -> Result { let input_stats = self.input.statistics()?; - let col_stats = self - .schema() - .fields() - .iter() - .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) - .collect::>(); + let col_stats = Statistics::unbounded_column_statistics(&self.schema()); let stats = match input_stats { // if the input does not reach the limit globally, return input stats Statistics { From 6ef4582d8799a31785ae78bea398ebdc643efd06 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 25 Sep 2023 18:01:49 +0300 Subject: [PATCH 15/53] Remove option from column stats --- datafusion/common/src/stats.rs | 4 +- .../src/datasource/file_format/parquet.rs | 25 ++--- .../physical_plan/file_scan_config.rs | 33 ++----- datafusion/core/src/datasource/statistics.rs | 54 +++++------ .../aggregate_statistics.rs | 16 ++-- .../src/physical_optimizer/join_selection.rs | 54 +++++------ datafusion/core/src/test/mod.rs | 8 +- datafusion/core/tests/custom_sources.rs | 33 ++++--- .../tests/custom_sources_cases/statistics.rs | 30 +++--- datafusion/core/tests/path_partition.rs | 10 +- .../physical-plan/src/aggregates/mod.rs | 4 +- datafusion/physical-plan/src/common.rs | 6 +- datafusion/physical-plan/src/filter.rs | 94 +++++++++---------- .../physical-plan/src/joins/cross_join.rs | 71 ++++---------- .../physical-plan/src/joins/hash_join.rs | 1 + .../src/joins/nested_loop_join.rs | 1 + .../src/joins/sort_merge_join.rs | 1 + datafusion/physical-plan/src/joins/utils.rs | 23 ++--- datafusion/physical-plan/src/limit.rs | 10 +- datafusion/physical-plan/src/projection.rs | 46 +++++---- datafusion/physical-plan/src/test/exec.rs | 7 +- datafusion/physical-plan/src/union.rs | 27 +++--- .../src/windows/bounded_window_agg_exec.rs | 13 +-- .../src/windows/window_agg_exec.rs | 16 ++-- .../proto/src/physical_plan/from_proto.rs | 6 +- datafusion/proto/src/physical_plan/mod.rs | 10 +- .../proto/src/physical_plan/to_proto.rs | 5 +- 27 files changed, 250 insertions(+), 358 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index a677ba6879c4..a129672737ba 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -33,7 +33,7 @@ pub struct Statistics { /// total bytes of the table rows pub total_byte_size: Option, /// Statistics on a column level - pub column_statistics: Option>, + pub column_statistics: Vec, /// If true, any field that is `Some(..)` is the actual value in the data provided by the operator (it is not /// an estimate). Any or all other fields might still be None, in which case no information is known. /// if false, any field that is `Some(..)` may contain an inexact estimate and may not be the actual value. @@ -48,7 +48,7 @@ impl Statistics { Self { num_rows: None, total_byte_size: None, - column_statistics: Some(Statistics::unbounded_column_statistics(schema)), + column_statistics: Statistics::unbounded_column_statistics(schema), is_exact: false, } } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 6fe0e0a8a49c..92fff6a425ca 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -555,14 +555,9 @@ async fn fetch_statistics( } let column_stats = if has_statistics { - Some(get_col_stats( - &table_schema, - null_counts, - &mut max_values, - &mut min_values, - )) + get_col_stats(&table_schema, null_counts, &mut max_values, &mut min_values) } else { - None + Statistics::unbounded_column_statistics(&table_schema) }; let statistics = Statistics { @@ -1166,15 +1161,15 @@ mod tests { fetch_statistics(store.as_ref(), schema.clone(), &meta[0], None).await?; assert_eq!(stats.num_rows, Some(3)); - let c1_stats = &stats.column_statistics.as_ref().expect("missing c1 stats")[0]; - let c2_stats = &stats.column_statistics.as_ref().expect("missing c2 stats")[1]; + let c1_stats = &stats.column_statistics[0]; + let c2_stats = &stats.column_statistics[1]; assert_eq!(c1_stats.null_count, Some(1)); assert_eq!(c2_stats.null_count, Some(3)); let stats = fetch_statistics(store.as_ref(), schema, &meta[1], None).await?; assert_eq!(stats.num_rows, Some(3)); - let c1_stats = &stats.column_statistics.as_ref().expect("missing c1 stats")[0]; - let c2_stats = &stats.column_statistics.as_ref().expect("missing c2 stats")[1]; + let c1_stats = &stats.column_statistics[0]; + let c2_stats = &stats.column_statistics[1]; assert_eq!(c1_stats.null_count, Some(3)); assert_eq!(c2_stats.null_count, Some(1)); assert_eq!(c2_stats.max_value, Some(ScalarValue::Int64(Some(2)))); @@ -1315,8 +1310,8 @@ mod tests { .await?; assert_eq!(stats.num_rows, Some(3)); - let c1_stats = &stats.column_statistics.as_ref().expect("missing c1 stats")[0]; - let c2_stats = &stats.column_statistics.as_ref().expect("missing c2 stats")[1]; + let c1_stats = &stats.column_statistics[0]; + let c2_stats = &stats.column_statistics[1]; assert_eq!(c1_stats.null_count, Some(1)); assert_eq!(c2_stats.null_count, Some(3)); @@ -1348,8 +1343,8 @@ mod tests { .await?; assert_eq!(stats.num_rows, Some(3)); - let c1_stats = &stats.column_statistics.as_ref().expect("missing c1 stats")[0]; - let c2_stats = &stats.column_statistics.as_ref().expect("missing c2 stats")[1]; + let c1_stats = &stats.column_statistics[0]; + let c2_stats = &stats.column_statistics[1]; assert_eq!(c1_stats.null_count, Some(1)); assert_eq!(c2_stats.null_count, Some(3)); 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 d82da71aa247..ff5980071b16 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -132,13 +132,7 @@ impl FileScanConfig { if idx < self.file_schema.fields().len() { let field = self.file_schema.field(idx).clone(); table_fields.push(field.clone()); - if let Some(file_cols_stats) = &self.statistics.column_statistics { - table_cols_stats.push(file_cols_stats[idx].clone()) - } else { - table_cols_stats.push(ColumnStatistics::new_with_unbounded_column( - field.data_type(), - )) - } + table_cols_stats.push(self.statistics.column_statistics[idx].clone()) } else { let partition_idx = idx - self.file_schema.fields().len(); let name = &self.table_partition_cols[partition_idx].0; @@ -154,7 +148,7 @@ impl FileScanConfig { is_exact: self.statistics.is_exact, // TODO correct byte size? total_byte_size: None, - column_statistics: Some(table_cols_stats), + column_statistics: table_cols_stats, }; let table_schema = Arc::new( @@ -523,10 +517,7 @@ mod tests { "partition columns are the last columns" ); assert_eq!( - proj_statistics - .column_statistics - .expect("projection creates column statistics") - .len(), + proj_statistics.column_statistics.len(), file_schema.fields().len() + 1 ); // TODO implement tests for partition column statistics once implemented @@ -548,14 +539,12 @@ mod tests { num_rows: Some(10), // assign the column index to distinct_count to help assert // the source statistic after the projection - column_statistics: Some( - (0..file_schema.fields().len()) - .map(|i| ColumnStatistics { - distinct_count: Some(i), - ..Default::default() - }) - .collect(), - ), + column_statistics: (0..file_schema.fields().len()) + .map(|i| ColumnStatistics { + distinct_count: Some(i), + ..Default::default() + }) + .collect(), total_byte_size: None, is_exact: false, }, @@ -570,9 +559,7 @@ mod tests { columns(&proj_schema), vec!["date".to_owned(), "c1".to_owned()] ); - let proj_stat_cols = proj_statistics - .column_statistics - .expect("projection creates column statistics"); + let proj_stat_cols = proj_statistics.column_statistics; assert_eq!(proj_stat_cols.len(), 2); // TODO implement tests for proj_stat_cols[0] once partition column // statistics are implemented diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 1b6a03e15c02..54545a403480 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -63,35 +63,34 @@ pub async fn get_statistics_with_limit( } else { file_stats.total_byte_size }; - if let Some(vec) = &file_stats.column_statistics { - has_statistics = true; - for (i, cs) in vec.iter().enumerate() { - null_counts[i] += cs.null_count.unwrap_or(0); - - if let Some(max_value) = &mut max_values[i] { - if let Some(file_max) = cs.max_value.clone() { - match max_value.update_batch(&[file_max.to_array()]) { - Ok(_) => {} - Err(_) => { - max_values[i] = None; - } + + has_statistics = true; + for (i, cs) in file_stats.column_statistics.iter().enumerate() { + null_counts[i] += cs.null_count.unwrap_or(0); + + if let Some(max_value) = &mut max_values[i] { + if let Some(file_max) = cs.max_value.clone() { + match max_value.update_batch(&[file_max.to_array()]) { + Ok(_) => {} + Err(_) => { + max_values[i] = None; } - } else { - max_values[i] = None; } + } else { + max_values[i] = None; } + } - if let Some(min_value) = &mut min_values[i] { - if let Some(file_min) = cs.min_value.clone() { - match min_value.update_batch(&[file_min.to_array()]) { - Ok(_) => {} - Err(_) => { - min_values[i] = None; - } + if let Some(min_value) = &mut min_values[i] { + if let Some(file_min) = cs.min_value.clone() { + match min_value.update_batch(&[file_min.to_array()]) { + Ok(_) => {} + Err(_) => { + min_values[i] = None; } - } else { - min_values[i] = None; } + } else { + min_values[i] = None; } } } @@ -112,14 +111,9 @@ pub async fn get_statistics_with_limit( } let column_stats = if has_statistics { - Some(get_col_stats( - &file_schema, - null_counts, - &mut max_values, - &mut min_values, - )) + get_col_stats(&file_schema, null_counts, &mut max_values, &mut min_values) } else { - None + Statistics::unbounded_column_statistics(&file_schema) }; let statistics = Statistics { diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 02337c7f7654..9ef5157b9a34 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -174,9 +174,9 @@ fn take_optimizable_column_count( agg_expr: &dyn AggregateExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { - if let (Some(num_rows), Some(col_stats), Some(casted_expr)) = ( + let col_stats = &stats.column_statistics; + if let (Some(num_rows), Some(casted_expr)) = ( stats.num_rows, - &stats.column_statistics, agg_expr.as_any().downcast_ref::(), ) { if casted_expr.expressions().len() == 1 { @@ -206,10 +206,8 @@ fn take_optimizable_min( agg_expr: &dyn AggregateExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { - if let (Some(col_stats), Some(casted_expr)) = ( - &stats.column_statistics, - agg_expr.as_any().downcast_ref::(), - ) { + let col_stats = &stats.column_statistics; + if let Some(casted_expr) = agg_expr.as_any().downcast_ref::() { if casted_expr.expressions().len() == 1 { // TODO optimize with exprs other than Column if let Some(col_expr) = casted_expr.expressions()[0] @@ -240,10 +238,8 @@ fn take_optimizable_max( agg_expr: &dyn AggregateExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { - if let (Some(col_stats), Some(casted_expr)) = ( - &stats.column_statistics, - agg_expr.as_any().downcast_ref::(), - ) { + let col_stats = &stats.column_statistics; + if let Some(casted_expr) = agg_expr.as_any().downcast_ref::() { if casted_expr.expressions().len() == 1 { // TODO optimize with exprs other than Column if let Some(col_expr) = casted_expr.expressions()[0] diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 7081f0c7be53..856741bf392c 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -606,9 +606,9 @@ mod tests_statistical { Statistics { num_rows: Some(10), total_byte_size: Some(100000), - column_statistics: Some(vec![ - ColumnStatistics::new_with_unbounded_column(&DataType::Int32), - ]), + column_statistics: vec![ColumnStatistics::new_with_unbounded_column( + &DataType::Int32, + )], is_exact: false, }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), @@ -618,9 +618,9 @@ mod tests_statistical { Statistics { num_rows: Some(100000), total_byte_size: Some(10), - column_statistics: Some(vec![ - ColumnStatistics::new_with_unbounded_column(&DataType::Int32), - ]), + column_statistics: vec![ColumnStatistics::new_with_unbounded_column( + &DataType::Int32, + )], is_exact: false, }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), @@ -637,13 +637,13 @@ mod tests_statistical { min: Option, max: Option, distinct_count: Option, - ) -> Option> { - Some(vec![ColumnStatistics { + ) -> Vec { + vec![ColumnStatistics { distinct_count, min_value: min.map(|size| ScalarValue::UInt64(Some(size))), max_value: max.map(|size| ScalarValue::UInt64(Some(size))), ..Default::default() - }]) + }] } /// Returns three plans with statistics of (min, max, distinct_count) @@ -1003,9 +1003,9 @@ mod tests_statistical { Statistics { num_rows: Some(10000000), total_byte_size: Some(10000000), - column_statistics: Some(vec![ - ColumnStatistics::new_with_unbounded_column(&DataType::Int32), - ]), + column_statistics: vec![ColumnStatistics::new_with_unbounded_column( + &DataType::Int32, + )], is_exact: false, }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), @@ -1015,9 +1015,9 @@ mod tests_statistical { Statistics { num_rows: Some(10), total_byte_size: Some(10), - column_statistics: Some(vec![ - ColumnStatistics::new_with_unbounded_column(&DataType::Int32), - ]), + column_statistics: vec![ColumnStatistics::new_with_unbounded_column( + &DataType::Int32, + )], is_exact: false, }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), @@ -1027,9 +1027,9 @@ mod tests_statistical { Statistics { num_rows: None, total_byte_size: None, - column_statistics: Some(vec![ - ColumnStatistics::new_with_unbounded_column(&DataType::Int32), - ]), + column_statistics: vec![ColumnStatistics::new_with_unbounded_column( + &DataType::Int32, + )], is_exact: false, }, Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), @@ -1090,9 +1090,9 @@ mod tests_statistical { Statistics { num_rows: Some(10000000), total_byte_size: Some(10000000), - column_statistics: Some(vec![ - ColumnStatistics::new_with_unbounded_column(&DataType::Int32), - ]), + column_statistics: vec![ColumnStatistics::new_with_unbounded_column( + &DataType::Int32, + )], is_exact: false, }, Schema::new(vec![Field::new("big_col1", DataType::Int32, false)]), @@ -1102,9 +1102,9 @@ mod tests_statistical { Statistics { num_rows: Some(20000000), total_byte_size: Some(20000000), - column_statistics: Some(vec![ - ColumnStatistics::new_with_unbounded_column(&DataType::Int32), - ]), + column_statistics: vec![ColumnStatistics::new_with_unbounded_column( + &DataType::Int32, + )], is_exact: false, }, Schema::new(vec![Field::new("big_col2", DataType::Int32, false)]), @@ -1114,9 +1114,9 @@ mod tests_statistical { Statistics { num_rows: None, total_byte_size: None, - column_statistics: Some(vec![ - ColumnStatistics::new_with_unbounded_column(&DataType::Int32), - ]), + column_statistics: vec![ColumnStatistics::new_with_unbounded_column( + &DataType::Int32, + )], is_exact: false, }, Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 86c5d6ce241c..69f06f6db7a7 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -308,12 +308,8 @@ pub struct StatisticsExec { } impl StatisticsExec { pub fn new(stats: Statistics, schema: Schema) -> Self { - assert!( - stats - .column_statistics - .as_ref() - .map(|cols| cols.len() == schema.fields().len()) - .unwrap_or(true), + assert_eq!( + stats.column_statistics.len(), schema.fields().len(), "if defined, the column statistics vector length should be the number of fields" ); Self { diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index bc20d0ea092a..d816fba067c9 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -159,23 +159,22 @@ impl ExecutionPlan for CustomExecutionPlan { is_exact: true, num_rows: Some(batch.num_rows()), total_byte_size: None, - column_statistics: Some( - self.projection - .clone() - .unwrap_or_else(|| (0..batch.columns().len()).collect()) - .iter() - .map(|i| ColumnStatistics { - null_count: Some(batch.column(*i).null_count()), - min_value: Some(ScalarValue::Int32(aggregate::min( - as_primitive_array::(batch.column(*i)).unwrap(), - ))), - max_value: Some(ScalarValue::Int32(aggregate::max( - as_primitive_array::(batch.column(*i)).unwrap(), - ))), - ..Default::default() - }) - .collect(), - ), + column_statistics: self + .projection + .clone() + .unwrap_or_else(|| (0..batch.columns().len()).collect()) + .iter() + .map(|i| ColumnStatistics { + null_count: Some(batch.column(*i).null_count()), + min_value: Some(ScalarValue::Int32(aggregate::min( + as_primitive_array::(batch.column(*i)).unwrap(), + ))), + max_value: Some(ScalarValue::Int32(aggregate::max( + as_primitive_array::(batch.column(*i)).unwrap(), + ))), + ..Default::default() + }) + .collect(), }) } } diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index 6d1aaeef5678..540d16ddf361 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -46,13 +46,10 @@ struct StatisticsValidation { impl StatisticsValidation { fn new(stats: Statistics, schema: SchemaRef) -> Self { - assert!( - stats - .column_statistics - .as_ref() - .map(|cols| cols.len() == schema.fields().len()) - .unwrap_or(true), - "if defined, the column statistics vector length should be the number of fields" + assert_eq!( + stats.column_statistics.len(), + schema.fields().len(), + "the column statistics vector length should be the number of fields" ); Self { stats, schema } } @@ -94,10 +91,10 @@ impl TableProvider for StatisticsValidation { let current_stat = self.stats.clone(); - let proj_col_stats = current_stat - .column_statistics - .map(|col_stat| projection.iter().map(|i| col_stat[*i].clone()).collect()); - + let proj_col_stats = projection + .iter() + .map(|i| current_stat.column_statistics[*i].clone()) + .collect(); Ok(Arc::new(Self::new( Statistics { is_exact: current_stat.is_exact, @@ -185,7 +182,7 @@ fn fully_defined() -> (Statistics, Schema) { num_rows: Some(13), is_exact: true, total_byte_size: None, // ignore byte size for now - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { distinct_count: Some(2), max_value: Some(ScalarValue::Int32(Some(1023))), @@ -198,7 +195,7 @@ fn fully_defined() -> (Statistics, Schema) { min_value: Some(ScalarValue::Int64(Some(-6783))), null_count: Some(5), }, - ]), + ], }, Schema::new(vec![ Field::new("c1", DataType::Int32, false), @@ -254,7 +251,7 @@ async fn sql_limit() -> Result<()> { Statistics { num_rows: Some(5), is_exact: true, - column_statistics: Some(col_stats), + column_statistics: col_stats, total_byte_size: None }, physical_plan.statistics()? @@ -286,10 +283,9 @@ async fn sql_window() -> Result<()> { let result = physical_plan.statistics()?; assert_eq!(stats.num_rows, result.num_rows); - assert!(result.column_statistics.is_some()); - let col_stats = result.column_statistics.unwrap(); + let col_stats = result.column_statistics; assert_eq!(2, col_stats.len()); - assert_eq!(stats.column_statistics.unwrap()[1], col_stats[0]); + assert_eq!(stats.column_statistics[1], col_stats[0]); Ok(()) } diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index a0f689968ced..3dd035222949 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -461,10 +461,7 @@ async fn parquet_statistics() -> Result<()> { let schema = physical_plan.schema(); assert_eq!(schema.fields().len(), 4); - let stat_cols = physical_plan - .statistics()? - .column_statistics - .expect("col stats should be defined"); + let stat_cols = physical_plan.statistics()?.column_statistics; assert_eq!(stat_cols.len(), 4); // stats for the first col are read from the parquet file assert_eq!(stat_cols[0].null_count, Some(3)); @@ -488,10 +485,7 @@ async fn parquet_statistics() -> Result<()> { let schema = physical_plan.schema(); assert_eq!(schema.fields().len(), 2); - let stat_cols = physical_plan - .statistics()? - .column_statistics - .expect("col stats should be defined"); + let stat_cols = physical_plan.statistics()?.column_statistics; assert_eq!(stat_cols.len(), 2); // stats for the first col are read from the parquet file assert_eq!(stat_cols[0].null_count, Some(1)); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index ac743cf4f2eb..089ed1cf8922 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1015,7 +1015,7 @@ impl ExecutionPlan for AggregateExec { Ok(Statistics { num_rows: Some(1), is_exact: true, - column_statistics: Some(column_statistics), + column_statistics, total_byte_size: None, }) } @@ -1023,7 +1023,7 @@ impl ExecutionPlan for AggregateExec { // the output row count is surely not larger than its input row count num_rows: self.input.statistics()?.num_rows, is_exact: false, - column_statistics: Some(column_statistics), + column_statistics, total_byte_size: None, }), } diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index bf16fe2e8d39..aaef07ef9363 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -164,7 +164,7 @@ pub fn compute_record_batch_statistics( Statistics { num_rows: Some(nb_rows), total_byte_size: Some(total_byte_size), - column_statistics: Some(column_statistics), + column_statistics, is_exact: true, } } @@ -702,7 +702,7 @@ mod tests { is_exact: true, num_rows: Some(3), total_byte_size: Some(464), // this might change a bit if the way we compute the size changes - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { distinct_count: None, max_value: Some(ScalarValue::Float32(None)), @@ -715,7 +715,7 @@ mod tests { min_value: Some(ScalarValue::Float64(None)), null_count: Some(0), }, - ]), + ], }; // Prevent test flakiness due to undefined / changing implementation details diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 5c77e51568be..f296850f83f3 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -158,17 +158,13 @@ impl ExecutionPlan for FilterExec { .statistics() .expect("Ordering equivalences need to handle the error case of statistics"); // Add the columns that have only one value (singleton) after filtering to constants. - if let Some(col_stats) = stats.column_statistics { - let constants = collect_columns(self.predicate()) - .into_iter() - .filter(|column| col_stats[column.index()].is_singleton()) - .map(|column| Arc::new(column) as Arc) - .collect::>(); - let filter_oeq = self.input.ordering_equivalence_properties(); - filter_oeq.with_constants(constants) - } else { - self.input.ordering_equivalence_properties() - } + let constants = collect_columns(self.predicate()) + .into_iter() + .filter(|column| stats.column_statistics[column.index()].is_singleton()) + .map(|column| Arc::new(column) as Arc) + .collect::>(); + let filter_oeq = self.input.ordering_equivalence_properties(); + filter_oeq.with_constants(constants) } fn with_new_children( @@ -210,16 +206,12 @@ impl ExecutionPlan for FilterExec { } let input_stats = self.input.statistics()?; - let column_stats = if let Some(column_stats) = input_stats.column_statistics { - column_stats - } else { - Statistics::unbounded_column_statistics(&self.input.schema()) - }; - let num_rows = input_stats.num_rows; let total_byte_size = input_stats.total_byte_size; - let input_analysis_ctx = - AnalysisContext::from_statistics(&self.input.schema(), &column_stats); + let input_analysis_ctx = AnalysisContext::from_statistics( + &self.input.schema(), + &input_stats.column_statistics, + ); let analysis_ctx = analyze(predicate, input_analysis_ctx)?; let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); @@ -228,19 +220,21 @@ impl ExecutionPlan for FilterExec { total_byte_size.map(|size| (size as f64 * selectivity).ceil() as usize); if let Some(analysis_boundaries) = analysis_ctx.boundaries { - let column_statistics = - collect_new_statistics(&column_stats, analysis_boundaries); + let column_statistics = collect_new_statistics( + &input_stats.column_statistics, + analysis_boundaries, + ); Ok(Statistics { num_rows, total_byte_size, - column_statistics: Some(column_statistics), + column_statistics, is_exact: false, }) } else { Ok(Statistics { num_rows, total_byte_size, - column_statistics: Some(column_stats.to_vec()), + column_statistics: input_stats.column_statistics, is_exact: false, }) } @@ -454,11 +448,11 @@ mod tests { Statistics { num_rows: Some(100), total_byte_size: Some(100 * bytes_per_row), - column_statistics: Some(vec![ColumnStatistics { + column_statistics: vec![ColumnStatistics { min_value: Some(ScalarValue::Int32(Some(1))), max_value: Some(ScalarValue::Int32(Some(100))), ..Default::default() - }]), + }], is_exact: false, }, schema.clone(), @@ -477,11 +471,11 @@ mod tests { assert_eq!(statistics.total_byte_size, Some(25 * bytes_per_row)); assert_eq!( statistics.column_statistics, - Some(vec![ColumnStatistics { + vec![ColumnStatistics { min_value: Some(ScalarValue::Int32(Some(1))), max_value: Some(ScalarValue::Int32(Some(25))), ..Default::default() - }]) + }] ); Ok(()) @@ -495,11 +489,11 @@ mod tests { let input = Arc::new(StatisticsExec::new( Statistics { num_rows: Some(100), - column_statistics: Some(vec![ColumnStatistics { + column_statistics: vec![ColumnStatistics { min_value: Some(ScalarValue::Int32(Some(1))), max_value: Some(ScalarValue::Int32(Some(100))), ..Default::default() - }]), + }], is_exact: false, total_byte_size: None, }, @@ -524,11 +518,11 @@ mod tests { assert_eq!(statistics.num_rows, Some(16)); assert_eq!( statistics.column_statistics, - Some(vec![ColumnStatistics { + vec![ColumnStatistics { min_value: Some(ScalarValue::Int32(Some(10))), max_value: Some(ScalarValue::Int32(Some(25))), ..Default::default() - }]) + }] ); Ok(()) @@ -546,7 +540,7 @@ mod tests { let input = Arc::new(StatisticsExec::new( Statistics { num_rows: Some(100), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { min_value: Some(ScalarValue::Int32(Some(1))), max_value: Some(ScalarValue::Int32(Some(100))), @@ -557,7 +551,7 @@ mod tests { max_value: Some(ScalarValue::Int32(Some(50))), ..Default::default() }, - ]), + ], is_exact: false, total_byte_size: None, }, @@ -591,7 +585,7 @@ mod tests { assert_eq!(statistics.num_rows, Some(2)); assert_eq!( statistics.column_statistics, - Some(vec![ + vec![ ColumnStatistics { min_value: Some(ScalarValue::Int32(Some(10))), max_value: Some(ScalarValue::Int32(Some(25))), @@ -602,7 +596,7 @@ mod tests { max_value: Some(ScalarValue::Int32(Some(50))), ..Default::default() } - ]) + ] ); Ok(()) @@ -647,7 +641,7 @@ mod tests { Statistics { num_rows: Some(1000), total_byte_size: Some(4000), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { min_value: Some(ScalarValue::Int32(Some(1))), max_value: Some(ScalarValue::Int32(Some(100))), @@ -663,7 +657,7 @@ mod tests { max_value: Some(ScalarValue::Float32(Some(1100.0))), ..Default::default() }, - ]), + ], is_exact: false, }, schema, @@ -725,7 +719,7 @@ mod tests { ]; let _ = exp_col_stats .into_iter() - .zip(statistics.column_statistics.unwrap()) + .zip(statistics.column_statistics) .map(|(expected, actual)| { if actual.min_value.clone().unwrap().data_type().is_floating() { // Windows rounds arithmetic operation results differently for floating point numbers. @@ -762,7 +756,7 @@ mod tests { Statistics { num_rows: Some(1000), total_byte_size: Some(4000), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { min_value: Some(ScalarValue::Int32(Some(1))), max_value: Some(ScalarValue::Int32(Some(100))), @@ -773,7 +767,7 @@ mod tests { max_value: Some(ScalarValue::Int32(Some(3))), ..Default::default() }, - ]), + ], is_exact: false, }, schema, @@ -818,7 +812,7 @@ mod tests { Statistics { num_rows: Some(1000), total_byte_size: Some(4000), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { min_value: Some(ScalarValue::Int32(Some(1))), max_value: Some(ScalarValue::Int32(Some(100))), @@ -829,7 +823,7 @@ mod tests { max_value: Some(ScalarValue::Int32(Some(3))), ..Default::default() }, - ]), + ], is_exact: false, }, schema, @@ -856,7 +850,7 @@ mod tests { assert_eq!(statistics.total_byte_size, Some(0)); assert_eq!( statistics.column_statistics, - Some(vec![ + vec![ ColumnStatistics { min_value: Some(ScalarValue::Int32(Some(1))), max_value: Some(ScalarValue::Int32(Some(100))), @@ -867,7 +861,7 @@ mod tests { max_value: Some(ScalarValue::Int32(Some(3))), ..Default::default() }, - ]) + ] ); Ok(()) @@ -883,7 +877,7 @@ mod tests { Statistics { num_rows: Some(1000), total_byte_size: Some(4000), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { min_value: Some(ScalarValue::Int32(Some(1))), max_value: Some(ScalarValue::Int32(Some(100))), @@ -894,7 +888,7 @@ mod tests { max_value: Some(ScalarValue::Int32(Some(100))), ..Default::default() }, - ]), + ], is_exact: false, }, schema, @@ -913,7 +907,7 @@ mod tests { assert_eq!(statistics.total_byte_size, Some(1960)); assert_eq!( statistics.column_statistics, - Some(vec![ + vec![ ColumnStatistics { min_value: Some(ScalarValue::Int32(Some(1))), max_value: Some(ScalarValue::Int32(Some(49))), @@ -924,7 +918,7 @@ mod tests { max_value: Some(ScalarValue::Int32(Some(100))), ..Default::default() }, - ]) + ] ); Ok(()) @@ -962,12 +956,12 @@ mod tests { let expected_filter_statistics = Statistics { num_rows: None, total_byte_size: None, - column_statistics: Some(vec![ColumnStatistics { + column_statistics: vec![ColumnStatistics { null_count: None, min_value: Some(ScalarValue::Int32(Some(5))), max_value: Some(ScalarValue::Int32(Some(10))), distinct_count: None, - }]), + }], is_exact: false, }; diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 6dba4efa48ce..adb279203706 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -260,9 +260,7 @@ impl ExecutionPlan for CrossJoinExec { fn statistics(&self) -> Result { Ok(stats_cartesian_product( self.left.statistics()?, - self.left.schema(), self.right.statistics()?, - self.right.schema(), )) } } @@ -270,9 +268,7 @@ impl ExecutionPlan for CrossJoinExec { /// [left/right]_col_count are required in case the column statistics are None fn stats_cartesian_product( left_stats: Statistics, - left_schema: SchemaRef, right_stats: Statistics, - right_schema: SchemaRef, ) -> Statistics { let left_row_count = left_stats.num_rows; let right_row_count = right_stats.num_rows; @@ -289,16 +285,8 @@ fn stats_cartesian_product( .zip(right_stats.total_byte_size) .map(|(a, b)| 2 * a * b); - let left_col_stats = if let Some(left_col_stats) = left_stats.column_statistics { - left_col_stats - } else { - Statistics::unbounded_column_statistics(&left_schema) - }; - let right_col_stats = if let Some(right_col_stats) = right_stats.column_statistics { - right_col_stats - } else { - Statistics::unbounded_column_statistics(&right_schema) - }; + let left_col_stats = left_stats.column_statistics; + let right_col_stats = right_stats.column_statistics; // the null counts must be multiplied by the row counts of the other side (if defined) // Min, max and distinct_count on the other hand are invariants. @@ -322,7 +310,7 @@ fn stats_cartesian_product( is_exact, num_rows, total_byte_size, - column_statistics: Some(cross_join_stats), + column_statistics: cross_join_stats, } } @@ -454,7 +442,6 @@ mod tests { use super::*; use crate::common; use crate::test::build_table_scan_i32; - use arrow_schema::{DataType, Field}; use datafusion_common::{assert_batches_sorted_eq, assert_contains}; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; @@ -483,7 +470,7 @@ mod tests { is_exact: true, num_rows: Some(left_row_count), total_byte_size: Some(left_bytes), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { distinct_count: Some(5), max_value: Some(ScalarValue::Int64(Some(21))), @@ -496,40 +483,28 @@ mod tests { min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), null_count: Some(3), }, - ]), + ], }; let right = Statistics { is_exact: true, num_rows: Some(right_row_count), total_byte_size: Some(right_bytes), - column_statistics: Some(vec![ColumnStatistics { + column_statistics: vec![ColumnStatistics { distinct_count: Some(3), max_value: Some(ScalarValue::Int64(Some(12))), min_value: Some(ScalarValue::Int64(Some(0))), null_count: Some(2), - }]), + }], }; - let result = stats_cartesian_product( - left, - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Int32, false), - Field::new("c", DataType::Int32, false), - ])), - right, - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Int32, false), - ])), - ); + let result = stats_cartesian_product(left, right); let expected = Statistics { is_exact: true, num_rows: Some(left_row_count * right_row_count), total_byte_size: Some(2 * left_bytes * right_bytes), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { distinct_count: Some(5), max_value: Some(ScalarValue::Int64(Some(21))), @@ -548,7 +523,7 @@ mod tests { min_value: Some(ScalarValue::Int64(Some(0))), null_count: Some(2 * left_row_count), }, - ]), + ], }; assert_eq!(result, expected); @@ -562,7 +537,7 @@ mod tests { is_exact: true, num_rows: Some(left_row_count), total_byte_size: Some(23), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { distinct_count: Some(5), max_value: Some(ScalarValue::Int64(Some(21))), @@ -575,40 +550,28 @@ mod tests { min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), null_count: Some(3), }, - ]), + ], }; let right = Statistics { is_exact: true, num_rows: None, // not defined! total_byte_size: None, // not defined! - column_statistics: Some(vec![ColumnStatistics { + column_statistics: vec![ColumnStatistics { distinct_count: Some(3), max_value: Some(ScalarValue::Int64(Some(12))), min_value: Some(ScalarValue::Int64(Some(0))), null_count: Some(2), - }]), + }], }; - let result = stats_cartesian_product( - left, - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Int32, false), - Field::new("c", DataType::Int32, false), - ])), - right, - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Int32, false), - ])), - ); + let result = stats_cartesian_product(left, right); let expected = Statistics { is_exact: true, num_rows: None, total_byte_size: None, - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { distinct_count: Some(5), max_value: Some(ScalarValue::Int64(Some(21))), @@ -627,7 +590,7 @@ mod tests { min_value: Some(ScalarValue::Int64(Some(0))), null_count: Some(2 * left_row_count), }, - ]), + ], }; assert_eq!(result, expected); diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index edb31821af1d..4fd8882c0178 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -502,6 +502,7 @@ impl ExecutionPlan for HashJoinExec { self.right.clone(), self.on.clone(), &self.join_type, + &self.schema, ) } } diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 59429186d606..368ae825caab 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -288,6 +288,7 @@ impl ExecutionPlan for NestedLoopJoinExec { self.right.clone(), vec![], &self.join_type, + &self.schema, ) } } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 85a761869800..dd74abe7713c 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -390,6 +390,7 @@ impl ExecutionPlan for SortMergeJoinExec { self.right.clone(), self.on.clone(), &self.join_type, + &self.schema, ) } } diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 2028eed64f8f..156d8d4f3e0b 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -726,14 +726,15 @@ pub(crate) fn estimate_join_statistics( right: Arc, on: JoinOn, join_type: &JoinType, + schema: &Schema, ) -> Result { let left_stats = left.statistics()?; let right_stats = right.statistics()?; let join_stats = estimate_join_cardinality(join_type, left_stats, right_stats, &on); let (num_rows, column_statistics) = match join_stats { - Some(stats) => (Some(stats.num_rows), Some(stats.column_statistics)), - None => (None, None), + Some(stats) => (Some(stats.num_rows), stats.column_statistics), + None => (None, Statistics::unbounded_column_statistics(schema)), }; Ok(Statistics { num_rows, @@ -757,8 +758,8 @@ fn estimate_join_cardinality( // Take the left_col_stats and right_col_stats using the index // obtained from index() method of the each element of 'on'. - let all_left_col_stats = left_stats.column_statistics?; - let all_right_col_stats = right_stats.column_statistics?; + let all_left_col_stats = left_stats.column_statistics; + let all_right_col_stats = right_stats.column_statistics; let (left_col_stats, right_col_stats) = on .iter() .map(|(left, right)| { @@ -1543,7 +1544,7 @@ mod tests { fn create_stats( num_rows: Option, - column_stats: Option>, + column_stats: Vec, is_exact: bool, ) -> Statistics { Statistics { @@ -1727,8 +1728,8 @@ mod tests { let join_on = vec![(Column::new("a", 0), Column::new("b", 0))]; let partial_join_stats = estimate_join_cardinality( &join_type, - create_stats(Some(left_num_rows), Some(left_col_stats.clone()), false), - create_stats(Some(right_num_rows), Some(right_col_stats.clone()), false), + create_stats(Some(left_num_rows), left_col_stats.clone(), false), + create_stats(Some(right_num_rows), right_col_stats.clone(), false), &join_on, ); @@ -1840,8 +1841,8 @@ mod tests { let partial_join_stats = estimate_join_cardinality( &join_type, - create_stats(Some(1000), Some(left_col_stats.clone()), false), - create_stats(Some(2000), Some(right_col_stats.clone()), false), + create_stats(Some(1000), left_col_stats.clone(), false), + create_stats(Some(2000), right_col_stats.clone(), false), &join_on, ) .unwrap(); @@ -1905,8 +1906,8 @@ mod tests { for (join_type, expected_num_rows) in cases { let partial_join_stats = estimate_join_cardinality( &join_type, - create_stats(Some(1000), Some(left_col_stats.clone()), true), - create_stats(Some(2000), Some(right_col_stats.clone()), true), + create_stats(Some(1000), left_col_stats.clone(), true), + create_stats(Some(2000), right_col_stats.clone(), true), &join_on, ) .unwrap(); diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 2fe2e678f60f..aba848ddce23 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -215,7 +215,7 @@ impl ExecutionPlan for GlobalLimitExec { Statistics { num_rows: Some(0), is_exact: input_stats.is_exact, - column_statistics: Some(col_stats), + column_statistics: col_stats, total_byte_size: None, } } else if nr <= max_row_num { @@ -227,7 +227,7 @@ impl ExecutionPlan for GlobalLimitExec { Statistics { num_rows: Some(max_row_num), is_exact: input_stats.is_exact, - column_statistics: Some(col_stats), + column_statistics: col_stats, total_byte_size: None, } } @@ -236,7 +236,7 @@ impl ExecutionPlan for GlobalLimitExec { // the result output row number will always be no greater than the limit number num_rows: Some(max_row_num), is_exact: false, - column_statistics: Some(col_stats), + column_statistics: col_stats, total_byte_size: None, }, }; @@ -384,14 +384,14 @@ impl ExecutionPlan for LocalLimitExec { // this is not actually exact, but will be when GlobalLimit is applied // TODO stats: find a more explicit way to vehiculate this information is_exact: input_stats.is_exact, - column_statistics: Some(col_stats), + column_statistics: col_stats, total_byte_size: None, }, _ => Statistics { // the result output row number will always be no greater than the limit number num_rows: Some(self.fetch * self.output_partitioning().partition_count()), is_exact: false, - column_statistics: Some(col_stats), + column_statistics: col_stats, total_byte_size: None, }, }; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index ebed44977f33..f2d653dee226 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -399,24 +399,22 @@ fn stats_projection( schema: SchemaRef, ) -> Statistics { let inner_exprs = exprs.collect::>(); - let column_statistics = stats.column_statistics.map(|input_col_stats| { - inner_exprs - .clone() - .into_iter() - .enumerate() - .map(|(index, e)| { - if let Some(col) = e.as_any().downcast_ref::() { - input_col_stats[col.index()].clone() - } else { - // TODO stats: estimate more statistics from expressions - // (expressions should compute their statistics themselves) - ColumnStatistics::new_with_unbounded_column( - schema.field(index).data_type(), - ) - } - }) - .collect() - }); + let column_statistics = inner_exprs + .clone() + .into_iter() + .enumerate() + .map(|(index, e)| { + if let Some(col) = e.as_any().downcast_ref::() { + stats.column_statistics[col.index()].clone() + } else { + // TODO stats: estimate more statistics from expressions + // (expressions should compute their statistics themselves) + ColumnStatistics::new_with_unbounded_column( + schema.field(index).data_type(), + ) + } + }) + .collect(); let primitive_row_size = inner_exprs .into_iter() @@ -537,7 +535,7 @@ mod tests { is_exact: true, num_rows: Some(5), total_byte_size: Some(23), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { distinct_count: Some(5), max_value: Some(ScalarValue::Int64(Some(21))), @@ -556,7 +554,7 @@ mod tests { min_value: Some(ScalarValue::Float32(Some(0.1))), null_count: None, }, - ]), + ], } } @@ -582,7 +580,7 @@ mod tests { is_exact: true, num_rows: Some(5), total_byte_size: Some(23), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { distinct_count: Some(1), max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), @@ -595,7 +593,7 @@ mod tests { min_value: Some(ScalarValue::Int64(Some(-4))), null_count: Some(0), }, - ]), + ], }; assert_eq!(result, expected); @@ -617,7 +615,7 @@ mod tests { is_exact: true, num_rows: Some(5), total_byte_size: Some(60), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { distinct_count: None, max_value: Some(ScalarValue::Float32(Some(1.1))), @@ -630,7 +628,7 @@ mod tests { min_value: Some(ScalarValue::Int64(Some(-4))), null_count: Some(0), }, - ]), + ], }; assert_eq!(result, expected); diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index a25e21b59254..94741c6ff89d 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -469,12 +469,9 @@ pub struct StatisticsExec { } impl StatisticsExec { pub fn new(stats: Statistics, schema: Schema) -> Self { - assert!( + assert_eq!( stats - .column_statistics - .as_ref() - .map(|cols| cols.len() == schema.fields().len()) - .unwrap_or(true), + .column_statistics.len(), schema.fields().len(), "if defined, the column statistics vector length should be the number of fields" ); Self { diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 7f2f7b9a1809..c09c3a9fd12b 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -597,15 +597,12 @@ fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { .total_byte_size .zip(right.total_byte_size) .map(|(a, b)| a + b); - left.column_statistics = - left.column_statistics - .zip(right.column_statistics) - .map(|(a, b)| { - a.into_iter() - .zip(b) - .map(|(ca, cb)| col_stats_union(ca, cb)) - .collect() - }); + left.column_statistics = left + .column_statistics + .into_iter() + .zip(right.column_statistics) + .map(|(a, b)| col_stats_union(a, b)) + .collect::>(); left } @@ -643,7 +640,7 @@ mod tests { is_exact: true, num_rows: Some(5), total_byte_size: Some(23), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { distinct_count: Some(5), max_value: Some(ScalarValue::Int64(Some(21))), @@ -662,14 +659,14 @@ mod tests { min_value: Some(ScalarValue::Float32(Some(0.1))), null_count: None, }, - ]), + ], }; let right = Statistics { is_exact: true, num_rows: Some(7), total_byte_size: Some(29), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { distinct_count: Some(3), max_value: Some(ScalarValue::Int64(Some(34))), @@ -688,7 +685,7 @@ mod tests { min_value: None, null_count: None, }, - ]), + ], }; let result = stats_union(left, right); @@ -696,7 +693,7 @@ mod tests { is_exact: true, num_rows: Some(12), total_byte_size: Some(52), - column_statistics: Some(vec![ + column_statistics: vec![ ColumnStatistics { distinct_count: None, max_value: Some(ScalarValue::Int64(Some(34))), @@ -715,7 +712,7 @@ mod tests { min_value: None, null_count: None, }, - ]), + ], }; assert_eq!(result, expected); 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 1648926d7e98..f8793be46f91 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -336,15 +336,8 @@ impl ExecutionPlan for BoundedWindowAggExec { let input_cols = self.input_schema.fields().len(); // TODO stats: some windowing function will maintain invariants such as min, max... let mut column_statistics = Vec::with_capacity(win_cols + input_cols); - if let Some(input_col_stats) = input_stat.column_statistics { - column_statistics.extend(input_col_stats); - } else { - for index in 0..input_cols { - column_statistics.push(ColumnStatistics::new_with_unbounded_column( - self.schema().field(index).data_type(), - )) - } - } + // copy stats of the input to the beginning of the schema. + column_statistics.extend(input_stat.column_statistics); for index in 0..win_cols { column_statistics.push(ColumnStatistics::new_with_unbounded_column( self.schema().field(index + input_cols).data_type(), @@ -353,7 +346,7 @@ impl ExecutionPlan for BoundedWindowAggExec { Ok(Statistics { is_exact: input_stat.is_exact, num_rows: input_stat.num_rows, - column_statistics: Some(column_statistics), + column_statistics, total_byte_size: None, }) } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 67f4cfd34623..d49f23cdadc2 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -260,14 +260,12 @@ impl ExecutionPlan for WindowAggExec { let input_cols = self.input_schema.fields().len(); // TODO stats: some windowing function will maintain invariants such as min, max... let mut column_statistics = Vec::with_capacity(win_cols + input_cols); - if let Some(input_col_stats) = input_stat.column_statistics { - column_statistics.extend(input_col_stats); - } else { - for index in 0..input_cols { - column_statistics.push(ColumnStatistics::new_with_unbounded_column( - self.schema().field(index).data_type(), - )) - } + // copy stats of the input to the beginning of the schema. + column_statistics.extend(input_stat.column_statistics); + for index in 0..win_cols { + column_statistics.push(ColumnStatistics::new_with_unbounded_column( + self.schema().field(index + input_cols).data_type(), + )) } for index in 0..win_cols { column_statistics.push(ColumnStatistics::new_with_unbounded_column( @@ -277,7 +275,7 @@ impl ExecutionPlan for WindowAggExec { Ok(Statistics { is_exact: input_stat.is_exact, num_rows: input_stat.num_rows, - column_statistics: Some(column_statistics), + column_statistics, total_byte_size: None, }) } diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 8f929be10b02..d7c470ea333a 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -627,11 +627,7 @@ impl TryFrom<&protobuf::Statistics> for Statistics { Some(s.total_byte_size as usize) }, // No column statistic (None) is encoded with empty array - column_statistics: if column_statistics.is_empty() { - None - } else { - Some(column_statistics) - }, + column_statistics, is_exact: s.is_exact, }) } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 417b2af31499..54c191fc941d 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1951,13 +1951,11 @@ mod roundtrip_tests { #[test] fn roundtrip_parquet_exec_with_pruning_predicate() -> Result<()> { + let schema = + Arc::new(Schema::new(vec![Field::new("col", DataType::Utf8, false)])); let scan_config = FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), - file_schema: Arc::new(Schema::new(vec![Field::new( - "col", - DataType::Utf8, - false, - )])), + file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new( "/path/to/file.parquet".to_string(), 1024, @@ -1965,7 +1963,7 @@ mod roundtrip_tests { statistics: Statistics { num_rows: Some(100), total_byte_size: Some(1024), - column_statistics: None, + column_statistics: Statistics::unbounded_column_statistics(&schema), is_exact: false, }, projection: None, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 076731c22b7f..fc797ec08959 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -658,10 +658,7 @@ impl From<&ColumnStatistics> for protobuf::ColumnStats { impl From<&Statistics> for protobuf::Statistics { fn from(s: &Statistics) -> protobuf::Statistics { let none_value = -1_i64; - let column_stats = match &s.column_statistics { - None => vec![], - Some(column_stats) => column_stats.iter().map(|s| s.into()).collect(), - }; + let column_stats = s.column_statistics.iter().map(|s| s.into()).collect(); protobuf::Statistics { num_rows: s.num_rows.map(|n| n as i64).unwrap_or(none_value), total_byte_size: s.total_byte_size.map(|n| n as i64).unwrap_or(none_value), From 6fa17a52062afb20138c7dd89d15ce660ac8ce0d Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 26 Sep 2023 17:07:00 +0300 Subject: [PATCH 16/53] exact info added --- datafusion/common/src/stats.rs | 211 ++++++-- .../src/datasource/file_format/parquet.rs | 55 +- .../physical_plan/file_scan_config.rs | 14 +- datafusion/core/src/datasource/statistics.rs | 46 +- .../aggregate_statistics.rs | 25 +- .../src/physical_optimizer/join_selection.rs | 99 ++-- datafusion/physical-expr/src/analysis.rs | 9 +- .../physical-plan/src/aggregates/mod.rs | 9 +- datafusion/physical-plan/src/common.rs | 45 +- datafusion/physical-plan/src/filter.rs | 222 ++++---- .../physical-plan/src/joins/cross_join.rs | 158 +++--- datafusion/physical-plan/src/joins/utils.rs | 270 ++++++---- datafusion/physical-plan/src/limit.rs | 79 +-- datafusion/physical-plan/src/projection.rs | 90 ++-- datafusion/physical-plan/src/union.rs | 124 +++-- .../src/windows/bounded_window_agg_exec.rs | 4 +- .../src/windows/window_agg_exec.rs | 4 +- datafusion/proto/proto/datafusion.proto | 39 +- datafusion/proto/src/generated/pbjson.rs | 479 ++++++++++++++++-- datafusion/proto/src/generated/prost.rs | 79 ++- 20 files changed, 1389 insertions(+), 672 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index a129672737ba..ba7c89c5c60d 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -17,10 +17,138 @@ //! This module provides data structures to represent statistics +use crate::ScalarValue; use arrow::datatypes::{DataType, Schema}; -use std::fmt::Display; +use core::fmt::Debug; +use std::fmt::{self, Display}; -use crate::ScalarValue; +#[derive(Clone, PartialEq, Eq)] +pub enum Sharpness { + Exact(T), + Inexact(T), + Absent, +} + +impl Sharpness { + pub fn get_value(&self) -> Option { + match self { + Sharpness::Exact(val) | Sharpness::Inexact(val) => Some(val.clone()), + Sharpness::Absent => None, + } + } + + pub fn map(self, f: F) -> Sharpness + where + F: Fn(T) -> T, + { + match self { + Sharpness::Exact(val) => Sharpness::Exact(f(val)), + Sharpness::Inexact(val) => Sharpness::Inexact(f(val)), + _ => self, + } + } + + pub fn is_exact(&self) -> Option { + match self { + Sharpness::Exact(_) => Some(true), + Sharpness::Inexact(_) => Some(false), + _ => None, + } + } + + pub fn max(&self, other: &Sharpness) -> Sharpness { + match (self, other) { + (Sharpness::Exact(a), Sharpness::Exact(b)) => { + Sharpness::Exact(if a >= b { a.clone() } else { b.clone() }) + } + (Sharpness::Inexact(a), Sharpness::Exact(b)) + | (Sharpness::Exact(a), Sharpness::Inexact(b)) + | (Sharpness::Inexact(a), Sharpness::Inexact(b)) => { + Sharpness::Inexact(if a >= b { a.clone() } else { b.clone() }) + } + (_, _) => Sharpness::Absent, + } + } + + pub fn min(&self, other: &Sharpness) -> Sharpness { + match (self, other) { + (Sharpness::Exact(a), Sharpness::Exact(b)) => { + Sharpness::Exact(if a >= b { b.clone() } else { a.clone() }) + } + (Sharpness::Inexact(a), Sharpness::Exact(b)) + | (Sharpness::Exact(a), Sharpness::Inexact(b)) + | (Sharpness::Inexact(a), Sharpness::Inexact(b)) => { + Sharpness::Inexact(if a >= b { b.clone() } else { a.clone() }) + } + (_, _) => Sharpness::Absent, + } + } +} + +impl Sharpness { + pub fn add(&self, other: &Sharpness) -> Sharpness { + match (self, other) { + (Sharpness::Exact(a), Sharpness::Exact(b)) => Sharpness::Exact(a + b), + (Sharpness::Inexact(a), Sharpness::Exact(b)) + | (Sharpness::Exact(a), Sharpness::Inexact(b)) + | (Sharpness::Inexact(a), Sharpness::Inexact(b)) => Sharpness::Inexact(a + b), + (_, _) => Sharpness::Absent, + } + } + + pub fn sub(&self, other: &Sharpness) -> Sharpness { + match (self, other) { + (Sharpness::Exact(a), Sharpness::Exact(b)) => Sharpness::Exact(a - b), + (Sharpness::Inexact(a), Sharpness::Exact(b)) + | (Sharpness::Exact(a), Sharpness::Inexact(b)) + | (Sharpness::Inexact(a), Sharpness::Inexact(b)) => Sharpness::Inexact(a - b), + (_, _) => Sharpness::Absent, + } + } + + pub fn multiply(&self, other: &Sharpness) -> Sharpness { + match (self, other) { + (Sharpness::Exact(a), Sharpness::Exact(b)) => Sharpness::Exact(a * b), + (Sharpness::Inexact(a), Sharpness::Exact(b)) + | (Sharpness::Exact(a), Sharpness::Inexact(b)) + | (Sharpness::Inexact(a), Sharpness::Inexact(b)) => Sharpness::Inexact(a * b), + (_, _) => Sharpness::Absent, + } + } +} + +impl Debug + for Sharpness +{ + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Sharpness::Exact(inner) => write!(f, "Exact Info:({:?})", inner), + Sharpness::Inexact(inner) => write!(f, "Inexact Info:({:?})", inner), + Sharpness::Absent => write!(f, "Absent Info"), + } + } +} + +impl Display + for Sharpness +{ + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + Sharpness::Exact(inner) => write!(f, "Exact Info:({})", inner), + Sharpness::Inexact(inner) => write!(f, "Inexact Info:({})", inner), + Sharpness::Absent => write!(f, "Absent Info"), + } + } +} + +impl Default for Sharpness +where + T: Debug + Clone + PartialEq + Eq + Display + PartialOrd, +{ + fn default() -> Self { + Sharpness::Absent + } +} /// Statistics for a relation /// Fields are optional and can be inexact because the sources @@ -29,15 +157,11 @@ use crate::ScalarValue; #[derive(Debug, Clone, PartialEq, Eq)] pub struct Statistics { /// The number of table rows - pub num_rows: Option, + pub num_rows: Sharpness, /// total bytes of the table rows - pub total_byte_size: Option, + pub total_byte_size: Sharpness, /// Statistics on a column level pub column_statistics: Vec, - /// If true, any field that is `Some(..)` is the actual value in the data provided by the operator (it is not - /// an estimate). Any or all other fields might still be None, in which case no information is known. - /// if false, any field that is `Some(..)` may contain an inexact estimate and may not be the actual value. - pub is_exact: bool, } impl Statistics { @@ -46,10 +170,9 @@ impl Statistics { /// known to give an opportunity to the current executor to shrink the bounds of some columns. pub fn new_with_unbounded_columns(schema: &Schema) -> Self { Self { - num_rows: None, - total_byte_size: None, + num_rows: Sharpness::Absent, + total_byte_size: Sharpness::Absent, column_statistics: Statistics::unbounded_column_statistics(schema), - is_exact: false, } } @@ -61,52 +184,61 @@ impl Statistics { .map(|field| { let inf = ScalarValue::try_from(field.data_type()).ok(); ColumnStatistics { - null_count: None, - max_value: inf.clone(), - min_value: inf, - distinct_count: None, + null_count: Sharpness::Absent, + max_value: inf + .clone() + .map(|val| Sharpness::Inexact(val)) + .unwrap_or(Sharpness::Absent), + min_value: inf + .map(|val| Sharpness::Inexact(val)) + .unwrap_or(Sharpness::Absent), + distinct_count: Sharpness::Absent, } }) .collect() } + + pub fn all_exact(&self) -> bool { + self.num_rows.is_exact().unwrap_or(false) + && self.total_byte_size.is_exact().unwrap_or(false) + && self.column_statistics.iter().all(|col_stat| { + col_stat.distinct_count.is_exact().unwrap_or(false) + && col_stat.max_value.is_exact().unwrap_or(false) + && col_stat.min_value.is_exact().unwrap_or(false) + && col_stat.null_count.is_exact().unwrap_or(false) + }) + } } impl Display for Statistics { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - if self.num_rows.is_none() && self.total_byte_size.is_none() && !self.is_exact { - return Ok(()); - } - - let rows = self - .num_rows - .map_or_else(|| "None".to_string(), |v| v.to_string()); - let bytes = self - .total_byte_size - .map_or_else(|| "None".to_string(), |v| v.to_string()); - - write!(f, "rows={}, bytes={}, exact={}", rows, bytes, self.is_exact)?; + write!( + f, + "Number of Rows={}, Number of Bytes={}, Columns Statistics={:?}", + self.num_rows, self.total_byte_size, self.column_statistics + )?; Ok(()) } } /// Statistics for a column within a relation -#[derive(Clone, Debug, Default, PartialEq, Eq)] +#[derive(Clone, Debug, PartialEq, Eq, Default)] pub struct ColumnStatistics { /// Number of null values on column - pub null_count: Option, + pub null_count: Sharpness, /// Maximum value of column - pub max_value: Option, + pub max_value: Sharpness, /// Minimum value of column - pub min_value: Option, + pub min_value: Sharpness, /// Number of distinct values - pub distinct_count: Option, + pub distinct_count: Sharpness, } impl ColumnStatistics { /// Column contains a single non null value (e.g constant). pub fn is_singleton(&self) -> bool { - match (&self.min_value, &self.max_value) { + match (&self.min_value.get_value(), &self.max_value.get_value()) { // Min and max values are the same and not infinity. (Some(min), Some(max)) => !min.is_null() && !max.is_null() && (min == max), (_, _) => false, @@ -117,10 +249,15 @@ impl ColumnStatistics { pub fn new_with_unbounded_column(dt: &DataType) -> ColumnStatistics { let null = ScalarValue::try_from(dt.clone()).ok(); ColumnStatistics { - null_count: None, - max_value: null.clone(), - min_value: null, - distinct_count: None, + null_count: Sharpness::Absent, + max_value: null + .clone() + .map(|val| Sharpness::Inexact(val)) + .unwrap_or(Sharpness::Absent), + min_value: null + .map(|val| Sharpness::Inexact(val)) + .unwrap_or(Sharpness::Absent), + distinct_count: Sharpness::Absent, } } } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 92fff6a425ca..1aaf30fdedf6 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -17,6 +17,7 @@ //! Parquet format abstractions +use datafusion_common::stats::Sharpness; use parquet::column::writer::ColumnCloseResult; use parquet::file::writer::SerializedFileWriter; use rand::distributions::DistString; @@ -508,7 +509,7 @@ async fn fetch_statistics( let mut num_rows = 0; let mut total_byte_size = 0; - let mut null_counts = vec![0; num_fields]; + let mut null_counts = vec![Sharpness::Exact(0); num_fields]; let mut has_statistics = false; let schema_adapter = SchemaAdapter::new(table_schema.clone()); @@ -534,7 +535,7 @@ async fn fetch_statistics( schema_adapter.map_column_index(table_idx, &file_schema) { if let Some((null_count, stats)) = column_stats.get(&file_idx) { - *null_cnt += *null_count as usize; + *null_cnt = null_cnt.add(&Sharpness::Exact(*null_count as usize)); summarize_min_max( &mut max_values, &mut min_values, @@ -548,7 +549,7 @@ async fn fetch_statistics( min_values[table_idx] = None; } } else { - *null_cnt += num_rows as usize; + *null_cnt = null_cnt.add(&Sharpness::Exact(num_rows as usize)); } } } @@ -561,10 +562,9 @@ async fn fetch_statistics( }; let statistics = Statistics { - num_rows: Some(num_rows as usize), - total_byte_size: Some(total_byte_size as usize), + num_rows: Sharpness::Exact(num_rows as usize), + total_byte_size: Sharpness::Exact(total_byte_size as usize), column_statistics: column_stats, - is_exact: true, }; Ok(statistics) @@ -1160,20 +1160,26 @@ mod tests { let stats = fetch_statistics(store.as_ref(), schema.clone(), &meta[0], None).await?; - assert_eq!(stats.num_rows, Some(3)); + assert_eq!(stats.num_rows, Sharpness::Exact(3)); let c1_stats = &stats.column_statistics[0]; let c2_stats = &stats.column_statistics[1]; - assert_eq!(c1_stats.null_count, Some(1)); - assert_eq!(c2_stats.null_count, Some(3)); + assert_eq!(c1_stats.null_count, Sharpness::Exact(1)); + assert_eq!(c2_stats.null_count, Sharpness::Exact(3)); let stats = fetch_statistics(store.as_ref(), schema, &meta[1], None).await?; - assert_eq!(stats.num_rows, Some(3)); + assert_eq!(stats.num_rows, Sharpness::Exact(3)); let c1_stats = &stats.column_statistics[0]; let c2_stats = &stats.column_statistics[1]; - assert_eq!(c1_stats.null_count, Some(3)); - assert_eq!(c2_stats.null_count, Some(1)); - assert_eq!(c2_stats.max_value, Some(ScalarValue::Int64(Some(2)))); - assert_eq!(c2_stats.min_value, Some(ScalarValue::Int64(Some(1)))); + assert_eq!(c1_stats.null_count, Sharpness::Exact(3)); + assert_eq!(c2_stats.null_count, Sharpness::Exact(1)); + assert_eq!( + c2_stats.max_value, + Sharpness::Exact(ScalarValue::Int64(Some(2))) + ); + assert_eq!( + c2_stats.min_value, + Sharpness::Exact(ScalarValue::Int64(Some(1))) + ); Ok(()) } @@ -1309,11 +1315,11 @@ mod tests { fetch_statistics(store.upcast().as_ref(), schema.clone(), &meta[0], Some(9)) .await?; - assert_eq!(stats.num_rows, Some(3)); + assert_eq!(stats.num_rows, Sharpness::Exact(3)); let c1_stats = &stats.column_statistics[0]; let c2_stats = &stats.column_statistics[1]; - assert_eq!(c1_stats.null_count, Some(1)); - assert_eq!(c2_stats.null_count, Some(3)); + assert_eq!(c1_stats.null_count, Sharpness::Exact(1)); + assert_eq!(c2_stats.null_count, Sharpness::Exact(3)); let store = Arc::new(RequestCountingObjectStore::new(Arc::new( LocalFileSystem::new(), @@ -1342,11 +1348,11 @@ mod tests { ) .await?; - assert_eq!(stats.num_rows, Some(3)); + assert_eq!(stats.num_rows, Sharpness::Exact(3)); let c1_stats = &stats.column_statistics[0]; let c2_stats = &stats.column_statistics[1]; - assert_eq!(c1_stats.null_count, Some(1)); - assert_eq!(c2_stats.null_count, Some(3)); + assert_eq!(c1_stats.null_count, Sharpness::Exact(1)); + assert_eq!(c2_stats.null_count, Sharpness::Exact(3)); let store = Arc::new(RequestCountingObjectStore::new(Arc::new( LocalFileSystem::new(), @@ -1386,8 +1392,8 @@ mod tests { assert_eq!(tt_batches, 4 /* 8/2 */); // test metadata - assert_eq!(exec.statistics()?.num_rows, Some(8)); - assert_eq!(exec.statistics()?.total_byte_size, Some(671)); + assert_eq!(exec.statistics()?.num_rows, Sharpness::Exact(8)); + assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Exact(671)); Ok(()) } @@ -1428,9 +1434,8 @@ mod tests { get_exec(&state, "alltypes_plain.parquet", projection, Some(1)).await?; // note: even if the limit is set, the executor rounds up to the batch size - assert_eq!(exec.statistics()?.num_rows, Some(8)); - assert_eq!(exec.statistics()?.total_byte_size, Some(671)); - assert!(exec.statistics()?.is_exact); + assert_eq!(exec.statistics()?.num_rows, Sharpness::Exact(8)); + assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Exact(671)); let batches = collect(exec, task_ctx).await?; assert_eq!(1, batches.len()); assert_eq!(11, batches[0].num_columns()); 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 ff5980071b16..acf9f557e166 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -32,7 +32,7 @@ use arrow::buffer::Buffer; use arrow::datatypes::{ArrowNativeType, UInt16Type}; use arrow_array::{ArrayRef, DictionaryArray, RecordBatch}; use arrow_schema::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::exec_err; +use datafusion_common::{exec_err, stats::Sharpness}; use datafusion_common::{ColumnStatistics, Statistics}; use datafusion_physical_expr::LexOrdering; @@ -145,9 +145,8 @@ impl FileScanConfig { let table_stats = Statistics { num_rows: self.statistics.num_rows, - is_exact: self.statistics.is_exact, // TODO correct byte size? - total_byte_size: None, + total_byte_size: Sharpness::Absent, column_statistics: table_cols_stats, }; @@ -536,17 +535,16 @@ mod tests { Arc::clone(&file_schema), Some(vec![file_schema.fields().len(), 0]), Statistics { - num_rows: Some(10), + num_rows: Sharpness::Inexact(10), // assign the column index to distinct_count to help assert // the source statistic after the projection column_statistics: (0..file_schema.fields().len()) .map(|i| ColumnStatistics { - distinct_count: Some(i), + distinct_count: Sharpness::Inexact(i), ..Default::default() }) .collect(), - total_byte_size: None, - is_exact: false, + total_byte_size: Sharpness::Absent, }, vec![( "date".to_owned(), @@ -563,7 +561,7 @@ mod tests { assert_eq!(proj_stat_cols.len(), 2); // TODO implement tests for proj_stat_cols[0] once partition column // statistics are implemented - assert_eq!(proj_stat_cols[1].distinct_count, Some(0)); + assert_eq!(proj_stat_cols[1].distinct_count, Sharpness::Inexact(0)); let col_names = conf.projected_file_column_names(); assert_eq!(col_names, Some(vec!["c1".to_owned()])); diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 54545a403480..895cf587c5d5 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -19,6 +19,7 @@ use crate::arrow::datatypes::{Schema, SchemaRef}; use crate::error::Result; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; use crate::physical_plan::{Accumulator, ColumnStatistics, Statistics}; +use datafusion_common::stats::Sharpness; use futures::Stream; use futures::StreamExt; @@ -34,7 +35,7 @@ pub async fn get_statistics_with_limit( ) -> Result<(Vec, Statistics)> { let mut result_files = vec![]; - let mut null_counts = vec![0; file_schema.fields().len()]; + let mut null_counts = vec![Sharpness::Exact(0 as usize); file_schema.fields().len()]; let mut has_statistics = false; let (mut max_values, mut min_values) = create_max_min_accs(&file_schema); @@ -44,32 +45,31 @@ pub async fn get_statistics_with_limit( // at least one file has them. If none of the files provide them, then they // will be omitted from the statistics. The missing values will be counted // as zero. - let mut num_rows = None; - let mut total_byte_size = None; + let mut num_rows = Sharpness::Absent; + let mut total_byte_size = Sharpness::Absent; // fusing the stream allows us to call next safely even once it is finished let mut all_files = Box::pin(all_files.fuse()); while let Some(res) = all_files.next().await { let (file, file_stats) = res?; result_files.push(file); - is_exact &= file_stats.is_exact; - num_rows = if let Some(num_rows) = num_rows { - Some(num_rows + file_stats.num_rows.unwrap_or(0)) + num_rows = if num_rows.get_value().is_some() { + num_rows.add(&file_stats.num_rows) } else { file_stats.num_rows }; - total_byte_size = if let Some(total_byte_size) = total_byte_size { - Some(total_byte_size + file_stats.total_byte_size.unwrap_or(0)) + total_byte_size = if total_byte_size.get_value().is_some() { + total_byte_size.add(&file_stats.total_byte_size) } else { file_stats.total_byte_size }; has_statistics = true; for (i, cs) in file_stats.column_statistics.iter().enumerate() { - null_counts[i] += cs.null_count.unwrap_or(0); + null_counts[i] = null_counts[i].add(&cs.null_count); if let Some(max_value) = &mut max_values[i] { - if let Some(file_max) = cs.max_value.clone() { + if let Some(file_max) = cs.max_value.get_value() { match max_value.update_batch(&[file_max.to_array()]) { Ok(_) => {} Err(_) => { @@ -82,7 +82,7 @@ pub async fn get_statistics_with_limit( } if let Some(min_value) = &mut min_values[i] { - if let Some(file_min) = cs.min_value.clone() { + if let Some(file_min) = cs.min_value.get_value() { match min_value.update_batch(&[file_min.to_array()]) { Ok(_) => {} Err(_) => { @@ -99,8 +99,13 @@ pub async fn get_statistics_with_limit( // files. This only applies when we know the number of rows. It also // currently ignores tables that have no statistics regarding the // number of rows. - if num_rows.unwrap_or(usize::MIN) > limit.unwrap_or(usize::MAX) { - break; + match (num_rows.get_value(), limit) { + (Some(nr), Some(limit)) => { + if nr > limit { + break; + } + } + _ => break, } } // if we still have files in the stream, it means that the limit kicked @@ -120,7 +125,6 @@ pub async fn get_statistics_with_limit( num_rows, total_byte_size, column_statistics: column_stats, - is_exact, }; Ok((result_files, statistics)) @@ -144,7 +148,7 @@ pub(crate) fn create_max_min_accs( pub(crate) fn get_col_stats( schema: &Schema, - null_counts: Vec, + null_counts: Vec>, max_values: &mut [Option], min_values: &mut [Option], ) -> Vec { @@ -159,10 +163,14 @@ pub(crate) fn get_col_stats( None => None, }; ColumnStatistics { - null_count: Some(null_counts[i]), - max_value, - min_value, - distinct_count: None, + null_count: null_counts[i], + max_value: max_value + .map(|val| Sharpness::Exact(val)) + .unwrap_or(Sharpness::Absent), + min_value: min_value + .map(|val| Sharpness::Exact(val)) + .unwrap_or(Sharpness::Absent), + distinct_count: Sharpness::Absent, } }) .collect() diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 9ef5157b9a34..8e02e863fbcd 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -19,6 +19,7 @@ use std::sync::Arc; use crate::config::ConfigOptions; +use datafusion_common::stats::Sharpness; use datafusion_common::tree_node::TreeNode; use datafusion_expr::utils::COUNT_STAR_EXPANSION; @@ -126,7 +127,7 @@ fn take_optimizable(node: &dyn ExecutionPlan) -> Result Option<(ScalarValue, &'static str)> { if let (Some(num_rows), Some(casted_expr)) = ( - stats.num_rows, + stats.num_rows.get_value(), agg_expr.as_any().downcast_ref::(), ) { // TODO implementing Eq on PhysicalExpr would help a lot here @@ -176,7 +177,7 @@ fn take_optimizable_column_count( ) -> Option<(ScalarValue, String)> { let col_stats = &stats.column_statistics; if let (Some(num_rows), Some(casted_expr)) = ( - stats.num_rows, + stats.num_rows.get_value(), agg_expr.as_any().downcast_ref::(), ) { if casted_expr.expressions().len() == 1 { @@ -185,11 +186,7 @@ fn take_optimizable_column_count( .as_any() .downcast_ref::() { - if let ColumnStatistics { - null_count: Some(val), - .. - } = &col_stats[col_expr.index()] - { + if let Some(val) = &col_stats[col_expr.index()].null_count.get_value() { return Some(( ScalarValue::Int64(Some((num_rows - val) as i64)), casted_expr.name().to_string(), @@ -214,11 +211,7 @@ fn take_optimizable_min( .as_any() .downcast_ref::() { - if let ColumnStatistics { - min_value: Some(val), - .. - } = &col_stats[col_expr.index()] - { + if let Some(val) = &col_stats[col_expr.index()].min_value.get_value() { // Exclude the unbounded case // As safest estimate, -inf, and + inf is used as bound of the column // If minimum is -inf, it is not exact. Hence we shouldn't do optimization @@ -246,11 +239,7 @@ fn take_optimizable_max( .as_any() .downcast_ref::() { - if let ColumnStatistics { - max_value: Some(val), - .. - } = &col_stats[col_expr.index()] - { + if let Some(val) = &col_stats[col_expr.index()].max_value.get_value() { // Exclude the unbounded case if !val.is_null() { return Some((val.clone(), casted_expr.name().to_string())); diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 856741bf392c..07ed3e2b2dd0 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -71,9 +71,15 @@ fn should_swap_join_order( let right_stats = right.statistics()?; // First compare `total_byte_size` of left and right side, // if information in this field is insufficient fallback to the `num_rows` - match (left_stats.total_byte_size, right_stats.total_byte_size) { + match ( + left_stats.total_byte_size.get_value(), + right_stats.total_byte_size.get_value(), + ) { (Some(l), Some(r)) => Ok(l > r), - _ => match (left_stats.num_rows, right_stats.num_rows) { + _ => match ( + left_stats.num_rows.get_value(), + right_stats.num_rows.get_value(), + ) { (Some(l), Some(r)) => Ok(l > r), _ => Ok(false), }, @@ -91,9 +97,9 @@ fn supports_collect_by_size( } else { return false; }; - if let Some(size) = stats.total_byte_size { + if let Some(size) = stats.total_byte_size.get_value() { size != 0 && size < collection_size_threshold - } else if let Some(row_count) = stats.num_rows { + } else if let Some(row_count) = stats.num_rows.get_value() { row_count != 0 && row_count < collection_size_threshold } else { false @@ -597,31 +603,29 @@ mod tests_statistical { use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::{JoinType, ScalarValue}; + use datafusion_common::{stats::Sharpness, JoinType, ScalarValue}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::PhysicalExpr; fn create_big_and_small() -> (Arc, Arc) { let big = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(10), - total_byte_size: Some(100000), + num_rows: Sharpness::Inexact(10), + total_byte_size: Sharpness::Inexact(100000), column_statistics: vec![ColumnStatistics::new_with_unbounded_column( &DataType::Int32, )], - is_exact: false, }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), )); let small = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(100000), - total_byte_size: Some(10), + num_rows: Sharpness::Inexact(100000), + total_byte_size: Sharpness::Inexact(10), column_statistics: vec![ColumnStatistics::new_with_unbounded_column( &DataType::Int32, )], - is_exact: false, }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), )); @@ -639,9 +643,15 @@ mod tests_statistical { distinct_count: Option, ) -> Vec { vec![ColumnStatistics { - distinct_count, - min_value: min.map(|size| ScalarValue::UInt64(Some(size))), - max_value: max.map(|size| ScalarValue::UInt64(Some(size))), + distinct_count: distinct_count + .map(|val| Sharpness::Inexact(val)) + .unwrap_or(Sharpness::Absent), + min_value: min + .map(|size| Sharpness::Inexact(ScalarValue::UInt64(Some(size)))) + .unwrap_or(Sharpness::Absent), + max_value: max + .map(|size| Sharpness::Inexact(ScalarValue::UInt64(Some(size)))) + .unwrap_or(Sharpness::Absent), ..Default::default() }] } @@ -657,42 +667,39 @@ mod tests_statistical { ) { let big = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(100_000), + num_rows: Sharpness::Inexact(100_000), column_statistics: create_column_stats( Some(0), Some(50_000), Some(50_000), ), - total_byte_size: None, - is_exact: false, + total_byte_size: Sharpness::Absent, }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), )); let medium = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(10_000), + num_rows: Sharpness::Inexact(0_000), column_statistics: create_column_stats( Some(1000), Some(5000), Some(1000), ), - total_byte_size: None, - is_exact: false, + total_byte_size: Sharpness::Absent, }, Schema::new(vec![Field::new("medium_col", DataType::Int32, false)]), )); let small = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(1000), + num_rows: Sharpness::Inexact(1000), column_statistics: create_column_stats( Some(0), Some(100_000), Some(1000), ), - total_byte_size: None, - is_exact: false, + total_byte_size: Sharpness::Absent, }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), )); @@ -743,11 +750,11 @@ mod tests_statistical { assert_eq!( swapped_join.left().statistics().unwrap().total_byte_size, - Some(10) + Sharpness::Inexact(10) ); assert_eq!( swapped_join.right().statistics().unwrap().total_byte_size, - Some(100000) + Sharpness::Inexact(100000) ); } @@ -794,11 +801,11 @@ mod tests_statistical { assert_eq!( swapped_join.left().statistics().unwrap().total_byte_size, - Some(100000) + Sharpness::Inexact(100000) ); assert_eq!( swapped_join.right().statistics().unwrap().total_byte_size, - Some(10) + Sharpness::Inexact(10) ); } @@ -839,11 +846,11 @@ mod tests_statistical { assert_eq!( swapped_join.left().statistics().unwrap().total_byte_size, - Some(10) + Sharpness::Inexact(10) ); assert_eq!( swapped_join.right().statistics().unwrap().total_byte_size, - Some(100000) + Sharpness::Inexact(100000) ); assert_eq!(original_schema, swapped_join.schema()); @@ -954,11 +961,11 @@ mod tests_statistical { assert_eq!( swapped_join.left().statistics().unwrap().total_byte_size, - Some(10) + Sharpness::Inexact(10) ); assert_eq!( swapped_join.right().statistics().unwrap().total_byte_size, - Some(100000) + Sharpness::Inexact(100000) ); } @@ -1001,36 +1008,33 @@ mod tests_statistical { async fn test_join_selection_collect_left() { let big = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(10000000), - total_byte_size: Some(10000000), + num_rows: Sharpness::Inexact(10000000), + total_byte_size: Sharpness::Inexact(10000000), column_statistics: vec![ColumnStatistics::new_with_unbounded_column( &DataType::Int32, )], - is_exact: false, }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), )); let small = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(10), - total_byte_size: Some(10), + num_rows: Sharpness::Inexact(10), + total_byte_size: Sharpness::Inexact(10), column_statistics: vec![ColumnStatistics::new_with_unbounded_column( &DataType::Int32, )], - is_exact: false, }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), )); let empty = Arc::new(StatisticsExec::new( Statistics { - num_rows: None, - total_byte_size: None, + num_rows: Sharpness::Absent, + total_byte_size: Sharpness::Absent, column_statistics: vec![ColumnStatistics::new_with_unbounded_column( &DataType::Int32, )], - is_exact: false, }, Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), )); @@ -1088,36 +1092,33 @@ mod tests_statistical { async fn test_join_selection_partitioned() { let big1 = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(10000000), - total_byte_size: Some(10000000), + num_rows: Sharpness::Inexact(10000000), + total_byte_size: Sharpness::Inexact(10000000), column_statistics: vec![ColumnStatistics::new_with_unbounded_column( &DataType::Int32, )], - is_exact: false, }, Schema::new(vec![Field::new("big_col1", DataType::Int32, false)]), )); let big2 = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(20000000), - total_byte_size: Some(20000000), + num_rows: Sharpness::Inexact(20000000), + total_byte_size: Sharpness::Inexact(20000000), column_statistics: vec![ColumnStatistics::new_with_unbounded_column( &DataType::Int32, )], - is_exact: false, }, Schema::new(vec![Field::new("big_col2", DataType::Int32, false)]), )); let empty = Arc::new(StatisticsExec::new( Statistics { - num_rows: None, - total_byte_size: None, + num_rows: Sharpness::Absent, + total_byte_size: Sharpness::Absent, column_statistics: vec![ColumnStatistics::new_with_unbounded_column( &DataType::Int32, )], - is_exact: false, }, Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), )); diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index d7c91b61da8b..24f60387ead7 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -22,6 +22,7 @@ use std::sync::Arc; use arrow::datatypes::Schema; +use datafusion_common::stats::Sharpness; use datafusion_common::{ColumnStatistics, DataFusionError, Result, ScalarValue}; use crate::expressions::Column; @@ -81,7 +82,7 @@ pub struct ExprBoundaries { /// Minimum and maximum values this expression can have. pub interval: Interval, /// Maximum number of distinct values this expression can produce, if known. - pub distinct_count: Option, + pub distinct_count: Sharpness, } impl ExprBoundaries { @@ -91,13 +92,13 @@ impl ExprBoundaries { column: Column::new(&col, index), interval: Interval::new( IntervalBound::new_closed( - stats.min_value.clone().unwrap_or(ScalarValue::Null), + stats.min_value.get_value().unwrap_or(ScalarValue::Null), ), IntervalBound::new_closed( - stats.max_value.clone().unwrap_or(ScalarValue::Null), + stats.max_value.get_value().unwrap_or(ScalarValue::Null), ), ), - distinct_count: stats.distinct_count, + distinct_count: stats.distinct_count.clone(), } } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 089ed1cf8922..820d92a20964 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -29,6 +29,7 @@ use crate::{ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; +use datafusion_common::stats::Sharpness; use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; @@ -1013,18 +1014,16 @@ impl ExecutionPlan for AggregateExec { if self.group_by.expr.is_empty() => { Ok(Statistics { - num_rows: Some(1), - is_exact: true, + num_rows: Sharpness::Exact(1), column_statistics, - total_byte_size: None, + total_byte_size: Sharpness::Absent, }) } _ => Ok(Statistics { // the output row count is surely not larger than its input row count num_rows: self.input.statistics()?.num_rows, - is_exact: false, column_statistics, - total_byte_size: None, + total_byte_size: Sharpness::Absent, }), } } diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index aaef07ef9363..cef96242aca4 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -23,6 +23,7 @@ use crate::{ColumnStatistics, ExecutionPlan, Statistics}; use arrow::datatypes::Schema; use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; use arrow::record_batch::RecordBatch; +use datafusion_common::stats::Sharpness; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; use datafusion_physical_expr::expressions::{BinaryExpr, Column}; @@ -155,17 +156,25 @@ pub fn compute_record_batch_statistics( for partition in batches.iter() { for batch in partition { for (stat_index, col_index) in projection.iter().enumerate() { - *column_statistics[stat_index].null_count.get_or_insert(0) += - batch.column(*col_index).null_count(); + column_statistics[stat_index].null_count = if let Sharpness::Exact(val) = + &column_statistics[stat_index].null_count + { + Sharpness::Exact(batch.column(*col_index).null_count() + val) + } else if let Sharpness::Inexact(val) = + &column_statistics[stat_index].null_count + { + Sharpness::Inexact(batch.column(*col_index).null_count() + val) + } else { + Sharpness::Exact(0) + }; } } } Statistics { - num_rows: Some(nb_rows), - total_byte_size: Some(total_byte_size), + num_rows: Sharpness::Exact(nb_rows), + total_byte_size: Sharpness::Exact(total_byte_size), column_statistics, - is_exact: true, } } @@ -676,9 +685,8 @@ mod tests { ])); let stats = compute_record_batch_statistics(&[], &schema, Some(vec![0, 1])); - assert_eq!(stats.num_rows, Some(0)); - assert!(stats.is_exact); - assert_eq!(stats.total_byte_size, Some(0)); + assert_eq!(stats.num_rows, Sharpness::Exact(0)); + assert_eq!(stats.total_byte_size, Sharpness::Exact(0)); Ok(()) } @@ -699,21 +707,20 @@ mod tests { compute_record_batch_statistics(&[vec![batch]], &schema, Some(vec![0, 1])); let mut expected = Statistics { - is_exact: true, - num_rows: Some(3), - total_byte_size: Some(464), // this might change a bit if the way we compute the size changes + num_rows: Sharpness::Exact(3), + total_byte_size: Sharpness::Exact(464), // this might change a bit if the way we compute the size changes column_statistics: vec![ ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Float32(None)), - min_value: Some(ScalarValue::Float32(None)), - null_count: Some(0), + distinct_count: Sharpness::Absent, + max_value: Sharpness::Exact(ScalarValue::Float32(None)), + min_value: Sharpness::Exact(ScalarValue::Float32(None)), + null_count: Sharpness::Exact(0), }, ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Float64(None)), - min_value: Some(ScalarValue::Float64(None)), - null_count: Some(0), + distinct_count: Sharpness::Absent, + max_value: Sharpness::Exact(ScalarValue::Float64(None)), + min_value: Sharpness::Exact(ScalarValue::Float64(None)), + null_count: Sharpness::Exact(0), }, ], }; diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index f296850f83f3..7e6b1d2b3134 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -37,6 +37,7 @@ use arrow::compute::filter_record_batch; use arrow::datatypes::{DataType, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::cast::as_boolean_array; +use datafusion_common::stats::Sharpness; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Operator; @@ -228,14 +229,12 @@ impl ExecutionPlan for FilterExec { num_rows, total_byte_size, column_statistics, - is_exact: false, }) } else { Ok(Statistics { num_rows, total_byte_size, column_statistics: input_stats.column_statistics, - is_exact: false, }) } } @@ -263,9 +262,17 @@ fn collect_new_statistics( )| { let closed_interval = interval.close_bounds(); ColumnStatistics { - null_count: input_column_stats[idx].null_count, - max_value: Some(closed_interval.upper.value), - min_value: Some(closed_interval.lower.value), + null_count: input_column_stats[idx].null_count.clone(), + max_value: if input_column_stats[idx].max_value.is_exact().is_some() { + Sharpness::Exact(closed_interval.upper.value) + } else { + Sharpness::Inexact(closed_interval.upper.value) + }, + min_value: if input_column_stats[idx].max_value.is_exact().is_some() { + Sharpness::Exact(closed_interval.lower.value) + } else { + Sharpness::Inexact(closed_interval.lower.value) + }, distinct_count, } }, @@ -446,14 +453,13 @@ mod tests { let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(100), - total_byte_size: Some(100 * bytes_per_row), + num_rows: Sharpness::Inexact(100), + total_byte_size: Sharpness::Inexact(100 * bytes_per_row), column_statistics: vec![ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }], - is_exact: false, }, schema.clone(), )); @@ -467,13 +473,16 @@ mod tests { Arc::new(FilterExec::try_new(predicate, input)?); let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Some(25)); - assert_eq!(statistics.total_byte_size, Some(25 * bytes_per_row)); + assert_eq!(statistics.num_rows, Sharpness::Inexact(25)); + assert_eq!( + statistics.total_byte_size, + Sharpness::Inexact(25 * bytes_per_row) + ); assert_eq!( statistics.column_statistics, vec![ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(25))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(25))), ..Default::default() }] ); @@ -488,14 +497,13 @@ mod tests { let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(100), + num_rows: Sharpness::Inexact(100), column_statistics: vec![ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }], - is_exact: false, - total_byte_size: None, + total_byte_size: Sharpness::Absent, }, schema.clone(), )); @@ -515,12 +523,12 @@ mod tests { )?); let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Some(16)); + assert_eq!(statistics.num_rows, Sharpness::Inexact(16)); assert_eq!( statistics.column_statistics, vec![ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(10))), - max_value: Some(ScalarValue::Int32(Some(25))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(10))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(25))), ..Default::default() }] ); @@ -539,21 +547,20 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(100), + num_rows: Sharpness::Inexact(100), column_statistics: vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(50))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(50))), ..Default::default() }, ], - is_exact: false, - total_byte_size: None, + total_byte_size: Sharpness::Absent, }, schema.clone(), )); @@ -582,18 +589,18 @@ mod tests { // // Which would result with a selectivity of '15/100 * 5/50' or 0.015 // and that means about %1.5 of the all rows (rounded up to 2 rows). - assert_eq!(statistics.num_rows, Some(2)); + assert_eq!(statistics.num_rows, Sharpness::Inexact(2)); assert_eq!( statistics.column_statistics, vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(10))), - max_value: Some(ScalarValue::Int32(Some(25))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(10))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(25))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(46))), - max_value: Some(ScalarValue::Int32(Some(50))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(46))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(50))), ..Default::default() } ] @@ -621,7 +628,7 @@ mod tests { Arc::new(FilterExec::try_new(predicate, input)?); let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, None); + assert_eq!(statistics.num_rows, Sharpness::Absent); Ok(()) } @@ -639,26 +646,25 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(1000), - total_byte_size: Some(4000), + num_rows: Sharpness::Inexact(1000), + total_byte_size: Sharpness::Inexact(4000), column_statistics: vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(3))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Float32(Some(1000.0))), - max_value: Some(ScalarValue::Float32(Some(1100.0))), + min_value: Sharpness::Inexact(ScalarValue::Float32(Some(1000.0))), + max_value: Sharpness::Inexact(ScalarValue::Float32(Some(1100.0))), ..Default::default() }, ], - is_exact: false, }, schema, )); @@ -698,22 +704,22 @@ mod tests { // 0.5 (from a) * 0.333333... (from b) * 0.798387... (from c) ≈ 0.1330... // num_rows after ceil => 133.0... => 134 // total_byte_size after ceil => 532.0... => 533 - assert_eq!(statistics.num_rows, Some(134)); - assert_eq!(statistics.total_byte_size, Some(533)); + assert_eq!(statistics.num_rows, Sharpness::Inexact(134)); + assert_eq!(statistics.total_byte_size, Sharpness::Inexact(533)); let exp_col_stats = vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(4))), - max_value: Some(ScalarValue::Int32(Some(53))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(4))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(53))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(3))), - max_value: Some(ScalarValue::Int32(Some(3))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(3))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Float32(Some(1000.0))), - max_value: Some(ScalarValue::Float32(Some(1075.0))), + min_value: Sharpness::Inexact(ScalarValue::Float32(Some(1000.0))), + max_value: Sharpness::Inexact(ScalarValue::Float32(Some(1075.0))), ..Default::default() }, ]; @@ -721,20 +727,24 @@ mod tests { .into_iter() .zip(statistics.column_statistics) .map(|(expected, actual)| { - if actual.min_value.clone().unwrap().data_type().is_floating() { - // Windows rounds arithmetic operation results differently for floating point numbers. - // Therefore, we check if the actual values are in an epsilon range. - let actual_min = actual.min_value.unwrap(); - let actual_max = actual.max_value.unwrap(); - let expected_min = expected.min_value.unwrap(); - let expected_max = expected.max_value.unwrap(); - let eps = ScalarValue::Float32(Some(1e-6)); - - assert!(actual_min.sub(&expected_min).unwrap() < eps); - assert!(actual_min.sub(&expected_min).unwrap() < eps); - - assert!(actual_max.sub(&expected_max).unwrap() < eps); - assert!(actual_max.sub(&expected_max).unwrap() < eps); + if let Some(val) = actual.min_value.get_value() { + if val.data_type().is_floating() { + // Windows rounds arithmetic operation results differently for floating point numbers. + // Therefore, we check if the actual values are in an epsilon range. + let actual_min = actual.min_value.get_value().unwrap(); + let actual_max = actual.max_value.get_value().unwrap(); + let expected_min = expected.min_value.get_value().unwrap(); + let expected_max = expected.max_value.get_value().unwrap(); + let eps = ScalarValue::Float32(Some(1e-6)); + + assert!(actual_min.sub(&expected_min).unwrap() < eps); + assert!(actual_min.sub(&expected_min).unwrap() < eps); + + assert!(actual_max.sub(&expected_max).unwrap() < eps); + assert!(actual_max.sub(&expected_max).unwrap() < eps); + } else { + assert_eq!(actual, expected); + } } else { assert_eq!(actual, expected); } @@ -754,21 +764,20 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(1000), - total_byte_size: Some(4000), + num_rows: Sharpness::Inexact(1000), + total_byte_size: Sharpness::Inexact(4000), column_statistics: vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(3))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, ], - is_exact: false, }, schema, )); @@ -792,8 +801,8 @@ mod tests { Arc::new(FilterExec::try_new(predicate, input)?); let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Some(1000)); - assert_eq!(statistics.total_byte_size, Some(4000)); + assert_eq!(statistics.num_rows, Sharpness::Inexact(1000)); + assert_eq!(statistics.total_byte_size, Sharpness::Inexact(4000)); assert_eq!(statistics.column_statistics, expected); Ok(()) @@ -810,21 +819,20 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(1000), - total_byte_size: Some(4000), + num_rows: Sharpness::Inexact(1000), + total_byte_size: Sharpness::Inexact(4000), column_statistics: vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(3))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, ], - is_exact: false, }, schema, )); @@ -846,19 +854,19 @@ mod tests { Arc::new(FilterExec::try_new(predicate, input)?); let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Some(0)); - assert_eq!(statistics.total_byte_size, Some(0)); + assert_eq!(statistics.num_rows, Sharpness::Inexact(0)); + assert_eq!(statistics.total_byte_size, Sharpness::Inexact(0)); assert_eq!( statistics.column_statistics, vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(3))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, ] @@ -875,21 +883,20 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Some(1000), - total_byte_size: Some(4000), + num_rows: Sharpness::Inexact(1000), + total_byte_size: Sharpness::Inexact(4000), column_statistics: vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ], - is_exact: false, }, schema, )); @@ -903,19 +910,19 @@ mod tests { Arc::new(FilterExec::try_new(predicate, input)?); let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Some(490)); - assert_eq!(statistics.total_byte_size, Some(1960)); + assert_eq!(statistics.num_rows, Sharpness::Inexact(490)); + assert_eq!(statistics.total_byte_size, Sharpness::Inexact(1960)); assert_eq!( statistics.column_statistics, vec![ ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(49))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(49))), ..Default::default() }, ColumnStatistics { - min_value: Some(ScalarValue::Int32(Some(1))), - max_value: Some(ScalarValue::Int32(Some(100))), + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ] @@ -954,15 +961,14 @@ mod tests { let filter_statistics = filter.statistics()?; let expected_filter_statistics = Statistics { - num_rows: None, - total_byte_size: None, + num_rows: Sharpness::Absent, + total_byte_size: Sharpness::Absent, column_statistics: vec![ColumnStatistics { - null_count: None, - min_value: Some(ScalarValue::Int32(Some(5))), - max_value: Some(ScalarValue::Int32(Some(10))), - distinct_count: None, + null_count: Sharpness::Absent, + min_value: Sharpness::Inexact(ScalarValue::Int32(Some(5))), + max_value: Sharpness::Inexact(ScalarValue::Int32(Some(10))), + distinct_count: Sharpness::Absent, }], - is_exact: false, }; assert_eq!(filter_statistics, expected_filter_statistics); diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index adb279203706..e05b762b13e7 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -18,6 +18,7 @@ //! Defines the cross join plan for loading the left side of the cross join //! and producing batches in parallel for the right partitions +use datafusion_common::stats::Sharpness; use futures::{ready, StreamExt}; use futures::{Stream, TryStreamExt}; use std::{any::Any, sync::Arc, task::Poll}; @@ -274,16 +275,12 @@ fn stats_cartesian_product( let right_row_count = right_stats.num_rows; // calculate global stats - let is_exact = left_stats.is_exact && right_stats.is_exact; - let num_rows = left_stats - .num_rows - .zip(right_stats.num_rows) - .map(|(a, b)| a * b); + let num_rows = left_row_count.multiply(&right_row_count); // the result size is two times a*b because you have the columns of both left and right let total_byte_size = left_stats .total_byte_size - .zip(right_stats.total_byte_size) - .map(|(a, b)| 2 * a * b); + .multiply(&right_stats.total_byte_size) + .multiply(&Sharpness::Exact(2)); let left_col_stats = left_stats.column_statistics; let right_col_stats = right_stats.column_statistics; @@ -293,13 +290,13 @@ fn stats_cartesian_product( let cross_join_stats = left_col_stats .into_iter() .map(|s| ColumnStatistics { - null_count: s.null_count.zip(right_row_count).map(|(a, b)| a * b), + null_count: s.null_count.multiply(&right_row_count), distinct_count: s.distinct_count, min_value: s.min_value, max_value: s.max_value, }) .chain(right_col_stats.into_iter().map(|s| ColumnStatistics { - null_count: s.null_count.zip(left_row_count).map(|(a, b)| a * b), + null_count: s.null_count.multiply(&left_row_count), distinct_count: s.distinct_count, min_value: s.min_value, max_value: s.max_value, @@ -307,7 +304,6 @@ fn stats_cartesian_product( .collect(); Statistics { - is_exact, num_rows, total_byte_size, column_statistics: cross_join_stats, @@ -467,61 +463,66 @@ mod tests { let right_bytes = 27; let left = Statistics { - is_exact: true, - num_rows: Some(left_row_count), - total_byte_size: Some(left_bytes), + num_rows: Sharpness::Exact(left_row_count), + total_byte_size: Sharpness::Exact(left_bytes), column_statistics: vec![ ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Sharpness::Exact(5), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), + null_count: Sharpness::Exact(0), }, ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: Some(3), + distinct_count: Sharpness::Exact(1), + max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Sharpness::Exact(3), }, ], }; let right = Statistics { - is_exact: true, - num_rows: Some(right_row_count), - total_byte_size: Some(right_bytes), + num_rows: Sharpness::Exact(right_row_count), + total_byte_size: Sharpness::Exact(right_bytes), column_statistics: vec![ColumnStatistics { - distinct_count: Some(3), - max_value: Some(ScalarValue::Int64(Some(12))), - min_value: Some(ScalarValue::Int64(Some(0))), - null_count: Some(2), + distinct_count: Sharpness::Exact(3), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(12))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(0))), + null_count: Sharpness::Exact(2), }], }; let result = stats_cartesian_product(left, right); let expected = Statistics { - is_exact: true, - num_rows: Some(left_row_count * right_row_count), - total_byte_size: Some(2 * left_bytes * right_bytes), + num_rows: Sharpness::Exact(left_row_count * right_row_count), + total_byte_size: Sharpness::Exact(2 * left_bytes * right_bytes), column_statistics: vec![ ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Sharpness::Exact(5), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), + null_count: Sharpness::Exact(0), }, ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: Some(3 * right_row_count), + distinct_count: Sharpness::Exact(1), + max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Sharpness::Exact(3 * right_row_count), }, ColumnStatistics { - distinct_count: Some(3), - max_value: Some(ScalarValue::Int64(Some(12))), - min_value: Some(ScalarValue::Int64(Some(0))), - null_count: Some(2 * left_row_count), + distinct_count: Sharpness::Exact(3), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(12))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(0))), + null_count: Sharpness::Exact(2 * left_row_count), }, ], }; @@ -534,61 +535,66 @@ mod tests { let left_row_count = 11; let left = Statistics { - is_exact: true, - num_rows: Some(left_row_count), - total_byte_size: Some(23), + num_rows: Sharpness::Exact(left_row_count), + total_byte_size: Sharpness::Exact(23), column_statistics: vec![ ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Sharpness::Exact(5), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), + null_count: Sharpness::Exact(0), }, ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: Some(3), + distinct_count: Sharpness::Exact(1), + max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Sharpness::Exact(3), }, ], }; let right = Statistics { - is_exact: true, - num_rows: None, // not defined! - total_byte_size: None, // not defined! + num_rows: Sharpness::Absent, + total_byte_size: Sharpness::Absent, column_statistics: vec![ColumnStatistics { - distinct_count: Some(3), - max_value: Some(ScalarValue::Int64(Some(12))), - min_value: Some(ScalarValue::Int64(Some(0))), - null_count: Some(2), + distinct_count: Sharpness::Exact(3), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(12))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(0))), + null_count: Sharpness::Exact(2), }], }; let result = stats_cartesian_product(left, right); let expected = Statistics { - is_exact: true, - num_rows: None, - total_byte_size: None, + num_rows: Sharpness::Absent, + total_byte_size: Sharpness::Absent, column_statistics: vec![ ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: None, // we don't know the row count on the right + distinct_count: Sharpness::Exact(5), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), + null_count: Sharpness::Absent, // we don't know the row count on the right }, ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: None, // we don't know the row count on the right + distinct_count: Sharpness::Exact(1), + max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Sharpness::Absent, // we don't know the row count on the right }, ColumnStatistics { - distinct_count: Some(3), - max_value: Some(ScalarValue::Int64(Some(12))), - min_value: Some(ScalarValue::Int64(Some(0))), - null_count: Some(2 * left_row_count), + distinct_count: Sharpness::Exact(3), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(12))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(0))), + null_count: Sharpness::Exact(2 * left_row_count), }, ], }; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 156d8d4f3e0b..56c389c445c4 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -39,6 +39,7 @@ use arrow::compute; use arrow::datatypes::{Field, Schema, SchemaBuilder}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::cast::as_boolean_array; +use datafusion_common::stats::Sharpness; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ exec_err, plan_err, DataFusionError, JoinType, Result, ScalarValue, SharedResult, @@ -733,14 +734,16 @@ pub(crate) fn estimate_join_statistics( let join_stats = estimate_join_cardinality(join_type, left_stats, right_stats, &on); let (num_rows, column_statistics) = match join_stats { - Some(stats) => (Some(stats.num_rows), stats.column_statistics), - None => (None, Statistics::unbounded_column_statistics(schema)), + Some(stats) => (Sharpness::Inexact(stats.num_rows), stats.column_statistics), + None => ( + Sharpness::Absent, + Statistics::unbounded_column_statistics(schema), + ), }; Ok(Statistics { num_rows, - total_byte_size: None, + total_byte_size: Sharpness::Absent, column_statistics, - is_exact: false, }) } @@ -753,29 +756,27 @@ fn estimate_join_cardinality( ) -> Option { match join_type { JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - let left_num_rows = left_stats.num_rows?; - let right_num_rows = right_stats.num_rows?; - - // Take the left_col_stats and right_col_stats using the index - // obtained from index() method of the each element of 'on'. - let all_left_col_stats = left_stats.column_statistics; - let all_right_col_stats = right_stats.column_statistics; let (left_col_stats, right_col_stats) = on .iter() .map(|(left, right)| { ( - all_left_col_stats[left.index()].clone(), - all_right_col_stats[right.index()].clone(), + left_stats.column_statistics[left.index()].clone(), + right_stats.column_statistics[right.index()].clone(), ) }) .unzip::<_, _, Vec<_>, Vec<_>>(); let ij_cardinality = estimate_inner_join_cardinality( - left_num_rows, - right_num_rows, - left_col_stats, - right_col_stats, - left_stats.is_exact && right_stats.is_exact, + Statistics { + num_rows: left_stats.num_rows.clone(), + total_byte_size: Sharpness::Absent, + column_statistics: left_col_stats, + }, + Statistics { + num_rows: right_stats.num_rows.clone(), + total_byte_size: Sharpness::Absent, + column_statistics: right_col_stats, + }, )?; // The cardinality for inner join can also be used to estimate @@ -784,25 +785,25 @@ fn estimate_join_cardinality( // joins (so that we don't underestimate the cardinality). let cardinality = match join_type { JoinType::Inner => ij_cardinality, - JoinType::Left => max(ij_cardinality, left_num_rows), - JoinType::Right => max(ij_cardinality, right_num_rows), - JoinType::Full => { - max(ij_cardinality, left_num_rows) - + max(ij_cardinality, right_num_rows) - - ij_cardinality - } + JoinType::Left => ij_cardinality.max(&left_stats.num_rows), + JoinType::Right => ij_cardinality.max(&right_stats.num_rows), + JoinType::Full => ij_cardinality + .max(&left_stats.num_rows) + .add(&ij_cardinality.max(&right_stats.num_rows)) + .sub(&ij_cardinality), _ => unreachable!(), }; Some(PartialJoinStatistics { - num_rows: cardinality, + num_rows: cardinality.get_value()?, // We don't do anything specific here, just combine the existing // statistics which might yield subpar results (although it is // true, esp regarding min/max). For a better estimation, we need // filter selectivity analysis first. - column_statistics: all_left_col_stats + column_statistics: left_stats + .column_statistics .into_iter() - .chain(all_right_col_stats) + .chain(right_stats.column_statistics) .collect(), }) } @@ -819,30 +820,45 @@ fn estimate_join_cardinality( /// a very conservative implementation that can quickly give up if there is not /// enough input statistics. fn estimate_inner_join_cardinality( - left_num_rows: usize, - right_num_rows: usize, - left_col_stats: Vec, - right_col_stats: Vec, - is_exact: bool, -) -> Option { + left_stats: Statistics, + right_stats: Statistics, +) -> Option> { // The algorithm here is partly based on the non-histogram selectivity estimation // from Spark's Catalyst optimizer. - - let mut join_selectivity = None; - for (left_stat, right_stat) in left_col_stats.iter().zip(right_col_stats.iter()) { - if (left_stat.min_value.clone()? > right_stat.max_value.clone()?) - || (left_stat.max_value.clone()? < right_stat.min_value.clone()?) - { - // If there is no overlap in any of the join columns, that means the join - // itself is disjoint and the cardinality is 0. Though we can only assume - // this when the statistics are exact (since it is a very strong assumption). - return if is_exact { Some(0) } else { None }; + let mut join_selectivity = Sharpness::Absent; + for (left_stat, right_stat) in left_stats + .column_statistics + .iter() + .zip(right_stats.column_statistics.iter()) + { + // If there is no overlap in any of the join columns, that means the join + // itself is disjoint and the cardinality is 0. Though we can only assume + // this when the statistics are exact (since it is a very strong assumption). + if left_stat.min_value.get_value()? > right_stat.max_value.get_value()? { + return match ( + left_stat.min_value.is_exact().unwrap(), + right_stat.max_value.is_exact().unwrap(), + ) { + (true, true) => Some(Sharpness::Exact(0)), + _ => Some(Sharpness::Inexact(0)), + }; + } + if left_stat.max_value.get_value()? > right_stat.min_value.get_value()? { + return match ( + left_stat.max_value.is_exact().unwrap(), + right_stat.min_value.is_exact().unwrap(), + ) { + (true, true) => Some(Sharpness::Exact(0)), + _ => Some(Sharpness::Inexact(0)), + }; } - let left_max_distinct = max_distinct_count(left_num_rows, left_stat.clone()); - let right_max_distinct = max_distinct_count(right_num_rows, right_stat.clone()); - let max_distinct = max(left_max_distinct, right_max_distinct); - if max_distinct > join_selectivity { + let left_max_distinct = + max_distinct_count(left_stats.num_rows.clone(), left_stat.clone())?; + let right_max_distinct = + max_distinct_count(right_stats.num_rows.clone(), right_stat.clone())?; + let max_distinct = left_max_distinct.max(&right_max_distinct); + if max_distinct.get_value().is_some() { // Seems like there are a few implementations of this algorithm that implement // exponential decay for the selectivity (like Hive's Optiq Optimizer). Needs // further exploration. @@ -853,9 +869,14 @@ fn estimate_inner_join_cardinality( // With the assumption that the smaller input's domain is generally represented in the bigger // input's domain, we can estimate the inner join's cardinality by taking the cartesian product // of the two inputs and normalizing it by the selectivity factor. + let left_num_rows = left_stats.num_rows.get_value()?; + let right_num_rows = right_stats.num_rows.get_value()?; match join_selectivity { - Some(selectivity) if selectivity > 0 => { - Some((left_num_rows * right_num_rows) / selectivity) + Sharpness::Exact(val) if val > 0 => { + Some(Sharpness::Exact((left_num_rows * right_num_rows) / val)) + } + Sharpness::Inexact(val) if val > 0 => { + Some(Sharpness::Inexact((left_num_rows * right_num_rows) / val)) } // Since we don't have any information about the selectivity (which is derived // from the number of distinct rows information) we can give up here for now. @@ -871,9 +892,18 @@ fn estimate_inner_join_cardinality( /// If distinct_count is available, uses it directly. If the column numeric, and /// has min/max values, then they might be used as a fallback option. Otherwise, /// returns None. -fn max_distinct_count(num_rows: usize, stats: ColumnStatistics) -> Option { - match (stats.distinct_count, stats.max_value, stats.min_value) { - (Some(_), _, _) => stats.distinct_count, +fn max_distinct_count( + num_rows: Sharpness, + stats: ColumnStatistics, +) -> Option> { + match ( + &stats.distinct_count, + stats.max_value.get_value(), + stats.min_value.get_value(), + ) { + (Sharpness::Exact(_), _, _) | (Sharpness::Inexact(_), _, _) => { + Some(stats.distinct_count) + } (_, Some(max), Some(min)) => { // Note that float support is intentionally omitted here, since the computation // of a range between two float values is not trivial and the result would be @@ -882,8 +912,18 @@ fn max_distinct_count(num_rows: usize, stats: ColumnStatistics) -> Option // The number can never be greater than the number of rows we have (minus // the nulls, since they don't count as distinct values). - let ceiling = num_rows - stats.null_count.unwrap_or(0); - Some(numeric_range.min(ceiling)) + let ceiling = + num_rows.get_value()? - stats.null_count.get_value().unwrap_or(0); + Some( + if num_rows.is_exact().unwrap() + && stats.max_value.is_exact().unwrap() + && stats.min_value.is_exact().unwrap() + { + Sharpness::Exact(numeric_range.min(ceiling)) + } else { + Sharpness::Inexact(numeric_range.min(ceiling)) + }, + ) } _ => None, } @@ -1547,11 +1587,22 @@ mod tests { column_stats: Vec, is_exact: bool, ) -> Statistics { - Statistics { - num_rows, - column_statistics: column_stats, - is_exact, - total_byte_size: None, + if is_exact { + Statistics { + num_rows: num_rows + .map(|size| Sharpness::Exact(size)) + .unwrap_or(Sharpness::Absent), + column_statistics: column_stats, + total_byte_size: Sharpness::Absent, + } + } else { + Statistics { + num_rows: num_rows + .map(|size| Sharpness::Inexact(size)) + .unwrap_or(Sharpness::Absent), + column_statistics: column_stats, + total_byte_size: Sharpness::Absent, + } } } @@ -1561,9 +1612,15 @@ mod tests { distinct_count: Option, ) -> ColumnStatistics { ColumnStatistics { - distinct_count, - min_value: min.map(|size| ScalarValue::Int64(Some(size))), - max_value: max.map(|size| ScalarValue::Int64(Some(size))), + distinct_count: distinct_count + .map(|count| Sharpness::Inexact(count)) + .unwrap_or(Sharpness::Absent), + min_value: min + .map(|size| Sharpness::Inexact(ScalarValue::Int64(Some(size)))) + .unwrap_or(Sharpness::Absent), + max_value: max + .map(|size| Sharpness::Inexact(ScalarValue::Int64(Some(size)))) + .unwrap_or(Sharpness::Absent), ..Default::default() } } @@ -1575,7 +1632,7 @@ mod tests { // over the expected output (since it depends on join type to join type). #[test] fn test_inner_join_cardinality_single_column() -> Result<()> { - let cases: Vec<(PartialStats, PartialStats, Option)> = vec![ + let cases: Vec<(PartialStats, PartialStats, Option>)> = vec![ // ----------------------------------------------------------------------------- // | left(rows, min, max, distinct), right(rows, min, max, distinct), expected | // ----------------------------------------------------------------------------- @@ -1587,55 +1644,55 @@ mod tests { ( (10, Some(1), Some(10), None), (10, Some(1), Some(10), None), - Some(10), + Some(Sharpness::Inexact(10)), ), // range(left) > range(right) ( (10, Some(6), Some(10), None), (10, Some(8), Some(10), None), - Some(20), + Some(Sharpness::Inexact(20)), ), // range(right) > range(left) ( (10, Some(8), Some(10), None), (10, Some(6), Some(10), None), - Some(20), + Some(Sharpness::Inexact(20)), ), // range(left) > len(left), range(right) > len(right) ( (10, Some(1), Some(15), None), (20, Some(1), Some(40), None), - Some(10), + Some(Sharpness::Inexact(10)), ), // When we have distinct count. ( (10, Some(1), Some(10), Some(10)), (10, Some(1), Some(10), Some(10)), - Some(10), + Some(Sharpness::Inexact(10)), ), // distinct(left) > distinct(right) ( (10, Some(1), Some(10), Some(5)), (10, Some(1), Some(10), Some(2)), - Some(20), + Some(Sharpness::Inexact(20)), ), // distinct(right) > distinct(left) ( (10, Some(1), Some(10), Some(2)), (10, Some(1), Some(10), Some(5)), - Some(20), + Some(Sharpness::Inexact(20)), ), // min(left) < 0 (range(left) > range(right)) ( (10, Some(-5), Some(5), None), (10, Some(1), Some(5), None), - Some(10), + Some(Sharpness::Inexact(10)), ), // min(right) < 0, max(right) < 0 (range(right) > range(left)) ( (10, Some(-25), Some(-20), None), (10, Some(-25), Some(-15), None), - Some(10), + Some(Sharpness::Inexact(10)), ), // range(left) < 0, range(right) >= 0 // (there isn't a case where both left and right ranges are negative @@ -1644,13 +1701,13 @@ mod tests { ( (10, Some(10), Some(0), None), (10, Some(0), Some(10), Some(5)), - Some(20), // It would have been ten if we have used abs(range(left)) + Some(Sharpness::Inexact(20)), // It would have been ten if we have used abs(range(left)) ), // range(left) = 1, range(right) = 1 ( (10, Some(1), Some(1), None), (10, Some(1), Some(1), None), - Some(100), + Some(Sharpness::Inexact(100)), ), // // Edge cases @@ -1714,11 +1771,16 @@ mod tests { assert_eq!( estimate_inner_join_cardinality( - left_num_rows, - right_num_rows, - left_col_stats.clone(), - right_col_stats.clone(), - false, + Statistics { + num_rows: Sharpness::Inexact(left_num_rows), + total_byte_size: Sharpness::Absent, + column_statistics: left_col_stats.clone(), + }, + Statistics { + num_rows: Sharpness::Inexact(right_num_rows), + total_byte_size: Sharpness::Absent, + column_statistics: right_col_stats.clone(), + }, ), expected_cardinality ); @@ -1734,7 +1796,9 @@ mod tests { ); assert_eq!( - partial_join_stats.clone().map(|s| s.num_rows), + partial_join_stats + .clone() + .map(|s| Sharpness::Inexact(s.num_rows)), expected_cardinality ); assert_eq!( @@ -1761,13 +1825,18 @@ mod tests { // count is 200, so we are going to pick it. assert_eq!( estimate_inner_join_cardinality( - 400, - 400, - left_col_stats, - right_col_stats, - false + Statistics { + num_rows: Sharpness::Inexact(400), + total_byte_size: Sharpness::Absent, + column_statistics: left_col_stats, + }, + Statistics { + num_rows: Sharpness::Inexact(400), + total_byte_size: Sharpness::Absent, + column_statistics: right_col_stats, + }, ), - Some((400 * 400) / 200) + Some(Sharpness::Inexact((400 * 400) / 200)) ); Ok(()) } @@ -1775,26 +1844,31 @@ mod tests { #[test] fn test_inner_join_cardinality_decimal_range() -> Result<()> { let left_col_stats = vec![ColumnStatistics { - distinct_count: None, - min_value: Some(ScalarValue::Decimal128(Some(32500), 14, 4)), - max_value: Some(ScalarValue::Decimal128(Some(35000), 14, 4)), + distinct_count: Sharpness::Absent, + min_value: Sharpness::Inexact(ScalarValue::Decimal128(Some(32500), 14, 4)), + max_value: Sharpness::Inexact(ScalarValue::Decimal128(Some(35000), 14, 4)), ..Default::default() }]; let right_col_stats = vec![ColumnStatistics { - distinct_count: None, - min_value: Some(ScalarValue::Decimal128(Some(33500), 14, 4)), - max_value: Some(ScalarValue::Decimal128(Some(34000), 14, 4)), + distinct_count: Sharpness::Absent, + min_value: Sharpness::Inexact(ScalarValue::Decimal128(Some(33500), 14, 4)), + max_value: Sharpness::Inexact(ScalarValue::Decimal128(Some(34000), 14, 4)), ..Default::default() }]; assert_eq!( estimate_inner_join_cardinality( - 100, - 100, - left_col_stats, - right_col_stats, - false + Statistics { + num_rows: Sharpness::Inexact(100), + total_byte_size: Sharpness::Absent, + column_statistics: left_col_stats, + }, + Statistics { + num_rows: Sharpness::Inexact(100), + total_byte_size: Sharpness::Absent, + column_statistics: right_col_stats, + }, ), None ); diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index aba848ddce23..a187718b1d24 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -33,6 +33,7 @@ use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics} use arrow::array::ArrayRef; use arrow::datatypes::SchemaRef; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; +use datafusion_common::stats::Sharpness; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::OrderingEquivalenceProperties; @@ -206,17 +207,28 @@ impl ExecutionPlan for GlobalLimitExec { }) .unwrap_or(usize::MAX); let col_stats = Statistics::unbounded_column_statistics(&self.schema()); + + let fetched_row_number_stats = Statistics { + num_rows: Sharpness::Exact(max_row_num), + column_statistics: col_stats.clone(), + total_byte_size: Sharpness::Absent, + }; + let stats = match input_stats { Statistics { - num_rows: Some(nr), .. + num_rows: Sharpness::Exact(nr), + .. + } + | Statistics { + num_rows: Sharpness::Inexact(nr), + .. } => { if nr <= skip { // if all input data will be skipped, return 0 Statistics { - num_rows: Some(0), - is_exact: input_stats.is_exact, + num_rows: Sharpness::Exact(0), column_statistics: col_stats, - total_byte_size: None, + total_byte_size: Sharpness::Absent, } } else if nr <= max_row_num { // if the input does not reach the "fetch" globally, return input stats @@ -224,21 +236,13 @@ impl ExecutionPlan for GlobalLimitExec { } else { // if the input is greater than the "fetch", the num_row will be the "fetch", // but we won't be able to predict the other statistics - Statistics { - num_rows: Some(max_row_num), - is_exact: input_stats.is_exact, - column_statistics: col_stats, - total_byte_size: None, - } + fetched_row_number_stats } } - _ => Statistics { + _ => { // the result output row number will always be no greater than the limit number - num_rows: Some(max_row_num), - is_exact: false, - column_statistics: col_stats, - total_byte_size: None, - }, + fetched_row_number_stats + } }; Ok(stats) } @@ -372,27 +376,44 @@ impl ExecutionPlan for LocalLimitExec { let stats = match input_stats { // if the input does not reach the limit globally, return input stats Statistics { - num_rows: Some(nr), .. + num_rows: Sharpness::Exact(nr), + .. + } + | Statistics { + num_rows: Sharpness::Inexact(nr), + .. } if nr <= self.fetch => input_stats, // if the input is greater than the limit, the num_row will be greater // than the limit because the partitions will be limited separatly // the statistic Statistics { - num_rows: Some(nr), .. + num_rows: Sharpness::Exact(nr), + .. } if nr > self.fetch => Statistics { - num_rows: Some(self.fetch), + num_rows: Sharpness::Exact(self.fetch), // this is not actually exact, but will be when GlobalLimit is applied // TODO stats: find a more explicit way to vehiculate this information - is_exact: input_stats.is_exact, column_statistics: col_stats, - total_byte_size: None, + total_byte_size: Sharpness::Absent, + }, + Statistics { + num_rows: Sharpness::Inexact(nr), + .. + } if nr > self.fetch => Statistics { + num_rows: Sharpness::Inexact(self.fetch), + // this is not actually exact, but will be when GlobalLimit is applied + // TODO stats: find a more explicit way to vehiculate this information + column_statistics: col_stats, + total_byte_size: Sharpness::Absent, }, _ => Statistics { // the result output row number will always be no greater than the limit number - num_rows: Some(self.fetch * self.output_partitioning().partition_count()), - is_exact: false, + num_rows: Sharpness::Inexact( + self.fetch * self.output_partitioning().partition_count(), + ), + column_statistics: col_stats, - total_byte_size: None, + total_byte_size: Sharpness::Absent, }, }; Ok(stats) @@ -737,10 +758,10 @@ mod tests { #[tokio::test] async fn test_row_number_statistics_for_global_limit() -> Result<()> { let row_count = row_number_statistics_for_global_limit(0, Some(10)).await?; - assert_eq!(row_count, Some(10)); + assert_eq!(row_count, Sharpness::Exact(10)); let row_count = row_number_statistics_for_global_limit(5, Some(10)).await?; - assert_eq!(row_count, Some(15)); + assert_eq!(row_count, Sharpness::Exact(15)); Ok(()) } @@ -748,7 +769,7 @@ mod tests { #[tokio::test] async fn test_row_number_statistics_for_local_limit() -> Result<()> { let row_count = row_number_statistics_for_local_limit(4, 10).await?; - assert_eq!(row_count, Some(10)); + assert_eq!(row_count, Sharpness::Exact(10)); Ok(()) } @@ -756,7 +777,7 @@ mod tests { async fn row_number_statistics_for_global_limit( skip: usize, fetch: Option, - ) -> Result> { + ) -> Result> { let num_partitions = 4; let csv = test::scan_partitioned(num_partitions); @@ -771,7 +792,7 @@ mod tests { async fn row_number_statistics_for_local_limit( num_partitions: usize, fetch: usize, - ) -> Result> { + ) -> Result> { let csv = test::scan_partitioned(num_partitions); assert_eq!(csv.output_partitioning().partition_count(), num_partitions); diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index f2d653dee226..f2f68262f4b1 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -36,6 +36,7 @@ use crate::{ use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; +use datafusion_common::stats::Sharpness; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; @@ -424,19 +425,17 @@ fn stats_projection( }) .try_fold(0usize, |init, v| v.map(|value| init + value)); - match (primitive_row_size, stats.num_rows) { - (Some(row_size), Some(row_count)) => { + match (primitive_row_size, stats.num_rows.get_value()) { + (Some(row_size), Some(_)) => { Statistics { - is_exact: stats.is_exact, - num_rows: stats.num_rows, + num_rows: stats.num_rows.clone(), column_statistics, // Use the row_size * row_count as the total byte size - total_byte_size: Some(row_size * row_count), + total_byte_size: Sharpness::Exact(row_size).multiply(&stats.num_rows), } } _ => { Statistics { - is_exact: stats.is_exact, num_rows: stats.num_rows, column_statistics, // TODO stats: knowing the type of the new columns we can guess the output size @@ -532,27 +531,30 @@ mod tests { fn get_stats() -> Statistics { Statistics { - is_exact: true, - num_rows: Some(5), - total_byte_size: Some(23), + num_rows: Sharpness::Exact(5), + total_byte_size: Sharpness::Exact(23), column_statistics: vec![ ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Sharpness::Exact(5), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), + null_count: Sharpness::Exact(0), }, ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: Some(3), + distinct_count: Sharpness::Exact(1), + max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Sharpness::Exact(3), }, ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Float32(Some(1.1))), - min_value: Some(ScalarValue::Float32(Some(0.1))), - null_count: None, + distinct_count: Sharpness::Absent, + max_value: Sharpness::Exact(ScalarValue::Float32(Some(1.1))), + min_value: Sharpness::Exact(ScalarValue::Float32(Some(0.1))), + null_count: Sharpness::Absent, }, ], } @@ -577,21 +579,24 @@ mod tests { let result = stats_projection(source, exprs.into_iter(), Arc::new(schema)); let expected = Statistics { - is_exact: true, - num_rows: Some(5), - total_byte_size: Some(23), + num_rows: Sharpness::Exact(5), + total_byte_size: Sharpness::Exact(23), column_statistics: vec![ ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: Some(3), + distinct_count: Sharpness::Exact(1), + max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Sharpness::Exact(3), }, ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Sharpness::Exact(5), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), + null_count: Sharpness::Exact(0), }, ], }; @@ -612,21 +617,20 @@ mod tests { let result = stats_projection(source, exprs.into_iter(), Arc::new(schema)); let expected = Statistics { - is_exact: true, - num_rows: Some(5), - total_byte_size: Some(60), + num_rows: Sharpness::Exact(5), + total_byte_size: Sharpness::Exact(60), column_statistics: vec![ ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Float32(Some(1.1))), - min_value: Some(ScalarValue::Float32(Some(0.1))), - null_count: None, + distinct_count: Sharpness::Absent, + max_value: Sharpness::Exact(ScalarValue::Float32(Some(1.1))), + min_value: Sharpness::Exact(ScalarValue::Float32(Some(0.1))), + null_count: Sharpness::Absent, }, ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Sharpness::Exact(5), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), + null_count: Sharpness::Exact(0), }, ], }; diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index c09c3a9fd12b..16853a7d9155 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -29,6 +29,7 @@ use arrow::{ datatypes::{Field, Schema, SchemaRef}, record_batch::RecordBatch, }; +use datafusion_common::stats::Sharpness; use datafusion_common::{exec_err, internal_err, DFSchemaRef, DataFusionError}; use futures::Stream; use itertools::Itertools; @@ -574,29 +575,17 @@ fn col_stats_union( mut left: ColumnStatistics, right: ColumnStatistics, ) -> ColumnStatistics { - left.distinct_count = None; - left.min_value = left - .min_value - .zip(right.min_value) - .map(|(a, b)| expressions::helpers::min(&a, &b)) - .and_then(Result::ok); - left.max_value = left - .max_value - .zip(right.max_value) - .map(|(a, b)| expressions::helpers::max(&a, &b)) - .and_then(Result::ok); - left.null_count = left.null_count.zip(right.null_count).map(|(a, b)| a + b); + left.distinct_count = Sharpness::Absent; + left.min_value = left.min_value.min(&right.min_value); + left.max_value = left.max_value.max(&right.max_value); + left.null_count = left.null_count.add(&right.null_count); left } fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { - left.is_exact = left.is_exact && right.is_exact; - left.num_rows = left.num_rows.zip(right.num_rows).map(|(a, b)| a + b); - left.total_byte_size = left - .total_byte_size - .zip(right.total_byte_size) - .map(|(a, b)| a + b); + left.num_rows = left.num_rows.add(&right.num_rows); + left.total_byte_size = left.total_byte_size.add(&right.total_byte_size); left.column_statistics = left .column_statistics .into_iter() @@ -637,80 +626,89 @@ mod tests { #[tokio::test] async fn test_stats_union() { let left = Statistics { - is_exact: true, - num_rows: Some(5), - total_byte_size: Some(23), + num_rows: Sharpness::Exact(5), + total_byte_size: Sharpness::Exact(23), column_statistics: vec![ ColumnStatistics { - distinct_count: Some(5), - max_value: Some(ScalarValue::Int64(Some(21))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(0), + distinct_count: Sharpness::Exact(5), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), + null_count: Sharpness::Exact(0), }, ColumnStatistics { - distinct_count: Some(1), - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: Some(3), + distinct_count: Sharpness::Exact(1), + max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Sharpness::Exact(3), }, ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Float32(Some(1.1))), - min_value: Some(ScalarValue::Float32(Some(0.1))), - null_count: None, + distinct_count: Sharpness::Absent, + max_value: Sharpness::Exact(ScalarValue::Float32(Some(1.1))), + min_value: Sharpness::Exact(ScalarValue::Float32(Some(0.1))), + null_count: Sharpness::Absent, }, ], }; let right = Statistics { - is_exact: true, - num_rows: Some(7), - total_byte_size: Some(29), + num_rows: Sharpness::Exact(7), + total_byte_size: Sharpness::Exact(29), column_statistics: vec![ ColumnStatistics { - distinct_count: Some(3), - max_value: Some(ScalarValue::Int64(Some(34))), - min_value: Some(ScalarValue::Int64(Some(1))), - null_count: Some(1), + distinct_count: Sharpness::Exact(3), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(34))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(1))), + null_count: Sharpness::Exact(1), }, ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Utf8(Some(String::from("c")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("b")))), - null_count: None, + distinct_count: Sharpness::Absent, + max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "c", + )))), + min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "b", + )))), + null_count: Sharpness::Absent, }, ColumnStatistics { - distinct_count: None, - max_value: None, - min_value: None, - null_count: None, + distinct_count: Sharpness::Absent, + max_value: Sharpness::Absent, + min_value: Sharpness::Absent, + null_count: Sharpness::Absent, }, ], }; let result = stats_union(left, right); let expected = Statistics { - is_exact: true, - num_rows: Some(12), - total_byte_size: Some(52), + num_rows: Sharpness::Exact(12), + total_byte_size: Sharpness::Exact(52), column_statistics: vec![ ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Int64(Some(34))), - min_value: Some(ScalarValue::Int64(Some(-4))), - null_count: Some(1), + distinct_count: Sharpness::Absent, + max_value: Sharpness::Exact(ScalarValue::Int64(Some(34))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), + null_count: Sharpness::Exact(1), }, ColumnStatistics { - distinct_count: None, - max_value: Some(ScalarValue::Utf8(Some(String::from("x")))), - min_value: Some(ScalarValue::Utf8(Some(String::from("a")))), - null_count: None, + distinct_count: Sharpness::Absent, + max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "x", + )))), + min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + "a", + )))), + null_count: Sharpness::Absent, }, ColumnStatistics { - distinct_count: None, - max_value: None, - min_value: None, - null_count: None, + distinct_count: Sharpness::Absent, + max_value: Sharpness::Absent, + min_value: Sharpness::Absent, + null_count: Sharpness::Absent, }, ], }; 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 f8793be46f91..9216101b7f3e 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -29,6 +29,7 @@ use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, WindowExpr, }; +use datafusion_common::stats::Sharpness; use datafusion_common::{exec_err, plan_err, Result}; use datafusion_execution::TaskContext; @@ -344,10 +345,9 @@ impl ExecutionPlan for BoundedWindowAggExec { )) } Ok(Statistics { - is_exact: input_stat.is_exact, num_rows: input_stat.num_rows, column_statistics, - total_byte_size: None, + total_byte_size: Sharpness::Absent, }) } } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index d49f23cdadc2..abce15b4fa43 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -36,6 +36,7 @@ use arrow::{ datatypes::{Schema, SchemaRef}, record_batch::RecordBatch, }; +use datafusion_common::stats::Sharpness; use datafusion_common::utils::{evaluate_partition_ranges, get_at_indices}; use datafusion_common::Result; use datafusion_common::{internal_err, plan_err, DataFusionError}; @@ -273,10 +274,9 @@ impl ExecutionPlan for WindowAggExec { )) } Ok(Statistics { - is_exact: input_stat.is_exact, num_rows: input_stat.num_rows, column_statistics, - total_byte_size: None, + total_byte_size: Sharpness::Absent, }) } } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index f2d65817b0c8..bcb01ac6a15c 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1526,18 +1526,43 @@ message PartitionStats { repeated ColumnStats column_stats = 4; } +message Sharpness{ + oneof sharpness_variant { + ExactInfo exact = 1; + InexactInfo inexact = 2; + AbsentInfo absent = 3; + } +} + +message ExactInfo { + oneof exact_variant{ + int64 value = 1; + ScalarValue scalar_value = 2; + } +} + +message InexactInfo { + oneof inexact_variant{ + int64 value = 1; + ScalarValue scalar_value = 2; + } +} + +message AbsentInfo { + // No fields needed for Absent +} + message Statistics { - int64 num_rows = 1; - int64 total_byte_size = 2; + Sharpness num_rows = 1; + Sharpness total_byte_size = 2; repeated ColumnStats column_stats = 3; - bool is_exact = 4; } message ColumnStats { - ScalarValue min_value = 1; - ScalarValue max_value = 2; - repeated uint32 null_count = 3; - repeated uint32 distinct_count = 4; + Sharpness min_value = 1; + Sharpness max_value = 2; + Sharpness null_count = 3; + Sharpness distinct_count = 4; } message NamedStructFieldExpr { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index ff9341ce0b92..671130d7dc1b 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -1,3 +1,74 @@ +impl serde::Serialize for AbsentInfo { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let len = 0; + let struct_ser = serializer.serialize_struct("datafusion.AbsentInfo", len)?; + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for AbsentInfo { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + } + 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, + { + Err(serde::de::Error::unknown_field(value, FIELDS)) + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = AbsentInfo; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.AbsentInfo") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + while map.next_key::()?.is_some() { + let _ = map.next_value::()?; + } + Ok(AbsentInfo { + }) + } + } + deserializer.deserialize_struct("datafusion.AbsentInfo", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for AggregateExecNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -3295,10 +3366,10 @@ impl serde::Serialize for ColumnStats { if self.max_value.is_some() { len += 1; } - if !self.null_count.is_empty() { + if self.null_count.is_some() { len += 1; } - if !self.distinct_count.is_empty() { + if self.distinct_count.is_some() { len += 1; } let mut struct_ser = serializer.serialize_struct("datafusion.ColumnStats", len)?; @@ -3308,11 +3379,11 @@ impl serde::Serialize for ColumnStats { if let Some(v) = self.max_value.as_ref() { struct_ser.serialize_field("maxValue", v)?; } - if !self.null_count.is_empty() { - struct_ser.serialize_field("nullCount", &self.null_count)?; + if let Some(v) = self.null_count.as_ref() { + struct_ser.serialize_field("nullCount", v)?; } - if !self.distinct_count.is_empty() { - struct_ser.serialize_field("distinctCount", &self.distinct_count)?; + if let Some(v) = self.distinct_count.as_ref() { + struct_ser.serialize_field("distinctCount", v)?; } struct_ser.end() } @@ -3406,27 +3477,21 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { if null_count__.is_some() { return Err(serde::de::Error::duplicate_field("nullCount")); } - null_count__ = - Some(map.next_value::>>()? - .into_iter().map(|x| x.0).collect()) - ; + null_count__ = map.next_value()?; } GeneratedField::DistinctCount => { if distinct_count__.is_some() { return Err(serde::de::Error::duplicate_field("distinctCount")); } - distinct_count__ = - Some(map.next_value::>>()? - .into_iter().map(|x| x.0).collect()) - ; + distinct_count__ = map.next_value()?; } } } Ok(ColumnStats { min_value: min_value__, max_value: max_value__, - null_count: null_count__.unwrap_or_default(), - distinct_count: distinct_count__.unwrap_or_default(), + null_count: null_count__, + distinct_count: distinct_count__, }) } } @@ -6177,6 +6242,115 @@ impl<'de> serde::Deserialize<'de> for EmptyRelationNode { deserializer.deserialize_struct("datafusion.EmptyRelationNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for ExactInfo { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.exact_variant.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.ExactInfo", len)?; + if let Some(v) = self.exact_variant.as_ref() { + match v { + exact_info::ExactVariant::Value(v) => { + struct_ser.serialize_field("value", ToString::to_string(&v).as_str())?; + } + exact_info::ExactVariant::ScalarValue(v) => { + struct_ser.serialize_field("scalarValue", v)?; + } + } + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for ExactInfo { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "value", + "scalar_value", + "scalarValue", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Value, + ScalarValue, + } + 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 { + "value" => Ok(GeneratedField::Value), + "scalarValue" | "scalar_value" => Ok(GeneratedField::ScalarValue), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = ExactInfo; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.ExactInfo") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut exact_variant__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Value => { + if exact_variant__.is_some() { + return Err(serde::de::Error::duplicate_field("value")); + } + exact_variant__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| exact_info::ExactVariant::Value(x.0)); + } + GeneratedField::ScalarValue => { + if exact_variant__.is_some() { + return Err(serde::de::Error::duplicate_field("scalarValue")); + } + exact_variant__ = map.next_value::<::std::option::Option<_>>()?.map(exact_info::ExactVariant::ScalarValue) +; + } + } + } + Ok(ExactInfo { + exact_variant: exact_variant__, + }) + } + } + deserializer.deserialize_struct("datafusion.ExactInfo", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for ExplainExecNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -8373,6 +8547,115 @@ impl<'de> serde::Deserialize<'de> for InListNode { deserializer.deserialize_struct("datafusion.InListNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for InexactInfo { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.inexact_variant.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.InexactInfo", len)?; + if let Some(v) = self.inexact_variant.as_ref() { + match v { + inexact_info::InexactVariant::Value(v) => { + struct_ser.serialize_field("value", ToString::to_string(&v).as_str())?; + } + inexact_info::InexactVariant::ScalarValue(v) => { + struct_ser.serialize_field("scalarValue", v)?; + } + } + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for InexactInfo { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "value", + "scalar_value", + "scalarValue", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Value, + ScalarValue, + } + 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 { + "value" => Ok(GeneratedField::Value), + "scalarValue" | "scalar_value" => Ok(GeneratedField::ScalarValue), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = InexactInfo; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.InexactInfo") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut inexact_variant__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Value => { + if inexact_variant__.is_some() { + return Err(serde::de::Error::duplicate_field("value")); + } + inexact_variant__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| inexact_info::InexactVariant::Value(x.0)); + } + GeneratedField::ScalarValue => { + if inexact_variant__.is_some() { + return Err(serde::de::Error::duplicate_field("scalarValue")); + } + inexact_variant__ = map.next_value::<::std::option::Option<_>>()?.map(inexact_info::InexactVariant::ScalarValue) +; + } + } + } + Ok(InexactInfo { + inexact_variant: inexact_variant__, + }) + } + } + deserializer.deserialize_struct("datafusion.InexactInfo", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for IntervalMonthDayNanoValue { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -21222,6 +21505,128 @@ impl<'de> serde::Deserialize<'de> for SelectionNode { deserializer.deserialize_struct("datafusion.SelectionNode", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for Sharpness { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.sharpness_variant.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.Sharpness", len)?; + if let Some(v) = self.sharpness_variant.as_ref() { + match v { + sharpness::SharpnessVariant::Exact(v) => { + struct_ser.serialize_field("exact", v)?; + } + sharpness::SharpnessVariant::Inexact(v) => { + struct_ser.serialize_field("inexact", v)?; + } + sharpness::SharpnessVariant::Absent(v) => { + struct_ser.serialize_field("absent", v)?; + } + } + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Sharpness { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "exact", + "inexact", + "absent", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Exact, + Inexact, + Absent, + } + 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 { + "exact" => Ok(GeneratedField::Exact), + "inexact" => Ok(GeneratedField::Inexact), + "absent" => Ok(GeneratedField::Absent), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Sharpness; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.Sharpness") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut sharpness_variant__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Exact => { + if sharpness_variant__.is_some() { + return Err(serde::de::Error::duplicate_field("exact")); + } + sharpness_variant__ = map.next_value::<::std::option::Option<_>>()?.map(sharpness::SharpnessVariant::Exact) +; + } + GeneratedField::Inexact => { + if sharpness_variant__.is_some() { + return Err(serde::de::Error::duplicate_field("inexact")); + } + sharpness_variant__ = map.next_value::<::std::option::Option<_>>()?.map(sharpness::SharpnessVariant::Inexact) +; + } + GeneratedField::Absent => { + if sharpness_variant__.is_some() { + return Err(serde::de::Error::duplicate_field("absent")); + } + sharpness_variant__ = map.next_value::<::std::option::Option<_>>()?.map(sharpness::SharpnessVariant::Absent) +; + } + } + } + Ok(Sharpness { + sharpness_variant: sharpness_variant__, + }) + } + } + deserializer.deserialize_struct("datafusion.Sharpness", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for SimilarToNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -21898,31 +22303,25 @@ impl serde::Serialize for Statistics { { use serde::ser::SerializeStruct; let mut len = 0; - if self.num_rows != 0 { + if self.num_rows.is_some() { len += 1; } - if self.total_byte_size != 0 { + if self.total_byte_size.is_some() { len += 1; } if !self.column_stats.is_empty() { len += 1; } - if self.is_exact { - len += 1; - } let mut struct_ser = serializer.serialize_struct("datafusion.Statistics", len)?; - if self.num_rows != 0 { - struct_ser.serialize_field("numRows", ToString::to_string(&self.num_rows).as_str())?; + if let Some(v) = self.num_rows.as_ref() { + struct_ser.serialize_field("numRows", v)?; } - if self.total_byte_size != 0 { - struct_ser.serialize_field("totalByteSize", ToString::to_string(&self.total_byte_size).as_str())?; + if let Some(v) = self.total_byte_size.as_ref() { + struct_ser.serialize_field("totalByteSize", v)?; } if !self.column_stats.is_empty() { struct_ser.serialize_field("columnStats", &self.column_stats)?; } - if self.is_exact { - struct_ser.serialize_field("isExact", &self.is_exact)?; - } struct_ser.end() } } @@ -21939,8 +22338,6 @@ impl<'de> serde::Deserialize<'de> for Statistics { "totalByteSize", "column_stats", "columnStats", - "is_exact", - "isExact", ]; #[allow(clippy::enum_variant_names)] @@ -21948,7 +22345,6 @@ impl<'de> serde::Deserialize<'de> for Statistics { NumRows, TotalByteSize, ColumnStats, - IsExact, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -21973,7 +22369,6 @@ impl<'de> serde::Deserialize<'de> for Statistics { "numRows" | "num_rows" => Ok(GeneratedField::NumRows), "totalByteSize" | "total_byte_size" => Ok(GeneratedField::TotalByteSize), "columnStats" | "column_stats" => Ok(GeneratedField::ColumnStats), - "isExact" | "is_exact" => Ok(GeneratedField::IsExact), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -21996,24 +22391,19 @@ impl<'de> serde::Deserialize<'de> for Statistics { let mut num_rows__ = None; let mut total_byte_size__ = None; let mut column_stats__ = None; - let mut is_exact__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::NumRows => { if num_rows__.is_some() { return Err(serde::de::Error::duplicate_field("numRows")); } - num_rows__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) - ; + num_rows__ = map.next_value()?; } GeneratedField::TotalByteSize => { if total_byte_size__.is_some() { return Err(serde::de::Error::duplicate_field("totalByteSize")); } - total_byte_size__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) - ; + total_byte_size__ = map.next_value()?; } GeneratedField::ColumnStats => { if column_stats__.is_some() { @@ -22021,19 +22411,12 @@ impl<'de> serde::Deserialize<'de> for Statistics { } column_stats__ = Some(map.next_value()?); } - GeneratedField::IsExact => { - if is_exact__.is_some() { - return Err(serde::de::Error::duplicate_field("isExact")); - } - is_exact__ = Some(map.next_value()?); - } } } Ok(Statistics { - num_rows: num_rows__.unwrap_or_default(), - total_byte_size: total_byte_size__.unwrap_or_default(), + num_rows: num_rows__, + total_byte_size: total_byte_size__, column_stats: column_stats__.unwrap_or_default(), - is_exact: is_exact__.unwrap_or_default(), }) } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 73105e4658e3..f3b0d6470863 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2173,27 +2173,82 @@ pub struct PartitionStats { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct Sharpness { + #[prost(oneof = "sharpness::SharpnessVariant", tags = "1, 2, 3")] + pub sharpness_variant: ::core::option::Option, +} +/// Nested message and enum types in `Sharpness`. +pub mod sharpness { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum SharpnessVariant { + #[prost(message, tag = "1")] + Exact(super::ExactInfo), + #[prost(message, tag = "2")] + Inexact(super::InexactInfo), + #[prost(message, tag = "3")] + Absent(super::AbsentInfo), + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ExactInfo { + #[prost(oneof = "exact_info::ExactVariant", tags = "1, 2")] + pub exact_variant: ::core::option::Option, +} +/// Nested message and enum types in `ExactInfo`. +pub mod exact_info { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum ExactVariant { + #[prost(int64, tag = "1")] + Value(i64), + #[prost(message, tag = "2")] + ScalarValue(super::ScalarValue), + } +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct InexactInfo { + #[prost(oneof = "inexact_info::InexactVariant", tags = "1, 2")] + pub inexact_variant: ::core::option::Option, +} +/// Nested message and enum types in `InexactInfo`. +pub mod inexact_info { + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum InexactVariant { + #[prost(int64, tag = "1")] + Value(i64), + #[prost(message, tag = "2")] + ScalarValue(super::ScalarValue), + } +} +/// No fields needed for Absent +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct AbsentInfo {} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct Statistics { - #[prost(int64, tag = "1")] - pub num_rows: i64, - #[prost(int64, tag = "2")] - pub total_byte_size: i64, + #[prost(message, optional, tag = "1")] + pub num_rows: ::core::option::Option, + #[prost(message, optional, tag = "2")] + pub total_byte_size: ::core::option::Option, #[prost(message, repeated, tag = "3")] pub column_stats: ::prost::alloc::vec::Vec, - #[prost(bool, tag = "4")] - pub is_exact: bool, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ColumnStats { #[prost(message, optional, tag = "1")] - pub min_value: ::core::option::Option, + pub min_value: ::core::option::Option, #[prost(message, optional, tag = "2")] - pub max_value: ::core::option::Option, - #[prost(uint32, repeated, tag = "3")] - pub null_count: ::prost::alloc::vec::Vec, - #[prost(uint32, repeated, tag = "4")] - pub distinct_count: ::prost::alloc::vec::Vec, + pub max_value: ::core::option::Option, + #[prost(message, optional, tag = "3")] + pub null_count: ::core::option::Option, + #[prost(message, optional, tag = "4")] + pub distinct_count: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] From b56a81edd3c4955e52438eee51680c78a74173f0 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 28 Sep 2023 11:38:12 +0300 Subject: [PATCH 17/53] error in agg optimization --- datafusion/common/src/stats.rs | 41 ++ .../core/src/datasource/file_format/csv.rs | 5 +- .../core/src/datasource/file_format/json.rs | 5 +- .../core/src/datasource/listing/table.rs | 29 +- .../physical_plan/file_scan_config.rs | 2 +- datafusion/core/src/datasource/statistics.rs | 135 ++++-- .../aggregate_statistics.rs | 17 +- .../enforce_distribution.rs | 9 +- .../src/physical_optimizer/join_selection.rs | 6 +- datafusion/core/tests/custom_sources.rs | 12 +- .../tests/custom_sources_cases/statistics.rs | 34 +- .../core/tests/parquet/file_statistics.rs | 22 +- datafusion/core/tests/path_partition.rs | 5 +- datafusion/physical-plan/src/common.rs | 2 +- datafusion/proto/proto/datafusion.proto | 27 +- datafusion/proto/src/generated/pbjson.rs | 436 +++++------------- datafusion/proto/src/generated/prost.rs | 86 ++-- .../proto/src/physical_plan/from_proto.rs | 118 ++++- datafusion/proto/src/physical_plan/mod.rs | 6 +- .../proto/src/physical_plan/to_proto.rs | 85 +++- 20 files changed, 535 insertions(+), 547 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index ba7c89c5c60d..658b043bcbae 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -208,6 +208,47 @@ impl Statistics { && col_stat.null_count.is_exact().unwrap_or(false) }) } + + pub fn make_inexact(self) -> Self { + Statistics { + num_rows: if let Sharpness::Exact(val) = &self.num_rows { + Sharpness::Inexact(val.clone()) + } else { + self.num_rows + }, + total_byte_size: if let Sharpness::Exact(val) = &self.total_byte_size { + Sharpness::Inexact(val.clone()) + } else { + self.total_byte_size + }, + column_statistics: self + .column_statistics + .iter() + .map(|cs| ColumnStatistics { + null_count: if let Sharpness::Exact(val) = &cs.null_count { + Sharpness::Inexact(val.clone()) + } else { + cs.null_count.clone() + }, + max_value: if let Sharpness::Exact(val) = &cs.max_value { + Sharpness::Inexact(val.clone()) + } else { + cs.max_value.clone() + }, + min_value: if let Sharpness::Exact(val) = &cs.min_value { + Sharpness::Inexact(val.clone()) + } else { + cs.min_value.clone() + }, + distinct_count: if let Sharpness::Exact(val) = &cs.distinct_count { + Sharpness::Inexact(val.clone()) + } else { + cs.distinct_count.clone() + }, + }) + .collect::>(), + } + } } impl Display for Statistics { diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index d184042bd657..05e33f5c1b96 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -610,6 +610,7 @@ mod tests { use chrono::DateTime; use datafusion_common::cast::as_string_array; use datafusion_common::internal_err; + use datafusion_common::stats::Sharpness; use datafusion_common::FileType; use datafusion_common::GetExt; use datafusion_expr::{col, lit}; @@ -642,8 +643,8 @@ mod tests { assert_eq!(tt_batches, 50 /* 100/2 */); // test metadata - assert_eq!(exec.statistics()?.num_rows, None); - assert_eq!(exec.statistics()?.total_byte_size, None); + assert_eq!(exec.statistics()?.num_rows, Sharpness::Absent); + assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Absent); Ok(()) } diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 7fc4d18a6b3c..c61a6d624bd4 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -378,6 +378,7 @@ impl DataSink for JsonSink { mod tests { use super::super::test_util::scan_format; use datafusion_common::cast::as_int64_array; + use datafusion_common::stats::Sharpness; use futures::StreamExt; use object_store::local::LocalFileSystem; @@ -408,8 +409,8 @@ mod tests { assert_eq!(tt_batches, 6 /* 12/2 */); // test metadata - assert_eq!(exec.statistics()?.num_rows, None); - assert_eq!(exec.statistics()?.total_byte_size, None); + assert_eq!(exec.statistics()?.num_rows, Sharpness::Absent); + assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Absent); Ok(()) } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 2bb011f11cd9..b90617969b46 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -944,20 +944,22 @@ impl ListingTable { ) })) .await?; - let file_list = stream::iter(file_list).flatten(); - // collect the statistics if required by the config let files = file_list .map(|part_file| async { let part_file = part_file?; - let statistics_result = if self.options.collect_stat { + let mut statistics_result = + Statistics::new_with_unbounded_columns(&self.file_schema); + if self.options.collect_stat { let statistics_cache = self.collected_statistics.clone(); match statistics_cache.get_with_extra( &part_file.object_meta.location, &part_file.object_meta, ) { - Some(statistics) => statistics.as_ref().clone(), + Some(statistics) => { + statistics_result = statistics.as_ref().clone() + } None => { let statistics = self .options @@ -974,12 +976,10 @@ impl ListingTable { statistics.clone().into(), &part_file.object_meta, ); - statistics + statistics_result = statistics; } } - } else { - Statistics::new_with_unbounded_columns(&self.file_schema) - }; + } Ok((part_file, statistics_result)) as Result<(PartitionedFile, Statistics)> }) @@ -1015,6 +1015,7 @@ mod tests { use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; use datafusion_common::assert_contains; + use datafusion_common::stats::Sharpness; use datafusion_common::GetExt; use datafusion_expr::LogicalPlanBuilder; use rstest::*; @@ -1065,8 +1066,8 @@ mod tests { assert_eq!(exec.output_partitioning().partition_count(), 1); // test metadata - assert_eq!(exec.statistics()?.num_rows, Some(8)); - assert_eq!(exec.statistics()?.total_byte_size, Some(671)); + assert_eq!(exec.statistics()?.num_rows, Sharpness::Exact(8)); + assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Exact(671)); Ok(()) } @@ -1088,8 +1089,8 @@ mod tests { let table = ListingTable::try_new(config)?; let exec = table.scan(&state, None, &[], None).await?; - assert_eq!(exec.statistics()?.num_rows, Some(8)); - assert_eq!(exec.statistics()?.total_byte_size, Some(671)); + assert_eq!(exec.statistics()?.num_rows, Sharpness::Exact(8)); + assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Exact(671)); Ok(()) } @@ -1112,8 +1113,8 @@ mod tests { let table = ListingTable::try_new(config)?; let exec = table.scan(&state, None, &[], None).await?; - assert_eq!(exec.statistics()?.num_rows, None); - assert_eq!(exec.statistics()?.total_byte_size, None); + assert_eq!(exec.statistics()?.num_rows, Sharpness::Absent); + assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Absent); Ok(()) } 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 acf9f557e166..c7112d3d207f 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -144,7 +144,7 @@ impl FileScanConfig { } let table_stats = Statistics { - num_rows: self.statistics.num_rows, + num_rows: self.statistics.num_rows.clone(), // TODO correct byte size? total_byte_size: Sharpness::Absent, column_statistics: table_cols_stats, diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 895cf587c5d5..5ac2d7e6697c 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -39,8 +39,6 @@ pub async fn get_statistics_with_limit( let mut has_statistics = false; let (mut max_values, mut min_values) = create_max_min_accs(&file_schema); - let mut is_exact = true; - // The number of rows and the total byte size can be calculated as long as // at least one file has them. If none of the files provide them, then they // will be omitted from the statistics. The missing values will be counted @@ -53,78 +51,119 @@ pub async fn get_statistics_with_limit( while let Some(res) = all_files.next().await { let (file, file_stats) = res?; result_files.push(file); - num_rows = if num_rows.get_value().is_some() { - num_rows.add(&file_stats.num_rows) + num_rows = if let Some(exactness) = num_rows.is_exact() { + if exactness { + if file_stats.num_rows == Sharpness::Absent { + Sharpness::Exact(0 as usize) + .add(&Sharpness::Exact(num_rows.get_value().unwrap())) + } else { + file_stats + .num_rows + .add(&Sharpness::Exact(num_rows.get_value().unwrap())) + } + } else { + if file_stats.num_rows == Sharpness::Absent { + Sharpness::Exact(0 as usize) + .add(&Sharpness::Inexact(num_rows.get_value().unwrap())) + } else { + file_stats + .num_rows + .add(&Sharpness::Inexact(num_rows.get_value().unwrap())) + } + } } else { file_stats.num_rows }; - total_byte_size = if total_byte_size.get_value().is_some() { - total_byte_size.add(&file_stats.total_byte_size) + total_byte_size = if let Some(exactness) = total_byte_size.is_exact() { + if exactness { + if file_stats.total_byte_size == Sharpness::Absent { + Sharpness::Exact(0 as usize) + .add(&Sharpness::Exact(total_byte_size.get_value().unwrap())) + } else { + file_stats + .total_byte_size + .add(&Sharpness::Exact(total_byte_size.get_value().unwrap())) + } + } else { + if file_stats.total_byte_size == Sharpness::Absent { + Sharpness::Exact(0 as usize) + .add(&Sharpness::Inexact(total_byte_size.get_value().unwrap())) + } else { + file_stats + .total_byte_size + .add(&Sharpness::Inexact(total_byte_size.get_value().unwrap())) + } + } } else { file_stats.total_byte_size }; - - has_statistics = true; - for (i, cs) in file_stats.column_statistics.iter().enumerate() { - null_counts[i] = null_counts[i].add(&cs.null_count); - - if let Some(max_value) = &mut max_values[i] { - if let Some(file_max) = cs.max_value.get_value() { - match max_value.update_batch(&[file_max.to_array()]) { - Ok(_) => {} - Err(_) => { - max_values[i] = None; + if !file_stats.column_statistics.is_empty() { + has_statistics = true; + for (i, cs) in file_stats.column_statistics.iter().enumerate() { + null_counts[i] = if cs.null_count == Sharpness::Absent { + null_counts[i].clone() + } else { + null_counts[i].add(&cs.null_count) + }; + + if let Some(max_value) = &mut max_values[i] { + if let Some(file_max) = cs.max_value.get_value() { + match max_value.update_batch(&[file_max.to_array()]) { + Ok(_) => {} + Err(_) => { + max_values[i] = None; + } } + } else { + max_values[i] = None; } - } else { - max_values[i] = None; } - } - if let Some(min_value) = &mut min_values[i] { - if let Some(file_min) = cs.min_value.get_value() { - match min_value.update_batch(&[file_min.to_array()]) { - Ok(_) => {} - Err(_) => { - min_values[i] = None; + if let Some(min_value) = &mut min_values[i] { + if let Some(file_min) = cs.min_value.get_value() { + match min_value.update_batch(&[file_min.to_array()]) { + Ok(_) => {} + Err(_) => { + min_values[i] = None; + } } + } else { + min_values[i] = None; } - } else { - min_values[i] = None; } } } - // If the number of rows exceeds the limit, we can stop processing // files. This only applies when we know the number of rows. It also // currently ignores tables that have no statistics regarding the // number of rows. - match (num_rows.get_value(), limit) { - (Some(nr), Some(limit)) => { - if nr > limit { - break; - } - } - _ => break, + if num_rows.get_value().unwrap_or(usize::MIN) > limit.unwrap_or(usize::MAX) { + break; } } - // if we still have files in the stream, it means that the limit kicked - // in and that the statistic could have been different if we processed - // the files in a different order. - if all_files.next().await.is_some() { - is_exact = false; - } let column_stats = if has_statistics { get_col_stats(&file_schema, null_counts, &mut max_values, &mut min_values) } else { - Statistics::unbounded_column_statistics(&file_schema) + Statistics::new_with_unbounded_columns(&file_schema).column_statistics }; - let statistics = Statistics { - num_rows, - total_byte_size, - column_statistics: column_stats, + let statistics = if all_files.next().await.is_some() { + // if we still have files in the stream, it means that the limit kicked + // in and that the statistic could have been different if we processed + // the files in a different order. + Statistics { + num_rows, + total_byte_size, + column_statistics: column_stats, + } + .make_inexact() + } else { + Statistics { + num_rows, + total_byte_size, + column_statistics: column_stats, + } }; Ok((result_files, statistics)) @@ -163,7 +202,7 @@ pub(crate) fn get_col_stats( None => None, }; ColumnStatistics { - null_count: null_counts[i], + null_count: null_counts[i].clone(), max_value: max_value .map(|val| Sharpness::Exact(val)) .unwrap_or(Sharpness::Absent), diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 8e02e863fbcd..b95d294b33e5 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -114,7 +114,7 @@ impl PhysicalOptimizerRule for AggregateStatistics { /// the `ExecutionPlan.children()` method that returns an owned reference. fn take_optimizable(node: &dyn ExecutionPlan) -> Result>> { if let Some(final_agg_exec) = node.as_any().downcast_ref::() { - if final_agg_exec.mode() == &AggregateMode::Final + if !final_agg_exec.mode().is_first_stage() && final_agg_exec.group_expr().is_empty() { let mut child = Arc::clone(final_agg_exec.input()); @@ -122,11 +122,12 @@ fn take_optimizable(node: &dyn ExecutionPlan) -> Result() { - if partial_agg_exec.mode() == &AggregateMode::Partial + if partial_agg_exec.mode().is_first_stage() && partial_agg_exec.group_expr().is_empty() && partial_agg_exec.filter_expr().iter().all(|e| e.is_none()) { let stats = partial_agg_exec.input().statistics()?; + println!("{:?}", stats); if stats.all_exact() { return Ok(Some(child)); } @@ -262,6 +263,7 @@ mod tests { use datafusion_common::cast::as_int64_array; use datafusion_physical_expr::expressions::cast; use datafusion_physical_expr::PhysicalExpr; + use datafusion_physical_plan::displayable; use crate::error::Result; use crate::logical_expr::Operator; @@ -302,10 +304,17 @@ mod tests { ) -> Result<()> { let session_ctx = SessionContext::new(); let state = session_ctx.state(); - let plan = Arc::new(plan) as _; + let plan: Arc = Arc::new(plan); + fn print_plan(plan: &Arc) -> Result<()> { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); + Ok(()) + } + print_plan(&plan); let optimized = AggregateStatistics::new() .optimize(Arc::clone(&plan), state.config_options())?; - + print_plan(&optimized); // A ProjectionExec is a sign that the count optimization was applied assert!(optimized.as_any().is::()); diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 53e4747026f4..195b708ca68e 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1254,9 +1254,12 @@ fn ensure_distribution( // Don't need to apply when the returned row count is not greater than 1: let stats = dist_context.plan.statistics()?; let mut repartition_beneficial_stat = true; - if stats.is_exact { - repartition_beneficial_stat = - stats.num_rows.map(|num_rows| num_rows > 1).unwrap_or(true); + if stats.all_exact() { + repartition_beneficial_stat = stats + .num_rows + .get_value() + .map(|num_rows| num_rows > 1) + .unwrap_or(true); } // Remove unnecessary repartition from the physical plan if any diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 07ed3e2b2dd0..3ef49cd92da7 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -925,9 +925,9 @@ mod tests_statistical { " HashJoinExec: mode=CollectLeft, join_type=Right, on=[(small_col@1, medium_col@0)]", " ProjectionExec: expr=[big_col@1 as big_col, small_col@0 as small_col]", " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(small_col@0, big_col@0)]", - " StatisticsExec: col_count=1, row_count=Some(1000)", - " StatisticsExec: col_count=1, row_count=Some(100000)", - " StatisticsExec: col_count=1, row_count=Some(10000)", + " StatisticsExec: col_count=1, row_count=Inexact Info:(1000)", + " StatisticsExec: col_count=1, row_count=Inexact Info:(100000)", + " StatisticsExec: col_count=1, row_count=Inexact Info:(0)", "", ]; assert_optimized!(expected, join); diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index d816fba067c9..a124b5d7807f 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -38,6 +38,7 @@ use datafusion::{error::Result, physical_plan::DisplayFormatType}; use datafusion_common::cast::as_primitive_array; use datafusion_common::project_schema; +use datafusion_common::stats::Sharpness; use futures::stream::Stream; use std::any::Any; use std::pin::Pin; @@ -156,20 +157,19 @@ impl ExecutionPlan for CustomExecutionPlan { fn statistics(&self) -> Result { let batch = TEST_CUSTOM_RECORD_BATCH!().unwrap(); Ok(Statistics { - is_exact: true, - num_rows: Some(batch.num_rows()), - total_byte_size: None, + num_rows: Sharpness::Exact(batch.num_rows()), + total_byte_size: Sharpness::Absent, column_statistics: self .projection .clone() .unwrap_or_else(|| (0..batch.columns().len()).collect()) .iter() .map(|i| ColumnStatistics { - null_count: Some(batch.column(*i).null_count()), - min_value: Some(ScalarValue::Int32(aggregate::min( + null_count: Sharpness::Exact(batch.column(*i).null_count()), + min_value: Sharpness::Exact(ScalarValue::Int32(aggregate::min( as_primitive_array::(batch.column(*i)).unwrap(), ))), - max_value: Some(ScalarValue::Int32(aggregate::max( + max_value: Sharpness::Exact(ScalarValue::Int32(aggregate::max( as_primitive_array::(batch.column(*i)).unwrap(), ))), ..Default::default() diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index 540d16ddf361..d4bba7ddc57b 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -34,7 +34,7 @@ use datafusion::{ use async_trait::async_trait; use datafusion::execution::context::{SessionState, TaskContext}; -use datafusion_common::project_schema; +use datafusion_common::{project_schema, stats::Sharpness}; /// This is a testing structure for statistics /// It will act both as a table provider and execution plan @@ -97,11 +97,10 @@ impl TableProvider for StatisticsValidation { .collect(); Ok(Arc::new(Self::new( Statistics { - is_exact: current_stat.is_exact, num_rows: current_stat.num_rows, column_statistics: proj_col_stats, // TODO stats: knowing the type of the new columns we can guess the output size - total_byte_size: None, + total_byte_size: Sharpness::Absent, }, projected_schema, ))) @@ -179,21 +178,20 @@ fn init_ctx(stats: Statistics, schema: Schema) -> Result { fn fully_defined() -> (Statistics, Schema) { ( Statistics { - num_rows: Some(13), - is_exact: true, - total_byte_size: None, // ignore byte size for now + num_rows: Sharpness::Exact(13), + total_byte_size: Sharpness::Absent, // ignore byte size for now column_statistics: vec![ ColumnStatistics { - distinct_count: Some(2), - max_value: Some(ScalarValue::Int32(Some(1023))), - min_value: Some(ScalarValue::Int32(Some(-24))), - null_count: Some(0), + distinct_count: Sharpness::Exact(2), + max_value: Sharpness::Exact(ScalarValue::Int32(Some(1023))), + min_value: Sharpness::Exact(ScalarValue::Int32(Some(-24))), + null_count: Sharpness::Exact(0), }, ColumnStatistics { - distinct_count: Some(13), - max_value: Some(ScalarValue::Int64(Some(5486))), - min_value: Some(ScalarValue::Int64(Some(-6783))), - null_count: Some(5), + distinct_count: Sharpness::Exact(13), + max_value: Sharpness::Exact(ScalarValue::Int64(Some(5486))), + min_value: Sharpness::Exact(ScalarValue::Int64(Some(-6783))), + null_count: Sharpness::Exact(5), }, ], }, @@ -231,8 +229,7 @@ async fn sql_filter() -> Result<()> { let physical_plan = df.create_physical_plan().await.unwrap(); let stats = physical_plan.statistics()?; - assert!(!stats.is_exact); - assert_eq!(stats.num_rows, Some(1)); + assert_eq!(stats.num_rows, Sharpness::Exact(1)); Ok(()) } @@ -249,10 +246,9 @@ async fn sql_limit() -> Result<()> { let col_stats = Statistics::unbounded_column_statistics(&schema); assert_eq!( Statistics { - num_rows: Some(5), - is_exact: true, + num_rows: Sharpness::Exact(5), column_statistics: col_stats, - total_byte_size: None + total_byte_size: Sharpness::Absent }, physical_plan.statistics()? ); diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index 7f01fcafb21b..1eab51d112c0 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -22,6 +22,7 @@ use datafusion::datasource::listing::{ use datafusion::datasource::TableProvider; use datafusion::execution::context::SessionState; use datafusion::prelude::SessionContext; +use datafusion_common::stats::Sharpness; use datafusion_execution::cache::cache_manager::CacheManagerConfig; use datafusion_execution::cache::cache_unit; use datafusion_execution::cache::cache_unit::DefaultFileStatisticsCache; @@ -49,8 +50,11 @@ async fn load_table_stats_with_session_level_cache() { assert_eq!(get_cache_size(&state1), 0); let exec1 = table1.scan(&state1, None, &[], None).await.unwrap(); - assert_eq!(exec1.statistics().unwrap().num_rows, Some(8)); - assert_eq!(exec1.statistics().unwrap().total_byte_size, Some(671)); + assert_eq!(exec1.statistics().unwrap().num_rows, Sharpness::Exact(8)); + assert_eq!( + exec1.statistics().unwrap().total_byte_size, + Sharpness::Exact(671) + ); assert_eq!(get_cache_size(&state1), 1); //Session 2 first time list files @@ -65,16 +69,22 @@ async fn load_table_stats_with_session_level_cache() { 0 ); let exec2 = table2.scan(&state2, None, &[], None).await.unwrap(); - assert_eq!(exec2.statistics().unwrap().num_rows, Some(8)); - assert_eq!(exec2.statistics().unwrap().total_byte_size, Some(671)); + assert_eq!(exec2.statistics().unwrap().num_rows, Sharpness::Exact(8)); + assert_eq!( + exec2.statistics().unwrap().total_byte_size, + Sharpness::Exact(671) + ); assert_eq!(get_cache_size(&state2), 1); //Session 1 second time list files //check session 1 cache result not show in session 2 assert_eq!(get_cache_size(&state1), 1); let exec3 = table1.scan(&state1, None, &[], None).await.unwrap(); - assert_eq!(exec3.statistics().unwrap().num_rows, Some(8)); - assert_eq!(exec3.statistics().unwrap().total_byte_size, Some(671)); + assert_eq!(exec3.statistics().unwrap().num_rows, Sharpness::Exact(8)); + assert_eq!( + exec3.statistics().unwrap().total_byte_size, + Sharpness::Exact(671) + ); // List same file no increase assert_eq!(get_cache_size(&state1), 1); } diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index 3dd035222949..34efc59216a7 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -18,6 +18,7 @@ //! Test queries on partitioned datasets use arrow::datatypes::DataType; +use datafusion_common::stats::Sharpness; use std::collections::BTreeSet; use std::fs::File; use std::io::{Read, Seek, SeekFrom}; @@ -464,7 +465,7 @@ async fn parquet_statistics() -> Result<()> { let stat_cols = physical_plan.statistics()?.column_statistics; assert_eq!(stat_cols.len(), 4); // stats for the first col are read from the parquet file - assert_eq!(stat_cols[0].null_count, Some(3)); + assert_eq!(stat_cols[0].null_count, Sharpness::Exact(3)); // TODO assert partition column (1,2,3) stats once implemented (#1186) assert_eq!( stat_cols[1], @@ -488,7 +489,7 @@ async fn parquet_statistics() -> Result<()> { let stat_cols = physical_plan.statistics()?.column_statistics; assert_eq!(stat_cols.len(), 2); // stats for the first col are read from the parquet file - assert_eq!(stat_cols[0].null_count, Some(1)); + assert_eq!(stat_cols[0].null_count, Sharpness::Exact(1)); // TODO assert partition column stats once implemented (#1186) assert_eq!( stat_cols[1], diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index cef96242aca4..b13ce7e2937a 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -726,7 +726,7 @@ mod tests { }; // Prevent test flakiness due to undefined / changing implementation details - expected.total_byte_size = actual.total_byte_size; + expected.total_byte_size = actual.total_byte_size.clone(); assert_eq!(actual, expected); Ok(()) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index bcb01ac6a15c..7bcacc4722ff 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1527,29 +1527,14 @@ message PartitionStats { } message Sharpness{ - oneof sharpness_variant { - ExactInfo exact = 1; - InexactInfo inexact = 2; - AbsentInfo absent = 3; - } -} - -message ExactInfo { - oneof exact_variant{ - int64 value = 1; - ScalarValue scalar_value = 2; - } -} - -message InexactInfo { - oneof inexact_variant{ - int64 value = 1; - ScalarValue scalar_value = 2; - } + SharpnessInfo sharpness_info = 1; + ScalarValue val = 2; } -message AbsentInfo { - // No fields needed for Absent +enum SharpnessInfo { + EXACT = 0; + INEXACT = 1; + ABSENT = 2; } message Statistics { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 671130d7dc1b..9ae328457b30 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -1,74 +1,3 @@ -impl serde::Serialize for AbsentInfo { - #[allow(deprecated)] - fn serialize(&self, serializer: S) -> std::result::Result - where - S: serde::Serializer, - { - use serde::ser::SerializeStruct; - let len = 0; - let struct_ser = serializer.serialize_struct("datafusion.AbsentInfo", len)?; - struct_ser.end() - } -} -impl<'de> serde::Deserialize<'de> for AbsentInfo { - #[allow(deprecated)] - fn deserialize(deserializer: D) -> std::result::Result - where - D: serde::Deserializer<'de>, - { - const FIELDS: &[&str] = &[ - ]; - - #[allow(clippy::enum_variant_names)] - enum GeneratedField { - } - 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, - { - Err(serde::de::Error::unknown_field(value, FIELDS)) - } - } - deserializer.deserialize_identifier(GeneratedVisitor) - } - } - struct GeneratedVisitor; - impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = AbsentInfo; - - fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct datafusion.AbsentInfo") - } - - fn visit_map(self, mut map: V) -> std::result::Result - where - V: serde::de::MapAccess<'de>, - { - while map.next_key::()?.is_some() { - let _ = map.next_value::()?; - } - Ok(AbsentInfo { - }) - } - } - deserializer.deserialize_struct("datafusion.AbsentInfo", FIELDS, GeneratedVisitor) - } -} impl serde::Serialize for AggregateExecNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -6242,115 +6171,6 @@ impl<'de> serde::Deserialize<'de> for EmptyRelationNode { deserializer.deserialize_struct("datafusion.EmptyRelationNode", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for ExactInfo { - #[allow(deprecated)] - fn serialize(&self, serializer: S) -> std::result::Result - where - S: serde::Serializer, - { - use serde::ser::SerializeStruct; - let mut len = 0; - if self.exact_variant.is_some() { - len += 1; - } - let mut struct_ser = serializer.serialize_struct("datafusion.ExactInfo", len)?; - if let Some(v) = self.exact_variant.as_ref() { - match v { - exact_info::ExactVariant::Value(v) => { - struct_ser.serialize_field("value", ToString::to_string(&v).as_str())?; - } - exact_info::ExactVariant::ScalarValue(v) => { - struct_ser.serialize_field("scalarValue", v)?; - } - } - } - struct_ser.end() - } -} -impl<'de> serde::Deserialize<'de> for ExactInfo { - #[allow(deprecated)] - fn deserialize(deserializer: D) -> std::result::Result - where - D: serde::Deserializer<'de>, - { - const FIELDS: &[&str] = &[ - "value", - "scalar_value", - "scalarValue", - ]; - - #[allow(clippy::enum_variant_names)] - enum GeneratedField { - Value, - ScalarValue, - } - 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 { - "value" => Ok(GeneratedField::Value), - "scalarValue" | "scalar_value" => Ok(GeneratedField::ScalarValue), - _ => Err(serde::de::Error::unknown_field(value, FIELDS)), - } - } - } - deserializer.deserialize_identifier(GeneratedVisitor) - } - } - struct GeneratedVisitor; - impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = ExactInfo; - - fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct datafusion.ExactInfo") - } - - fn visit_map(self, mut map: V) -> std::result::Result - where - V: serde::de::MapAccess<'de>, - { - let mut exact_variant__ = None; - while let Some(k) = map.next_key()? { - match k { - GeneratedField::Value => { - if exact_variant__.is_some() { - return Err(serde::de::Error::duplicate_field("value")); - } - exact_variant__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| exact_info::ExactVariant::Value(x.0)); - } - GeneratedField::ScalarValue => { - if exact_variant__.is_some() { - return Err(serde::de::Error::duplicate_field("scalarValue")); - } - exact_variant__ = map.next_value::<::std::option::Option<_>>()?.map(exact_info::ExactVariant::ScalarValue) -; - } - } - } - Ok(ExactInfo { - exact_variant: exact_variant__, - }) - } - } - deserializer.deserialize_struct("datafusion.ExactInfo", FIELDS, GeneratedVisitor) - } -} impl serde::Serialize for ExplainExecNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -8547,115 +8367,6 @@ impl<'de> serde::Deserialize<'de> for InListNode { deserializer.deserialize_struct("datafusion.InListNode", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for InexactInfo { - #[allow(deprecated)] - fn serialize(&self, serializer: S) -> std::result::Result - where - S: serde::Serializer, - { - use serde::ser::SerializeStruct; - let mut len = 0; - if self.inexact_variant.is_some() { - len += 1; - } - let mut struct_ser = serializer.serialize_struct("datafusion.InexactInfo", len)?; - if let Some(v) = self.inexact_variant.as_ref() { - match v { - inexact_info::InexactVariant::Value(v) => { - struct_ser.serialize_field("value", ToString::to_string(&v).as_str())?; - } - inexact_info::InexactVariant::ScalarValue(v) => { - struct_ser.serialize_field("scalarValue", v)?; - } - } - } - struct_ser.end() - } -} -impl<'de> serde::Deserialize<'de> for InexactInfo { - #[allow(deprecated)] - fn deserialize(deserializer: D) -> std::result::Result - where - D: serde::Deserializer<'de>, - { - const FIELDS: &[&str] = &[ - "value", - "scalar_value", - "scalarValue", - ]; - - #[allow(clippy::enum_variant_names)] - enum GeneratedField { - Value, - ScalarValue, - } - 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 { - "value" => Ok(GeneratedField::Value), - "scalarValue" | "scalar_value" => Ok(GeneratedField::ScalarValue), - _ => Err(serde::de::Error::unknown_field(value, FIELDS)), - } - } - } - deserializer.deserialize_identifier(GeneratedVisitor) - } - } - struct GeneratedVisitor; - impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = InexactInfo; - - fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct datafusion.InexactInfo") - } - - fn visit_map(self, mut map: V) -> std::result::Result - where - V: serde::de::MapAccess<'de>, - { - let mut inexact_variant__ = None; - while let Some(k) = map.next_key()? { - match k { - GeneratedField::Value => { - if inexact_variant__.is_some() { - return Err(serde::de::Error::duplicate_field("value")); - } - inexact_variant__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| inexact_info::InexactVariant::Value(x.0)); - } - GeneratedField::ScalarValue => { - if inexact_variant__.is_some() { - return Err(serde::de::Error::duplicate_field("scalarValue")); - } - inexact_variant__ = map.next_value::<::std::option::Option<_>>()?.map(inexact_info::InexactVariant::ScalarValue) -; - } - } - } - Ok(InexactInfo { - inexact_variant: inexact_variant__, - }) - } - } - deserializer.deserialize_struct("datafusion.InexactInfo", FIELDS, GeneratedVisitor) - } -} impl serde::Serialize for IntervalMonthDayNanoValue { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -21513,22 +21224,20 @@ impl serde::Serialize for Sharpness { { use serde::ser::SerializeStruct; let mut len = 0; - if self.sharpness_variant.is_some() { + if self.sharpness_info != 0 { + len += 1; + } + if self.val.is_some() { len += 1; } let mut struct_ser = serializer.serialize_struct("datafusion.Sharpness", len)?; - if let Some(v) = self.sharpness_variant.as_ref() { - match v { - sharpness::SharpnessVariant::Exact(v) => { - struct_ser.serialize_field("exact", v)?; - } - sharpness::SharpnessVariant::Inexact(v) => { - struct_ser.serialize_field("inexact", v)?; - } - sharpness::SharpnessVariant::Absent(v) => { - struct_ser.serialize_field("absent", v)?; - } - } + if self.sharpness_info != 0 { + let v = SharpnessInfo::from_i32(self.sharpness_info) + .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.sharpness_info)))?; + struct_ser.serialize_field("sharpnessInfo", &v)?; + } + if let Some(v) = self.val.as_ref() { + struct_ser.serialize_field("val", v)?; } struct_ser.end() } @@ -21540,16 +21249,15 @@ impl<'de> serde::Deserialize<'de> for Sharpness { D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "exact", - "inexact", - "absent", + "sharpness_info", + "sharpnessInfo", + "val", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { - Exact, - Inexact, - Absent, + SharpnessInfo, + Val, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -21571,9 +21279,8 @@ impl<'de> serde::Deserialize<'de> for Sharpness { E: serde::de::Error, { match value { - "exact" => Ok(GeneratedField::Exact), - "inexact" => Ok(GeneratedField::Inexact), - "absent" => Ok(GeneratedField::Absent), + "sharpnessInfo" | "sharpness_info" => Ok(GeneratedField::SharpnessInfo), + "val" => Ok(GeneratedField::Val), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -21593,40 +21300,109 @@ impl<'de> serde::Deserialize<'de> for Sharpness { where V: serde::de::MapAccess<'de>, { - let mut sharpness_variant__ = None; + let mut sharpness_info__ = None; + let mut val__ = None; while let Some(k) = map.next_key()? { match k { - GeneratedField::Exact => { - if sharpness_variant__.is_some() { - return Err(serde::de::Error::duplicate_field("exact")); + GeneratedField::SharpnessInfo => { + if sharpness_info__.is_some() { + return Err(serde::de::Error::duplicate_field("sharpnessInfo")); } - sharpness_variant__ = map.next_value::<::std::option::Option<_>>()?.map(sharpness::SharpnessVariant::Exact) -; + sharpness_info__ = Some(map.next_value::()? as i32); } - GeneratedField::Inexact => { - if sharpness_variant__.is_some() { - return Err(serde::de::Error::duplicate_field("inexact")); + GeneratedField::Val => { + if val__.is_some() { + return Err(serde::de::Error::duplicate_field("val")); } - sharpness_variant__ = map.next_value::<::std::option::Option<_>>()?.map(sharpness::SharpnessVariant::Inexact) -; - } - GeneratedField::Absent => { - if sharpness_variant__.is_some() { - return Err(serde::de::Error::duplicate_field("absent")); - } - sharpness_variant__ = map.next_value::<::std::option::Option<_>>()?.map(sharpness::SharpnessVariant::Absent) -; + val__ = map.next_value()?; } } } Ok(Sharpness { - sharpness_variant: sharpness_variant__, + sharpness_info: sharpness_info__.unwrap_or_default(), + val: val__, }) } } deserializer.deserialize_struct("datafusion.Sharpness", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for SharpnessInfo { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + let variant = match self { + Self::Exact => "EXACT", + Self::Inexact => "INEXACT", + Self::Absent => "ABSENT", + }; + serializer.serialize_str(variant) + } +} +impl<'de> serde::Deserialize<'de> for SharpnessInfo { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "EXACT", + "INEXACT", + "ABSENT", + ]; + + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = SharpnessInfo; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + fn visit_i64(self, v: i64) -> std::result::Result + where + E: serde::de::Error, + { + use std::convert::TryFrom; + i32::try_from(v) + .ok() + .and_then(SharpnessInfo::from_i32) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) + }) + } + + fn visit_u64(self, v: u64) -> std::result::Result + where + E: serde::de::Error, + { + use std::convert::TryFrom; + i32::try_from(v) + .ok() + .and_then(SharpnessInfo::from_i32) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) + }) + } + + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "EXACT" => Ok(SharpnessInfo::Exact), + "INEXACT" => Ok(SharpnessInfo::Inexact), + "ABSENT" => Ok(SharpnessInfo::Absent), + _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + } + } + } + deserializer.deserialize_any(GeneratedVisitor) + } +} impl serde::Serialize for SimilarToNode { #[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 f3b0d6470863..19cbe4adad35 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2174,60 +2174,11 @@ pub struct PartitionStats { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Sharpness { - #[prost(oneof = "sharpness::SharpnessVariant", tags = "1, 2, 3")] - pub sharpness_variant: ::core::option::Option, -} -/// Nested message and enum types in `Sharpness`. -pub mod sharpness { - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] - pub enum SharpnessVariant { - #[prost(message, tag = "1")] - Exact(super::ExactInfo), - #[prost(message, tag = "2")] - Inexact(super::InexactInfo), - #[prost(message, tag = "3")] - Absent(super::AbsentInfo), - } -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ExactInfo { - #[prost(oneof = "exact_info::ExactVariant", tags = "1, 2")] - pub exact_variant: ::core::option::Option, -} -/// Nested message and enum types in `ExactInfo`. -pub mod exact_info { - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] - pub enum ExactVariant { - #[prost(int64, tag = "1")] - Value(i64), - #[prost(message, tag = "2")] - ScalarValue(super::ScalarValue), - } -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct InexactInfo { - #[prost(oneof = "inexact_info::InexactVariant", tags = "1, 2")] - pub inexact_variant: ::core::option::Option, -} -/// Nested message and enum types in `InexactInfo`. -pub mod inexact_info { - #[allow(clippy::derive_partial_eq_without_eq)] - #[derive(Clone, PartialEq, ::prost::Oneof)] - pub enum InexactVariant { - #[prost(int64, tag = "1")] - Value(i64), - #[prost(message, tag = "2")] - ScalarValue(super::ScalarValue), - } + #[prost(enumeration = "SharpnessInfo", tag = "1")] + pub sharpness_info: i32, + #[prost(message, optional, tag = "2")] + pub val: ::core::option::Option, } -/// No fields needed for Absent -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct AbsentInfo {} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Statistics { @@ -3180,3 +3131,32 @@ impl JoinSide { } } } +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum SharpnessInfo { + Exact = 0, + Inexact = 1, + Absent = 2, +} +impl SharpnessInfo { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + SharpnessInfo::Exact => "EXACT", + SharpnessInfo::Inexact => "INEXACT", + SharpnessInfo::Absent => "ABSENT", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "EXACT" => Some(Self::Exact), + "INEXACT" => Some(Self::Inexact), + "ABSENT" => Some(Self::Absent), + _ => None, + } + } +} diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index d7c470ea333a..429817683678 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -41,10 +41,14 @@ use datafusion::physical_plan::{ functions, Partitioning, }; use datafusion::physical_plan::{ColumnStatistics, PhysicalExpr, Statistics}; +use datafusion_common::stats::Sharpness; +use datafusion_common::ScalarValue; use datafusion_common::{not_impl_err, DataFusionError, Result}; use object_store::path::Path; use object_store::ObjectMeta; use std::convert::{TryFrom, TryInto}; +use std::fmt; +use std::fmt::Display; use std::ops::Deref; use std::sync::Arc; @@ -582,18 +586,102 @@ impl TryFrom<&protobuf::FileGroup> for Vec { impl From<&protobuf::ColumnStats> for ColumnStatistics { fn from(cs: &protobuf::ColumnStats) -> ColumnStatistics { ColumnStatistics { - null_count: if cs.null_count.is_empty() { - None + null_count: if cs.null_count.is_none() { + Sharpness::Absent } else { - Some(cs.null_count[0] as usize) + protobuf::Sharpness::from(cs.null_count.clone().unwrap()).into() }, - max_value: cs.max_value.as_ref().map(|m| m.try_into().unwrap()), - min_value: cs.min_value.as_ref().map(|m| m.try_into().unwrap()), - distinct_count: if cs.distinct_count.is_empty() { - None + max_value: if cs.max_value.is_none() { + Sharpness::Absent } else { - Some(cs.distinct_count[0] as usize) + protobuf::Sharpness::from(cs.max_value.clone().unwrap()).into() }, + min_value: if cs.min_value.is_none() { + Sharpness::Absent + } else { + protobuf::Sharpness::from(cs.min_value.clone().unwrap()).into() + }, + distinct_count: if cs.distinct_count.is_none() { + Sharpness::Absent + } else { + protobuf::Sharpness::from(cs.distinct_count.clone().unwrap()).into() + }, + } + } +} + +impl From for Sharpness { + fn from(s: protobuf::Sharpness) -> Self { + let sharpness_type = + if let Some(s_type) = protobuf::SharpnessInfo::from_i32(s.sharpness_info) { + s_type + } else { + return Sharpness::Absent; + }; + match sharpness_type { + protobuf::SharpnessInfo::Exact => { + if s.val.is_none() { + Sharpness::Absent + } else { + if let Ok(ScalarValue::UInt64(Some(val))) = + ScalarValue::try_from(&s.val.unwrap()) + { + Sharpness::Exact(val as usize) + } else { + Sharpness::Absent + } + } + } + protobuf::SharpnessInfo::Inexact => { + if s.val.is_none() { + Sharpness::Absent + } else { + if let Ok(ScalarValue::UInt64(Some(val))) = + ScalarValue::try_from(&s.val.unwrap()) + { + Sharpness::Inexact(val as usize) + } else { + Sharpness::Absent + } + } + } + protobuf::SharpnessInfo::Absent => Sharpness::Absent, + } + } +} + +impl From for Sharpness { + fn from(s: protobuf::Sharpness) -> Self { + let sharpness_type = + if let Some(s_type) = protobuf::SharpnessInfo::from_i32(s.sharpness_info) { + s_type + } else { + return Sharpness::Absent; + }; + match sharpness_type { + protobuf::SharpnessInfo::Exact => { + if s.val.is_none() { + Sharpness::Absent + } else { + if let Ok(val) = ScalarValue::try_from(&s.val.unwrap()) { + Sharpness::Exact(val) + } else { + Sharpness::Absent + } + } + } + protobuf::SharpnessInfo::Inexact => { + if s.val.is_none() { + Sharpness::Absent + } else { + if let Ok(val) = ScalarValue::try_from(&s.val.unwrap()) { + Sharpness::Inexact(val) + } else { + Sharpness::Absent + } + } + } + protobuf::SharpnessInfo::Absent => Sharpness::Absent, } } } @@ -612,23 +700,21 @@ impl TryFrom<&protobuf::Statistics> for Statistics { fn try_from(s: &protobuf::Statistics) -> Result { // Keep it sync with Statistics::to_proto - let none_value = -1_i64; let column_statistics = s.column_stats.iter().map(|s| s.into()).collect::>(); Ok(Statistics { - num_rows: if s.num_rows == none_value { - None + num_rows: if s.num_rows.is_none() { + Sharpness::Absent } else { - Some(s.num_rows as usize) + protobuf::Sharpness::from(s.num_rows.clone().unwrap()).into() }, - total_byte_size: if s.total_byte_size == none_value { - None + total_byte_size: if s.total_byte_size.is_none() { + Sharpness::Absent } else { - Some(s.total_byte_size as usize) + protobuf::Sharpness::from(s.total_byte_size.clone().unwrap()).into() }, // No column statistic (None) is encoded with empty array column_statistics, - is_exact: s.is_exact, }) } } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 54c191fc941d..0464bc7905d2 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1542,6 +1542,7 @@ mod roundtrip_tests { prelude::SessionContext, scalar::ScalarValue, }; + use datafusion_common::stats::Sharpness; use datafusion_common::Result; use datafusion_expr::{ Accumulator, AccumulatorFactoryFunction, AggregateUDF, ReturnTypeFunction, @@ -1961,10 +1962,9 @@ mod roundtrip_tests { 1024, )]], statistics: Statistics { - num_rows: Some(100), - total_byte_size: Some(1024), + num_rows: Sharpness::Exact(100), + total_byte_size: Sharpness::Exact(1024), column_statistics: Statistics::unbounded_column_statistics(&schema), - is_exact: false, }, projection: None, limit: None, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index fc797ec08959..0584e583f197 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -63,17 +63,22 @@ use datafusion::physical_plan::expressions::{ }; use datafusion::physical_plan::{AggregateExpr, PhysicalExpr}; -use crate::protobuf::{self, physical_window_expr_node}; use crate::protobuf::{ physical_aggregate_expr_node, PhysicalSortExprNode, PhysicalSortExprNodeCollection, ScalarValue, }; +use crate::{ + logical_plan::from_proto::Error, + protobuf::{self, physical_window_expr_node, scalar_value::Value}, +}; use datafusion::logical_expr::BuiltinScalarFunction; use datafusion::physical_expr::expressions::{GetFieldAccessExpr, GetIndexedFieldExpr}; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::joins::utils::JoinSide; use datafusion::physical_plan::udaf::AggregateFunctionExpr; -use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; +use datafusion_common::{ + internal_err, not_impl_err, stats::Sharpness, DataFusionError, Result, +}; impl TryFrom> for protobuf::PhysicalExprNode { type Error = DataFusionError; @@ -644,26 +649,80 @@ impl TryFrom<&[PartitionedFile]> for protobuf::FileGroup { } } -impl From<&ColumnStatistics> for protobuf::ColumnStats { - fn from(cs: &ColumnStatistics) -> protobuf::ColumnStats { - protobuf::ColumnStats { - min_value: cs.min_value.as_ref().map(|m| m.try_into().unwrap()), - max_value: cs.max_value.as_ref().map(|m| m.try_into().unwrap()), - null_count: cs.null_count.map(|n| vec![n as u32]).unwrap_or(vec![]), - distinct_count: cs.distinct_count.map(|n| vec![n as u32]).unwrap_or(vec![]), +impl From<&Sharpness> for protobuf::Sharpness { + fn from(s: &Sharpness) -> protobuf::Sharpness { + match s { + Sharpness::Exact(val) => protobuf::Sharpness { + sharpness_info: protobuf::SharpnessInfo::Exact.into(), + val: Some(ScalarValue { + value: Some(Value::Uint64Value(val.clone() as u64)), + }), + }, + Sharpness::Inexact(val) => protobuf::Sharpness { + sharpness_info: protobuf::SharpnessInfo::Inexact.into(), + val: Some(ScalarValue { + value: Some(Value::Uint64Value(val.clone() as u64)), + }), + }, + Sharpness::Absent => protobuf::Sharpness { + sharpness_info: protobuf::SharpnessInfo::Absent.into(), + val: Some(ScalarValue { value: None }), + }, + } + } +} + +impl From<&Sharpness> for protobuf::Sharpness { + fn from(s: &Sharpness) -> protobuf::Sharpness { + match s { + Sharpness::Exact(val) | Sharpness::Inexact(val) => { + let res: Result = val.try_into().map_err(|_| { + DataFusionError::Internal("Undefined sharpness".to_owned()) + }); + if res.is_err() { + return protobuf::Sharpness { + sharpness_info: protobuf::SharpnessInfo::Absent.into(), + val: Some(ScalarValue { value: None }), + }; + }; + if s.is_exact().unwrap() { + protobuf::Sharpness { + sharpness_info: protobuf::SharpnessInfo::Exact.into(), + val: Some(val.try_into().unwrap()), + } + } else { + protobuf::Sharpness { + sharpness_info: protobuf::SharpnessInfo::Inexact.into(), + val: Some(val.try_into().unwrap()), + } + } + } + Sharpness::Absent => protobuf::Sharpness { + sharpness_info: protobuf::SharpnessInfo::Absent.into(), + val: Some(ScalarValue { value: None }), + }, } } } impl From<&Statistics> for protobuf::Statistics { fn from(s: &Statistics) -> protobuf::Statistics { - let none_value = -1_i64; let column_stats = s.column_statistics.iter().map(|s| s.into()).collect(); protobuf::Statistics { - num_rows: s.num_rows.map(|n| n as i64).unwrap_or(none_value), - total_byte_size: s.total_byte_size.map(|n| n as i64).unwrap_or(none_value), + num_rows: Some(protobuf::Sharpness::from(&s.num_rows)), + total_byte_size: Some(protobuf::Sharpness::from(&s.total_byte_size)), column_stats, - is_exact: s.is_exact, + } + } +} + +impl From<&ColumnStatistics> for protobuf::ColumnStats { + fn from(s: &ColumnStatistics) -> protobuf::ColumnStats { + protobuf::ColumnStats { + min_value: Some(protobuf::Sharpness::from(&s.min_value)), + max_value: Some(protobuf::Sharpness::from(&s.max_value)), + null_count: Some(protobuf::Sharpness::from(&s.null_count)), + distinct_count: Some(protobuf::Sharpness::from(&s.distinct_count)), } } } From cb11054ae1c630f6f64b14641f6de059f8a0786d Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 28 Sep 2023 14:57:01 +0300 Subject: [PATCH 18/53] bugs are fixed --- datafusion/common/src/stats.rs | 40 +++++--------- datafusion/core/src/datasource/statistics.rs | 42 ++++++--------- .../aggregate_statistics.rs | 48 ++++++++++------- .../src/physical_optimizer/join_selection.rs | 8 +-- datafusion/core/src/physical_planner.rs | 4 +- .../tests/custom_sources_cases/statistics.rs | 3 +- datafusion/core/tests/sql/explain_analyze.rs | 2 +- datafusion/physical-plan/src/common.rs | 21 +++----- datafusion/physical-plan/src/filter.rs | 29 +++++----- datafusion/physical-plan/src/joins/utils.rs | 37 +++++-------- datafusion/physical-plan/src/union.rs | 2 +- .../proto/src/physical_plan/from_proto.rs | 54 ++++++++----------- .../proto/src/physical_plan/to_proto.rs | 9 ++-- .../sqllogictest/test_files/explain.slt | 8 +-- 14 files changed, 133 insertions(+), 174 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 658b043bcbae..e68d3826ca37 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -22,10 +22,11 @@ use arrow::datatypes::{DataType, Schema}; use core::fmt::Debug; use std::fmt::{self, Display}; -#[derive(Clone, PartialEq, Eq)] +#[derive(Clone, PartialEq, Eq, Default)] pub enum Sharpness { Exact(T), Inexact(T), + #[default] Absent, } @@ -122,8 +123,8 @@ impl Debug { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { - Sharpness::Exact(inner) => write!(f, "Exact Info:({:?})", inner), - Sharpness::Inexact(inner) => write!(f, "Inexact Info:({:?})", inner), + Sharpness::Exact(inner) => write!(f, "Exact:({:?})", inner), + Sharpness::Inexact(inner) => write!(f, "Approximate:({:?})", inner), Sharpness::Absent => write!(f, "Absent Info"), } } @@ -134,22 +135,13 @@ impl Display { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { - Sharpness::Exact(inner) => write!(f, "Exact Info:({})", inner), - Sharpness::Inexact(inner) => write!(f, "Inexact Info:({})", inner), + Sharpness::Exact(inner) => write!(f, "Exact:({})", inner), + Sharpness::Inexact(inner) => write!(f, "Approximate:({})", inner), Sharpness::Absent => write!(f, "Absent Info"), } } } -impl Default for Sharpness -where - T: Debug + Clone + PartialEq + Eq + Display + PartialOrd, -{ - fn default() -> Self { - Sharpness::Absent - } -} - /// Statistics for a relation /// Fields are optional and can be inexact because the sources /// sometimes provide approximate estimates for performance reasons @@ -187,11 +179,9 @@ impl Statistics { null_count: Sharpness::Absent, max_value: inf .clone() - .map(|val| Sharpness::Inexact(val)) - .unwrap_or(Sharpness::Absent), - min_value: inf - .map(|val| Sharpness::Inexact(val)) + .map(Sharpness::Inexact) .unwrap_or(Sharpness::Absent), + min_value: inf.map(Sharpness::Inexact).unwrap_or(Sharpness::Absent), distinct_count: Sharpness::Absent, } }) @@ -212,12 +202,12 @@ impl Statistics { pub fn make_inexact(self) -> Self { Statistics { num_rows: if let Sharpness::Exact(val) = &self.num_rows { - Sharpness::Inexact(val.clone()) + Sharpness::Inexact(*val) } else { self.num_rows }, total_byte_size: if let Sharpness::Exact(val) = &self.total_byte_size { - Sharpness::Inexact(val.clone()) + Sharpness::Inexact(*val) } else { self.total_byte_size }, @@ -226,7 +216,7 @@ impl Statistics { .iter() .map(|cs| ColumnStatistics { null_count: if let Sharpness::Exact(val) = &cs.null_count { - Sharpness::Inexact(val.clone()) + Sharpness::Inexact(*val) } else { cs.null_count.clone() }, @@ -241,7 +231,7 @@ impl Statistics { cs.min_value.clone() }, distinct_count: if let Sharpness::Exact(val) = &cs.distinct_count { - Sharpness::Inexact(val.clone()) + Sharpness::Inexact(*val) } else { cs.distinct_count.clone() }, @@ -293,11 +283,9 @@ impl ColumnStatistics { null_count: Sharpness::Absent, max_value: null .clone() - .map(|val| Sharpness::Inexact(val)) - .unwrap_or(Sharpness::Absent), - min_value: null - .map(|val| Sharpness::Inexact(val)) + .map(Sharpness::Inexact) .unwrap_or(Sharpness::Absent), + min_value: null.map(Sharpness::Inexact).unwrap_or(Sharpness::Absent), distinct_count: Sharpness::Absent, } } diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 5ac2d7e6697c..3c1d340ecc3a 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -35,7 +35,7 @@ pub async fn get_statistics_with_limit( ) -> Result<(Vec, Statistics)> { let mut result_files = vec![]; - let mut null_counts = vec![Sharpness::Exact(0 as usize); file_schema.fields().len()]; + let mut null_counts = vec![Sharpness::Exact(0_usize); file_schema.fields().len()]; let mut has_statistics = false; let (mut max_values, mut min_values) = create_max_min_accs(&file_schema); @@ -54,22 +54,20 @@ pub async fn get_statistics_with_limit( num_rows = if let Some(exactness) = num_rows.is_exact() { if exactness { if file_stats.num_rows == Sharpness::Absent { - Sharpness::Exact(0 as usize) + Sharpness::Exact(0_usize) .add(&Sharpness::Exact(num_rows.get_value().unwrap())) } else { file_stats .num_rows .add(&Sharpness::Exact(num_rows.get_value().unwrap())) } + } else if file_stats.num_rows == Sharpness::Absent { + Sharpness::Exact(0_usize) + .add(&Sharpness::Inexact(num_rows.get_value().unwrap())) } else { - if file_stats.num_rows == Sharpness::Absent { - Sharpness::Exact(0 as usize) - .add(&Sharpness::Inexact(num_rows.get_value().unwrap())) - } else { - file_stats - .num_rows - .add(&Sharpness::Inexact(num_rows.get_value().unwrap())) - } + file_stats + .num_rows + .add(&Sharpness::Inexact(num_rows.get_value().unwrap())) } } else { file_stats.num_rows @@ -77,22 +75,20 @@ pub async fn get_statistics_with_limit( total_byte_size = if let Some(exactness) = total_byte_size.is_exact() { if exactness { if file_stats.total_byte_size == Sharpness::Absent { - Sharpness::Exact(0 as usize) + Sharpness::Exact(0_usize) .add(&Sharpness::Exact(total_byte_size.get_value().unwrap())) } else { file_stats .total_byte_size .add(&Sharpness::Exact(total_byte_size.get_value().unwrap())) } + } else if file_stats.total_byte_size == Sharpness::Absent { + Sharpness::Exact(0_usize) + .add(&Sharpness::Inexact(total_byte_size.get_value().unwrap())) } else { - if file_stats.total_byte_size == Sharpness::Absent { - Sharpness::Exact(0 as usize) - .add(&Sharpness::Inexact(total_byte_size.get_value().unwrap())) - } else { - file_stats - .total_byte_size - .add(&Sharpness::Inexact(total_byte_size.get_value().unwrap())) - } + file_stats + .total_byte_size + .add(&Sharpness::Inexact(total_byte_size.get_value().unwrap())) } } else { file_stats.total_byte_size @@ -203,12 +199,8 @@ pub(crate) fn get_col_stats( }; ColumnStatistics { null_count: null_counts[i].clone(), - max_value: max_value - .map(|val| Sharpness::Exact(val)) - .unwrap_or(Sharpness::Absent), - min_value: min_value - .map(|val| Sharpness::Exact(val)) - .unwrap_or(Sharpness::Absent), + max_value: max_value.map(Sharpness::Exact).unwrap_or(Sharpness::Absent), + min_value: min_value.map(Sharpness::Exact).unwrap_or(Sharpness::Absent), distinct_count: Sharpness::Absent, } }) diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index b95d294b33e5..c226f7d15b0b 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -19,16 +19,13 @@ use std::sync::Arc; use crate::config::ConfigOptions; -use datafusion_common::stats::Sharpness; use datafusion_common::tree_node::TreeNode; use datafusion_expr::utils::COUNT_STAR_EXPANSION; -use crate::physical_plan::aggregates::{AggregateExec, AggregateMode}; +use crate::physical_plan::aggregates::AggregateExec; use crate::physical_plan::empty::EmptyExec; use crate::physical_plan::projection::ProjectionExec; -use crate::physical_plan::{ - expressions, AggregateExpr, ColumnStatistics, ExecutionPlan, Statistics, -}; +use crate::physical_plan::{expressions, AggregateExpr, ExecutionPlan, Statistics}; use crate::scalar::ScalarValue; use super::optimizer::PhysicalOptimizerRule; @@ -108,7 +105,6 @@ impl PhysicalOptimizerRule for AggregateStatistics { /// assert if the node passed as argument is a final `AggregateExec` node that can be optimized: /// - its child (with possible intermediate layers) is a partial `AggregateExec` node /// - they both have no grouping expression -/// - the statistics are exact /// If this is the case, return a ref to the partial `AggregateExec`, else `None`. /// We would have preferred to return a casted ref to AggregateExec but the recursion requires /// the `ExecutionPlan.children()` method that returns an owned reference. @@ -126,11 +122,7 @@ fn take_optimizable(node: &dyn ExecutionPlan) -> Result Option<(ScalarValue, &'static str)> { + if !stats.num_rows.is_exact().unwrap_or(false) { + return None; + } if let (Some(num_rows), Some(casted_expr)) = ( stats.num_rows.get_value(), agg_expr.as_any().downcast_ref::(), @@ -177,6 +172,13 @@ fn take_optimizable_column_count( stats: &Statistics, ) -> Option<(ScalarValue, String)> { let col_stats = &stats.column_statistics; + if !stats.num_rows.is_exact().unwrap_or(false) + || !col_stats + .iter() + .all(|cs| cs.null_count.is_exact().unwrap_or(false)) + { + return None; + } if let (Some(num_rows), Some(casted_expr)) = ( stats.num_rows.get_value(), agg_expr.as_any().downcast_ref::(), @@ -205,6 +207,12 @@ fn take_optimizable_min( stats: &Statistics, ) -> Option<(ScalarValue, String)> { let col_stats = &stats.column_statistics; + if !col_stats + .iter() + .all(|cs| cs.min_value.is_exact().unwrap_or(false)) + { + return None; + } if let Some(casted_expr) = agg_expr.as_any().downcast_ref::() { if casted_expr.expressions().len() == 1 { // TODO optimize with exprs other than Column @@ -233,6 +241,12 @@ fn take_optimizable_max( stats: &Statistics, ) -> Option<(ScalarValue, String)> { let col_stats = &stats.column_statistics; + if !col_stats + .iter() + .all(|cs| cs.max_value.is_exact().unwrap_or(false)) + { + return None; + } if let Some(casted_expr) = agg_expr.as_any().downcast_ref::() { if casted_expr.expressions().len() == 1 { // TODO optimize with exprs other than Column @@ -263,7 +277,7 @@ mod tests { use datafusion_common::cast::as_int64_array; use datafusion_physical_expr::expressions::cast; use datafusion_physical_expr::PhysicalExpr; - use datafusion_physical_plan::displayable; + use datafusion_physical_plan::aggregates::AggregateMode; use crate::error::Result; use crate::logical_expr::Operator; @@ -305,16 +319,10 @@ mod tests { let session_ctx = SessionContext::new(); let state = session_ctx.state(); let plan: Arc = Arc::new(plan); - fn print_plan(plan: &Arc) -> Result<()> { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); - Ok(()) - } - print_plan(&plan); + let optimized = AggregateStatistics::new() .optimize(Arc::clone(&plan), state.config_options())?; - print_plan(&optimized); + // A ProjectionExec is a sign that the count optimization was applied assert!(optimized.as_any().is::()); diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 3ef49cd92da7..f8f30e429124 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -644,7 +644,7 @@ mod tests_statistical { ) -> Vec { vec![ColumnStatistics { distinct_count: distinct_count - .map(|val| Sharpness::Inexact(val)) + .map(Sharpness::Inexact) .unwrap_or(Sharpness::Absent), min_value: min .map(|size| Sharpness::Inexact(ScalarValue::UInt64(Some(size)))) @@ -925,9 +925,9 @@ mod tests_statistical { " HashJoinExec: mode=CollectLeft, join_type=Right, on=[(small_col@1, medium_col@0)]", " ProjectionExec: expr=[big_col@1 as big_col, small_col@0 as small_col]", " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(small_col@0, big_col@0)]", - " StatisticsExec: col_count=1, row_count=Inexact Info:(1000)", - " StatisticsExec: col_count=1, row_count=Inexact Info:(100000)", - " StatisticsExec: col_count=1, row_count=Inexact Info:(0)", + " StatisticsExec: col_count=1, row_count=Approximate:(1000)", + " StatisticsExec: col_count=1, row_count=Approximate:(100000)", + " StatisticsExec: col_count=1, row_count=Approximate:(0)", "", ]; assert_optimized!(expected, join); diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index bd7d1d8acfc9..0193b1d6555b 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2755,8 +2755,10 @@ mod tests { digraph { 1[shape=box label="ProjectionExec: expr=[id@0 + 2 as employee.id + Int32(2)]", tooltip=""] - 2[shape=box label="EmptyExec: produce_one_row=false", tooltip=""] + 2[shape=box label="RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", tooltip=""] 1 -> 2 [arrowhead=none, arrowtail=normal, dir=back] + 3[shape=box label="EmptyExec: produce_one_row=false", tooltip=""] + 2 -> 3 [arrowhead=none, arrowtail=normal, dir=back] } // End DataFusion GraphViz Plan "#; diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index d4bba7ddc57b..3bbe880a7949 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -227,9 +227,8 @@ async fn sql_filter() -> Result<()> { .unwrap(); let physical_plan = df.create_physical_plan().await.unwrap(); - let stats = physical_plan.statistics()?; - assert_eq!(stats.num_rows, Sharpness::Exact(1)); + assert_eq!(stats.num_rows, Sharpness::Inexact(1)); Ok(()) } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 06120c01ce86..4bf1567099bb 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -826,5 +826,5 @@ async fn csv_explain_analyze_with_statistics() { .to_string(); // should contain scan statistics - assert_contains!(&formatted, ", statistics=[]"); + assert_contains!(&formatted, ", statistics=[Number of Rows=Absent Info, Number of Bytes=Absent Info, Columns Statistics=[ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Utf8(NULL)), min_value: Exact:(Utf8(NULL)), distinct_count: Absent Info }]]"); } diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index b13ce7e2937a..60b8b9346a52 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -156,17 +156,8 @@ pub fn compute_record_batch_statistics( for partition in batches.iter() { for batch in partition { for (stat_index, col_index) in projection.iter().enumerate() { - column_statistics[stat_index].null_count = if let Sharpness::Exact(val) = - &column_statistics[stat_index].null_count - { - Sharpness::Exact(batch.column(*col_index).null_count() + val) - } else if let Sharpness::Inexact(val) = - &column_statistics[stat_index].null_count - { - Sharpness::Inexact(batch.column(*col_index).null_count() + val) - } else { - Sharpness::Exact(0) - }; + column_statistics[stat_index].null_count = + Sharpness::Exact(batch.column(*col_index).null_count()); } } } @@ -712,14 +703,14 @@ mod tests { column_statistics: vec![ ColumnStatistics { distinct_count: Sharpness::Absent, - max_value: Sharpness::Exact(ScalarValue::Float32(None)), - min_value: Sharpness::Exact(ScalarValue::Float32(None)), + max_value: Sharpness::Inexact(ScalarValue::Float32(None)), + min_value: Sharpness::Inexact(ScalarValue::Float32(None)), null_count: Sharpness::Exact(0), }, ColumnStatistics { distinct_count: Sharpness::Absent, - max_value: Sharpness::Exact(ScalarValue::Float64(None)), - min_value: Sharpness::Exact(ScalarValue::Float64(None)), + max_value: Sharpness::Inexact(ScalarValue::Float64(None)), + min_value: Sharpness::Inexact(ScalarValue::Float64(None)), null_count: Sharpness::Exact(0), }, ], diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 7e6b1d2b3134..30f758ae87f3 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -216,9 +216,14 @@ impl ExecutionPlan for FilterExec { let analysis_ctx = analyze(predicate, input_analysis_ctx)?; let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); - let num_rows = num_rows.map(|num| (num as f64 * selectivity).ceil() as usize); - let total_byte_size = - total_byte_size.map(|size| (size as f64 * selectivity).ceil() as usize); + let num_rows = match num_rows.get_value() { + Some(nr) => Sharpness::Inexact((nr as f64 * selectivity).ceil() as usize), + None => Sharpness::Absent, + }; + let total_byte_size = match total_byte_size.get_value() { + Some(tbs) => Sharpness::Inexact((tbs as f64 * selectivity).ceil() as usize), + None => Sharpness::Absent, + }; if let Some(analysis_boundaries) = analysis_ctx.boundaries { let column_statistics = collect_new_statistics( @@ -262,18 +267,16 @@ fn collect_new_statistics( )| { let closed_interval = interval.close_bounds(); ColumnStatistics { - null_count: input_column_stats[idx].null_count.clone(), - max_value: if input_column_stats[idx].max_value.is_exact().is_some() { - Sharpness::Exact(closed_interval.upper.value) - } else { - Sharpness::Inexact(closed_interval.upper.value) + null_count: match input_column_stats[idx].null_count.get_value() { + Some(nc) => Sharpness::Inexact(nc), + None => Sharpness::Absent, }, - min_value: if input_column_stats[idx].max_value.is_exact().is_some() { - Sharpness::Exact(closed_interval.lower.value) - } else { - Sharpness::Inexact(closed_interval.lower.value) + max_value: Sharpness::Inexact(closed_interval.upper.value), + min_value: Sharpness::Inexact(closed_interval.lower.value), + distinct_count: match distinct_count.get_value() { + Some(dc) => Sharpness::Inexact(dc), + None => Sharpness::Absent, }, - distinct_count, } }, ) diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 56c389c445c4..3b7fb7a892e9 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -17,7 +17,6 @@ //! Join related functionality used both on logical and physical plans -use std::cmp::max; use std::collections::HashSet; use std::fmt::{Display, Formatter}; use std::future::Future; @@ -843,7 +842,7 @@ fn estimate_inner_join_cardinality( _ => Some(Sharpness::Inexact(0)), }; } - if left_stat.max_value.get_value()? > right_stat.min_value.get_value()? { + if left_stat.max_value.get_value()? < right_stat.min_value.get_value()? { return match ( left_stat.max_value.is_exact().unwrap(), right_stat.min_value.is_exact().unwrap(), @@ -1589,16 +1588,14 @@ mod tests { ) -> Statistics { if is_exact { Statistics { - num_rows: num_rows - .map(|size| Sharpness::Exact(size)) - .unwrap_or(Sharpness::Absent), + num_rows: num_rows.map(Sharpness::Exact).unwrap_or(Sharpness::Absent), column_statistics: column_stats, total_byte_size: Sharpness::Absent, } } else { Statistics { num_rows: num_rows - .map(|size| Sharpness::Inexact(size)) + .map(Sharpness::Inexact) .unwrap_or(Sharpness::Absent), column_statistics: column_stats, total_byte_size: Sharpness::Absent, @@ -1613,7 +1610,7 @@ mod tests { ) -> ColumnStatistics { ColumnStatistics { distinct_count: distinct_count - .map(|count| Sharpness::Inexact(count)) + .map(Sharpness::Inexact) .unwrap_or(Sharpness::Absent), min_value: min .map(|size| Sharpness::Inexact(ScalarValue::Int64(Some(size)))) @@ -1699,9 +1696,9 @@ mod tests { // so one of them is always going to work, this just proves negative // ranges with bigger absolute values are not are not accidentally used). ( - (10, Some(10), Some(0), None), + (10, Some(-10), Some(0), None), (10, Some(0), Some(10), Some(5)), - Some(Sharpness::Inexact(20)), // It would have been ten if we have used abs(range(left)) + Some(Sharpness::Inexact(10)), ), // range(left) = 1, range(right) = 1 ( @@ -1732,22 +1729,12 @@ mod tests { ( (10, Some(0), Some(10), None), (10, Some(11), Some(20), None), - None, + Some(Sharpness::Inexact(0)), ), ( (10, Some(11), Some(20), None), (10, Some(0), Some(10), None), - None, - ), - ( - (10, Some(5), Some(10), Some(10)), - (10, Some(11), Some(3), Some(10)), - None, - ), - ( - (10, Some(10), Some(5), Some(10)), - (10, Some(3), Some(7), Some(10)), - None, + Some(Sharpness::Inexact(0)), ), // distinct(left) = 0, distinct(right) = 0 ( @@ -1782,7 +1769,7 @@ mod tests { column_statistics: right_col_stats.clone(), }, ), - expected_cardinality + expected_cardinality.clone() ); // We should also be able to use join_cardinality to get the same results @@ -1799,11 +1786,13 @@ mod tests { partial_join_stats .clone() .map(|s| Sharpness::Inexact(s.num_rows)), - expected_cardinality + expected_cardinality.clone() ); assert_eq!( partial_join_stats.map(|s| s.column_statistics), - expected_cardinality.map(|_| [left_col_stats, right_col_stats].concat()) + expected_cardinality + .clone() + .map(|_| [left_col_stats, right_col_stats].concat()) ); } Ok(()) diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 16853a7d9155..c4ae4a82460c 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -43,8 +43,8 @@ use super::{ SendableRecordBatchStream, Statistics, }; use crate::common::get_meet_of_orderings; +use crate::metrics::BaselineMetrics; use crate::stream::ObservedStream; -use crate::{expressions, metrics::BaselineMetrics}; use datafusion_common::Result; use datafusion_execution::TaskContext; use tokio::macros::support::thread_rng_n; diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 429817683678..25f7f9ce176c 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -47,8 +47,6 @@ use datafusion_common::{not_impl_err, DataFusionError, Result}; use object_store::path::Path; use object_store::ObjectMeta; use std::convert::{TryFrom, TryInto}; -use std::fmt; -use std::fmt::Display; use std::ops::Deref; use std::sync::Arc; @@ -589,22 +587,22 @@ impl From<&protobuf::ColumnStats> for ColumnStatistics { null_count: if cs.null_count.is_none() { Sharpness::Absent } else { - protobuf::Sharpness::from(cs.null_count.clone().unwrap()).into() + cs.null_count.clone().unwrap().into() }, max_value: if cs.max_value.is_none() { Sharpness::Absent } else { - protobuf::Sharpness::from(cs.max_value.clone().unwrap()).into() + cs.max_value.clone().unwrap().into() }, min_value: if cs.min_value.is_none() { Sharpness::Absent } else { - protobuf::Sharpness::from(cs.min_value.clone().unwrap()).into() + cs.min_value.clone().unwrap().into() }, distinct_count: if cs.distinct_count.is_none() { Sharpness::Absent } else { - protobuf::Sharpness::from(cs.distinct_count.clone().unwrap()).into() + cs.distinct_count.clone().unwrap().into() }, } } @@ -622,27 +620,23 @@ impl From for Sharpness { protobuf::SharpnessInfo::Exact => { if s.val.is_none() { Sharpness::Absent + } else if let Ok(ScalarValue::UInt64(Some(val))) = + ScalarValue::try_from(&s.val.unwrap()) + { + Sharpness::Exact(val as usize) } else { - if let Ok(ScalarValue::UInt64(Some(val))) = - ScalarValue::try_from(&s.val.unwrap()) - { - Sharpness::Exact(val as usize) - } else { - Sharpness::Absent - } + Sharpness::Absent } } protobuf::SharpnessInfo::Inexact => { if s.val.is_none() { Sharpness::Absent + } else if let Ok(ScalarValue::UInt64(Some(val))) = + ScalarValue::try_from(&s.val.unwrap()) + { + Sharpness::Inexact(val as usize) } else { - if let Ok(ScalarValue::UInt64(Some(val))) = - ScalarValue::try_from(&s.val.unwrap()) - { - Sharpness::Inexact(val as usize) - } else { - Sharpness::Absent - } + Sharpness::Absent } } protobuf::SharpnessInfo::Absent => Sharpness::Absent, @@ -662,23 +656,19 @@ impl From for Sharpness { protobuf::SharpnessInfo::Exact => { if s.val.is_none() { Sharpness::Absent + } else if let Ok(val) = ScalarValue::try_from(&s.val.unwrap()) { + Sharpness::Exact(val) } else { - if let Ok(val) = ScalarValue::try_from(&s.val.unwrap()) { - Sharpness::Exact(val) - } else { - Sharpness::Absent - } + Sharpness::Absent } } protobuf::SharpnessInfo::Inexact => { if s.val.is_none() { Sharpness::Absent + } else if let Ok(val) = ScalarValue::try_from(&s.val.unwrap()) { + Sharpness::Inexact(val) } else { - if let Ok(val) = ScalarValue::try_from(&s.val.unwrap()) { - Sharpness::Inexact(val) - } else { - Sharpness::Absent - } + Sharpness::Absent } } protobuf::SharpnessInfo::Absent => Sharpness::Absent, @@ -706,12 +696,12 @@ impl TryFrom<&protobuf::Statistics> for Statistics { num_rows: if s.num_rows.is_none() { Sharpness::Absent } else { - protobuf::Sharpness::from(s.num_rows.clone().unwrap()).into() + s.num_rows.clone().unwrap().into() }, total_byte_size: if s.total_byte_size.is_none() { Sharpness::Absent } else { - protobuf::Sharpness::from(s.total_byte_size.clone().unwrap()).into() + s.total_byte_size.clone().unwrap().into() }, // No column statistic (None) is encoded with empty array column_statistics, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 0584e583f197..290d93693db9 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -63,14 +63,11 @@ use datafusion::physical_plan::expressions::{ }; use datafusion::physical_plan::{AggregateExpr, PhysicalExpr}; +use crate::protobuf::{self, physical_window_expr_node, scalar_value::Value}; use crate::protobuf::{ physical_aggregate_expr_node, PhysicalSortExprNode, PhysicalSortExprNodeCollection, ScalarValue, }; -use crate::{ - logical_plan::from_proto::Error, - protobuf::{self, physical_window_expr_node, scalar_value::Value}, -}; use datafusion::logical_expr::BuiltinScalarFunction; use datafusion::physical_expr::expressions::{GetFieldAccessExpr, GetIndexedFieldExpr}; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; @@ -655,13 +652,13 @@ impl From<&Sharpness> for protobuf::Sharpness { Sharpness::Exact(val) => protobuf::Sharpness { sharpness_info: protobuf::SharpnessInfo::Exact.into(), val: Some(ScalarValue { - value: Some(Value::Uint64Value(val.clone() as u64)), + value: Some(Value::Uint64Value(*val as u64)), }), }, Sharpness::Inexact(val) => protobuf::Sharpness { sharpness_info: protobuf::SharpnessInfo::Inexact.into(), val: Some(ScalarValue { - value: Some(Value::Uint64Value(val.clone() as u64)), + value: Some(Value::Uint64Value(*val as u64)), }), }, Sharpness::Absent => protobuf::Sharpness { diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index b1ba1eb36d11..7110252c3f36 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -265,8 +265,8 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[rows=10, bytes=None, exact=false] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[] +GlobalLimitExec: skip=0, fetch=10, statistics=[Number of Rows=Exact:(10), Number of Bytes=Absent Info, Columns Statistics=[ColumnStatistics { null_count: Absent Info, max_value: Approximate:(Int32(NULL)), min_value: Approximate:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Absent Info, max_value: Approximate:(Int32(NULL)), min_value: Approximate:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Absent Info, max_value: Approximate:(Int32(NULL)), min_value: Approximate:(Int32(NULL)), distinct_count: Absent Info }]] +--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Number of Rows=Absent Info, Number of Bytes=Absent Info, Columns Statistics=[ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }]] # Parquet scan with statistics collected statement ok @@ -279,8 +279,8 @@ query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[rows=8, bytes=None, exact=true] ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[rows=8, bytes=None, exact=true] +GlobalLimitExec: skip=0, fetch=10, statistics=[Number of Rows=Exact:(8), Number of Bytes=Absent Info, Columns Statistics=[ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Boolean(NULL)), min_value: Exact:(Boolean(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int64(NULL)), min_value: Exact:(Int64(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Float32(NULL)), min_value: Exact:(Float32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Float64(NULL)), min_value: Exact:(Float64(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Binary(NULL)), min_value: Exact:(Binary(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Binary(NULL)), min_value: Exact:(Binary(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(TimestampNanosecond(NULL, None)), min_value: Exact:(TimestampNanosecond(NULL, None)), distinct_count: Absent Info }]] +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Number of Rows=Exact:(8), Number of Bytes=Absent Info, Columns Statistics=[ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Boolean(NULL)), min_value: Exact:(Boolean(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int64(NULL)), min_value: Exact:(Int64(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Float32(NULL)), min_value: Exact:(Float32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Float64(NULL)), min_value: Exact:(Float64(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Binary(NULL)), min_value: Exact:(Binary(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Binary(NULL)), min_value: Exact:(Binary(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(TimestampNanosecond(NULL, None)), min_value: Exact:(TimestampNanosecond(NULL, None)), distinct_count: Absent Info }]] statement ok set datafusion.execution.collect_statistics = false; From 946c34105dbc74fe2b02ae3f6ec6f0f499313882 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 28 Sep 2023 16:16:13 +0300 Subject: [PATCH 19/53] negative expr support --- datafusion/common/src/stats.rs | 19 ++++ .../aggregate_statistics.rs | 82 ++++++++--------- datafusion/physical-expr/src/analysis.rs | 4 +- .../physical-expr/src/expressions/negative.rs | 88 ++++++++++++++++++- .../src/intervals/interval_aritmetic.rs | 1 + .../physical-expr/src/intervals/utils.rs | 6 +- 6 files changed, 149 insertions(+), 51 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index e68d3826ca37..c6ef2e036dfc 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -22,6 +22,8 @@ use arrow::datatypes::{DataType, Schema}; use core::fmt::Debug; use std::fmt::{self, Display}; +/// To deal with information whose exactness is not guaranteed, it can be wrapped with [`Sharpness`] +/// to express its reliability, such as in Statistics. #[derive(Clone, PartialEq, Eq, Default)] pub enum Sharpness { Exact(T), @@ -31,6 +33,7 @@ pub enum Sharpness { } impl Sharpness { + /// If the information is known somehow, it return the value. Otherwise, it returns None. pub fn get_value(&self) -> Option { match self { Sharpness::Exact(val) | Sharpness::Inexact(val) => Some(val.clone()), @@ -38,6 +41,7 @@ impl Sharpness { } } + /// Value in the [`Sharpness`] is mapped to the function result wrapped with same exactness state. pub fn map(self, f: F) -> Sharpness where F: Fn(T) -> T, @@ -49,6 +53,8 @@ impl Sharpness { } } + /// Returns Some(true) if the information is exact, or Some(false) if not exact. + /// If the information does not even exist, it returns None. pub fn is_exact(&self) -> Option { match self { Sharpness::Exact(_) => Some(true), @@ -57,6 +63,8 @@ impl Sharpness { } } + /// Returns the greater one between two exact or inexact values. + /// If one of them is a [`Sharpness::Absent`], it returns [`Sharpness::Absent`]. pub fn max(&self, other: &Sharpness) -> Sharpness { match (self, other) { (Sharpness::Exact(a), Sharpness::Exact(b)) => { @@ -71,6 +79,8 @@ impl Sharpness { } } + /// Returns the smaller one between two exact or inexact values. + /// If one of them is a [`Sharpness::Absent`], it returns [`Sharpness::Absent`]. pub fn min(&self, other: &Sharpness) -> Sharpness { match (self, other) { (Sharpness::Exact(a), Sharpness::Exact(b)) => { @@ -87,6 +97,8 @@ impl Sharpness { } impl Sharpness { + /// Calculates the sum of two exact or inexact values in the type of [`usize`]. + /// If one of them is a [`Sharpness::Absent`], it returns [`Sharpness::Absent`]. pub fn add(&self, other: &Sharpness) -> Sharpness { match (self, other) { (Sharpness::Exact(a), Sharpness::Exact(b)) => Sharpness::Exact(a + b), @@ -97,6 +109,8 @@ impl Sharpness { } } + /// Calculates the difference of two exact or inexact values in the type of [`usize`]. + /// If one of them is a [`Sharpness::Absent`], it returns [`Sharpness::Absent`]. pub fn sub(&self, other: &Sharpness) -> Sharpness { match (self, other) { (Sharpness::Exact(a), Sharpness::Exact(b)) => Sharpness::Exact(a - b), @@ -107,6 +121,8 @@ impl Sharpness { } } + /// Calculates the multiplication of two exact or inexact values in the type of [`usize`]. + /// If one of them is a [`Sharpness::Absent`], it returns [`Sharpness::Absent`]. pub fn multiply(&self, other: &Sharpness) -> Sharpness { match (self, other) { (Sharpness::Exact(a), Sharpness::Exact(b)) => Sharpness::Exact(a * b), @@ -188,6 +204,7 @@ impl Statistics { .collect() } + /// Returns true if all the statistical parameters contain exact information. pub fn all_exact(&self) -> bool { self.num_rows.is_exact().unwrap_or(false) && self.total_byte_size.is_exact().unwrap_or(false) @@ -199,6 +216,8 @@ impl Statistics { }) } + /// If the exactness of a [`Statistics`] instance is lost, this function relaxes + /// the exactness of all information by converting them [`Sharpness::Inexact`]. pub fn make_inexact(self) -> Self { Statistics { num_rows: if let Sharpness::Exact(val) = &self.num_rows { diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index c226f7d15b0b..617a2307c9e1 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -51,7 +51,7 @@ impl PhysicalOptimizerRule for AggregateStatistics { plan: Arc, _config: &ConfigOptions, ) -> Result> { - if let Some(partial_agg_exec) = take_optimizable(&*plan)? { + if let Some(partial_agg_exec) = take_optimizable(&*plan) { let partial_agg_exec = partial_agg_exec .as_any() .downcast_ref::() @@ -108,7 +108,7 @@ impl PhysicalOptimizerRule for AggregateStatistics { /// If this is the case, return a ref to the partial `AggregateExec`, else `None`. /// We would have preferred to return a casted ref to AggregateExec but the recursion requires /// the `ExecutionPlan.children()` method that returns an owned reference. -fn take_optimizable(node: &dyn ExecutionPlan) -> Result>> { +fn take_optimizable(node: &dyn ExecutionPlan) -> Option> { if let Some(final_agg_exec) = node.as_any().downcast_ref::() { if !final_agg_exec.mode().is_first_stage() && final_agg_exec.group_expr().is_empty() @@ -122,7 +122,7 @@ fn take_optimizable(node: &dyn ExecutionPlan) -> Result Result Option<(ScalarValue, &'static str)> { - if !stats.num_rows.is_exact().unwrap_or(false) { - return None; - } if let (Some(num_rows), Some(casted_expr)) = ( - stats.num_rows.get_value(), + stats.num_rows.is_exact().and_then(|exact| { + if exact { + stats.num_rows.get_value() + } else { + None + } + }), agg_expr.as_any().downcast_ref::(), ) { // TODO implementing Eq on PhysicalExpr would help a lot here @@ -166,21 +169,20 @@ fn take_optimizable_table_count( None } -/// If this agg_expr is a count that can be derived from the statistics, return it +/// If this agg_expr is a count that can be exactly derived from the statistics, return it. fn take_optimizable_column_count( agg_expr: &dyn AggregateExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { let col_stats = &stats.column_statistics; - if !stats.num_rows.is_exact().unwrap_or(false) - || !col_stats - .iter() - .all(|cs| cs.null_count.is_exact().unwrap_or(false)) - { - return None; - } if let (Some(num_rows), Some(casted_expr)) = ( - stats.num_rows.get_value(), + stats.num_rows.is_exact().and_then(|exact| { + if exact { + stats.num_rows.get_value() + } else { + None + } + }), agg_expr.as_any().downcast_ref::(), ) { if casted_expr.expressions().len() == 1 { @@ -189,11 +191,14 @@ fn take_optimizable_column_count( .as_any() .downcast_ref::() { - if let Some(val) = &col_stats[col_expr.index()].null_count.get_value() { - return Some(( - ScalarValue::Int64(Some((num_rows - val) as i64)), - casted_expr.name().to_string(), - )); + let current_val = &col_stats[col_expr.index()].null_count; + if let Some(val) = current_val.get_value() { + if current_val.is_exact().unwrap_or(false) { + return Some(( + ScalarValue::Int64(Some((num_rows - val) as i64)), + casted_expr.name().to_string(), + )); + } } } } @@ -201,18 +206,12 @@ fn take_optimizable_column_count( None } -/// If this agg_expr is a min that is defined in the statistics, return it +/// If this agg_expr is a min that is exactly defined in the statistics, return it. fn take_optimizable_min( agg_expr: &dyn AggregateExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { let col_stats = &stats.column_statistics; - if !col_stats - .iter() - .all(|cs| cs.min_value.is_exact().unwrap_or(false)) - { - return None; - } if let Some(casted_expr) = agg_expr.as_any().downcast_ref::() { if casted_expr.expressions().len() == 1 { // TODO optimize with exprs other than Column @@ -220,12 +219,9 @@ fn take_optimizable_min( .as_any() .downcast_ref::() { - if let Some(val) = &col_stats[col_expr.index()].min_value.get_value() { - // Exclude the unbounded case - // As safest estimate, -inf, and + inf is used as bound of the column - // If minimum is -inf, it is not exact. Hence we shouldn't do optimization - // based on this statistic - if !val.is_null() { + let current_val = &col_stats[col_expr.index()].min_value; + if let Some(val) = current_val.get_value() { + if !val.is_null() && current_val.is_exact().unwrap_or(false) { return Some((val.clone(), casted_expr.name().to_string())); } } @@ -235,18 +231,12 @@ fn take_optimizable_min( None } -/// If this agg_expr is a max that is defined in the statistics, return it +/// If this agg_expr is a max that is exactly defined in the statistics, return it. fn take_optimizable_max( agg_expr: &dyn AggregateExpr, stats: &Statistics, ) -> Option<(ScalarValue, String)> { let col_stats = &stats.column_statistics; - if !col_stats - .iter() - .all(|cs| cs.max_value.is_exact().unwrap_or(false)) - { - return None; - } if let Some(casted_expr) = agg_expr.as_any().downcast_ref::() { if casted_expr.expressions().len() == 1 { // TODO optimize with exprs other than Column @@ -254,9 +244,9 @@ fn take_optimizable_max( .as_any() .downcast_ref::() { - if let Some(val) = &col_stats[col_expr.index()].max_value.get_value() { - // Exclude the unbounded case - if !val.is_null() { + let current_val = &col_stats[col_expr.index()].max_value; + if let Some(val) = current_val.get_value() { + if !val.is_null() && current_val.is_exact().unwrap_or(false) { return Some((val.clone(), casted_expr.name().to_string())); } } diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index 24f60387ead7..c571c5f69f04 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -188,8 +188,8 @@ fn shrink_boundaries( } }); let graph_nodes = graph.gather_node_indices(&[expr.clone()]); - // Since the propagation result success, the graph has at least one element, and - // empty check is also done at the outer scope. + // Since the propagation result is success, the graph has at least one element. + // An empty check is also done at the outer scope, do not repeat it here. let (_, root_index) = graph_nodes[0]; let final_result = graph.get_interval(root_index); diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index 90430cb2bbda..5957f253155e 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -21,6 +21,7 @@ use std::any::Any; use std::hash::{Hash, Hasher}; use std::sync::Arc; +use crate::intervals::{Interval, IntervalBound}; use crate::physical_expr::down_cast_any_ref; use crate::sort_properties::SortProperties; use crate::PhysicalExpr; @@ -105,6 +106,43 @@ impl PhysicalExpr for NegativeExpr { self.hash(&mut s); } + /// Given the child interval of a NegativeExpr, it calculates the NegativeExpr's interval. + /// It replaces the upper and lower bounds after multiplying them with -1. + /// Ex: (a, b] => [-b, a) + fn evaluate_bounds(&self, children: &[&Interval]) -> Result { + Ok(Interval::new( + IntervalBound::new( + children[0].upper.value.arithmetic_negate()?, + children[0].upper.open, + ), + IntervalBound::new( + children[0].lower.value.arithmetic_negate()?, + children[0].lower.open, + ), + )) + } + + /// Updates the child interval of a NegativeExpr by intersecting the original + /// interval of child with the possibly shrunk NegativeExpr interval. + fn propagate_constraints( + &self, + interval: &Interval, + children: &[&Interval], + ) -> Result>> { + let child_interval = children[0]; + let negated_interval = Interval::new( + IntervalBound::new( + interval.upper.value.arithmetic_negate()?, + interval.upper.open, + ), + IntervalBound::new( + interval.lower.value.arithmetic_negate()?, + interval.lower.open, + ), + ); + Ok(vec![child_interval.intersect(negated_interval)?]) + } + /// The ordering of a [`NegativeExpr`] is simply the reverse of its child. fn get_ordering(&self, children: &[SortProperties]) -> SortProperties { -children[0] @@ -144,11 +182,12 @@ pub fn negative( #[cfg(test)] mod tests { use super::*; - use crate::expressions::col; + use crate::expressions::{col, Column}; #[allow(unused_imports)] use arrow::array::*; use arrow::datatypes::*; use arrow_schema::DataType::{Float32, Float64, Int16, Int32, Int64, Int8}; + use datafusion_common::ScalarValue; use datafusion_common::{cast::as_primitive_array, Result}; use paste::paste; @@ -187,4 +226,51 @@ mod tests { test_array_negative_op!(Float64, 23456.0f64, 12345.0f64); Ok(()) } + + #[test] + fn test_evaluate_bounds() -> Result<()> { + let negative_expr = NegativeExpr { + arg: Arc::new(Column::new("a", 0)), + }; + let child_interval = Interval::new( + IntervalBound::new(ScalarValue::Int64(Some(-2)), true), + IntervalBound::new(ScalarValue::Int64(Some(1)), false), + ); + let negative_expr_interval = Interval::new( + IntervalBound::new(ScalarValue::Int64(Some(-1)), false), + IntervalBound::new(ScalarValue::Int64(Some(2)), true), + ); + assert_eq!( + negative_expr.evaluate_bounds(&[&child_interval])?, + negative_expr_interval + ); + Ok(()) + } + + #[test] + fn propagate_constraints() -> Result<()> { + let negative_expr = NegativeExpr { + arg: Arc::new(Column::new("a", 0)), + }; + let original_child_interval = Interval::new( + IntervalBound::new(ScalarValue::Int64(Some(-2)), false), + IntervalBound::new(ScalarValue::Int64(Some(3)), false), + ); + let negative_expr_interval = Interval::new( + IntervalBound::new(ScalarValue::Int64(Some(0)), true), + IntervalBound::new(ScalarValue::Int64(Some(4)), false), + ); + let after_propagation = vec![Some(Interval::new( + IntervalBound::new(ScalarValue::Int64(Some(-2)), false), + IntervalBound::new(ScalarValue::Int64(Some(0)), true), + ))]; + assert_eq!( + negative_expr.propagate_constraints( + &negative_expr_interval, + &[&original_child_interval] + )?, + after_propagation + ); + Ok(()) + } } diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index 5b9a5886a990..3f8977c33c12 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -660,6 +660,7 @@ fn next_value(value: ScalarValue) -> ScalarValue { } /// This function computes the cardinality ratio of the given intervals. +/// If it cannot be calculated, it returns 1.0 meaning full selective. pub fn cardinality_ratio( initial_interval: &Interval, final_interval: &Interval, diff --git a/datafusion/physical-expr/src/intervals/utils.rs b/datafusion/physical-expr/src/intervals/utils.rs index 6d8f8e407e5a..41d9c1672b0e 100644 --- a/datafusion/physical-expr/src/intervals/utils.rs +++ b/datafusion/physical-expr/src/intervals/utils.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use super::{Interval, IntervalBound}; use crate::{ - expressions::{BinaryExpr, CastExpr, Column, Literal}, + expressions::{BinaryExpr, CastExpr, Column, Literal, NegativeExpr}, PhysicalExpr, }; @@ -37,7 +37,7 @@ const DT_MS_MASK: i64 = 0xFFFF_FFFF; /// Currently, we do not support all [`PhysicalExpr`]s for interval calculations. /// We do not support every type of [`Operator`]s either. Over time, this check /// will relax as more types of `PhysicalExpr`s and `Operator`s are supported. -/// Currently, [`CastExpr`], [`BinaryExpr`], [`Column`] and [`Literal`] are supported. +/// Currently, [`CastExpr`], [`NegativeExpr`], [`BinaryExpr`], [`Column`] and [`Literal`] are supported. pub fn check_support(expr: &Arc, schema: SchemaRef) -> bool { let expr_any = expr.as_any(); if let Some(binary_expr) = expr_any.downcast_ref::() { @@ -58,6 +58,8 @@ pub fn check_support(expr: &Arc, schema: SchemaRef) -> bool { } } else if let Some(cast) = expr_any.downcast_ref::() { check_support(cast.expr(), schema) + } else if let Some(negative) = expr_any.downcast_ref::() { + check_support(negative.arg(), schema) } else { false } From 47dc4e75c0cfd17902418e15988ed0260faec9be Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 29 Sep 2023 09:50:01 +0300 Subject: [PATCH 20/53] fix after merge --- datafusion/proto/src/generated/pbjson.rs | 2158 +++++++++-------- .../proto/src/physical_plan/from_proto.rs | 22 +- 2 files changed, 1097 insertions(+), 1083 deletions(-) diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 9ae328457b30..1f9ac243cf63 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -47,8 +47,8 @@ impl serde::Serialize for AggregateExecNode { struct_ser.serialize_field("aggrExpr", &self.aggr_expr)?; } if self.mode != 0 { - let v = AggregateMode::from_i32(self.mode) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.mode)))?; + let v = AggregateMode::try_from(self.mode) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.mode)))?; struct_ser.serialize_field("mode", &v)?; } if let Some(v) = self.input.as_ref() { @@ -166,7 +166,7 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { formatter.write_str("struct datafusion.AggregateExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -181,73 +181,73 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { let mut groups__ = None; let mut filter_expr__ = None; let mut order_by_expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::GroupExpr => { if group_expr__.is_some() { return Err(serde::de::Error::duplicate_field("groupExpr")); } - group_expr__ = Some(map.next_value()?); + group_expr__ = Some(map_.next_value()?); } GeneratedField::AggrExpr => { if aggr_expr__.is_some() { return Err(serde::de::Error::duplicate_field("aggrExpr")); } - aggr_expr__ = Some(map.next_value()?); + aggr_expr__ = Some(map_.next_value()?); } GeneratedField::Mode => { if mode__.is_some() { return Err(serde::de::Error::duplicate_field("mode")); } - mode__ = Some(map.next_value::()? as i32); + mode__ = Some(map_.next_value::()? as i32); } GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::GroupExprName => { if group_expr_name__.is_some() { return Err(serde::de::Error::duplicate_field("groupExprName")); } - group_expr_name__ = Some(map.next_value()?); + group_expr_name__ = Some(map_.next_value()?); } GeneratedField::AggrExprName => { if aggr_expr_name__.is_some() { return Err(serde::de::Error::duplicate_field("aggrExprName")); } - aggr_expr_name__ = Some(map.next_value()?); + aggr_expr_name__ = Some(map_.next_value()?); } GeneratedField::InputSchema => { if input_schema__.is_some() { return Err(serde::de::Error::duplicate_field("inputSchema")); } - input_schema__ = map.next_value()?; + input_schema__ = map_.next_value()?; } GeneratedField::NullExpr => { if null_expr__.is_some() { return Err(serde::de::Error::duplicate_field("nullExpr")); } - null_expr__ = Some(map.next_value()?); + null_expr__ = Some(map_.next_value()?); } GeneratedField::Groups => { if groups__.is_some() { return Err(serde::de::Error::duplicate_field("groups")); } - groups__ = Some(map.next_value()?); + groups__ = Some(map_.next_value()?); } GeneratedField::FilterExpr => { if filter_expr__.is_some() { return Err(serde::de::Error::duplicate_field("filterExpr")); } - filter_expr__ = Some(map.next_value()?); + filter_expr__ = Some(map_.next_value()?); } GeneratedField::OrderByExpr => { if order_by_expr__.is_some() { return Err(serde::de::Error::duplicate_field("orderByExpr")); } - order_by_expr__ = Some(map.next_value()?); + order_by_expr__ = Some(map_.next_value()?); } } } @@ -294,8 +294,8 @@ impl serde::Serialize for AggregateExprNode { } let mut struct_ser = serializer.serialize_struct("datafusion.AggregateExprNode", len)?; if self.aggr_function != 0 { - let v = AggregateFunction::from_i32(self.aggr_function) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.aggr_function)))?; + let v = AggregateFunction::try_from(self.aggr_function) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.aggr_function)))?; struct_ser.serialize_field("aggrFunction", &v)?; } if !self.expr.is_empty() { @@ -377,7 +377,7 @@ impl<'de> serde::Deserialize<'de> for AggregateExprNode { formatter.write_str("struct datafusion.AggregateExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -386,37 +386,37 @@ impl<'de> serde::Deserialize<'de> for AggregateExprNode { let mut distinct__ = None; let mut filter__ = None; let mut order_by__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::AggrFunction => { if aggr_function__.is_some() { return Err(serde::de::Error::duplicate_field("aggrFunction")); } - aggr_function__ = Some(map.next_value::()? as i32); + aggr_function__ = Some(map_.next_value::()? as i32); } GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = Some(map.next_value()?); + expr__ = Some(map_.next_value()?); } GeneratedField::Distinct => { if distinct__.is_some() { return Err(serde::de::Error::duplicate_field("distinct")); } - distinct__ = Some(map.next_value()?); + distinct__ = Some(map_.next_value()?); } GeneratedField::Filter => { if filter__.is_some() { return Err(serde::de::Error::duplicate_field("filter")); } - filter__ = map.next_value()?; + filter__ = map_.next_value()?; } GeneratedField::OrderBy => { if order_by__.is_some() { return Err(serde::de::Error::duplicate_field("orderBy")); } - order_by__ = Some(map.next_value()?); + order_by__ = Some(map_.next_value()?); } } } @@ -535,10 +535,9 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(AggregateFunction::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -548,10 +547,9 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(AggregateFunction::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -647,10 +645,9 @@ impl<'de> serde::Deserialize<'de> for AggregateMode { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(AggregateMode::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -660,10 +657,9 @@ impl<'de> serde::Deserialize<'de> for AggregateMode { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(AggregateMode::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -774,32 +770,32 @@ impl<'de> serde::Deserialize<'de> for AggregateNode { formatter.write_str("struct datafusion.AggregateNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut group_expr__ = None; let mut aggr_expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::GroupExpr => { if group_expr__.is_some() { return Err(serde::de::Error::duplicate_field("groupExpr")); } - group_expr__ = Some(map.next_value()?); + group_expr__ = Some(map_.next_value()?); } GeneratedField::AggrExpr => { if aggr_expr__.is_some() { return Err(serde::de::Error::duplicate_field("aggrExpr")); } - aggr_expr__ = Some(map.next_value()?); + aggr_expr__ = Some(map_.next_value()?); } } } @@ -910,7 +906,7 @@ impl<'de> serde::Deserialize<'de> for AggregateUdfExprNode { formatter.write_str("struct datafusion.AggregateUDFExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -918,31 +914,31 @@ impl<'de> serde::Deserialize<'de> for AggregateUdfExprNode { let mut args__ = None; let mut filter__ = None; let mut order_by__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::FunName => { if fun_name__.is_some() { return Err(serde::de::Error::duplicate_field("funName")); } - fun_name__ = Some(map.next_value()?); + fun_name__ = Some(map_.next_value()?); } GeneratedField::Args => { if args__.is_some() { return Err(serde::de::Error::duplicate_field("args")); } - args__ = Some(map.next_value()?); + args__ = Some(map_.next_value()?); } GeneratedField::Filter => { if filter__.is_some() { return Err(serde::de::Error::duplicate_field("filter")); } - filter__ = map.next_value()?; + filter__ = map_.next_value()?; } GeneratedField::OrderBy => { if order_by__.is_some() { return Err(serde::de::Error::duplicate_field("orderBy")); } - order_by__ = Some(map.next_value()?); + order_by__ = Some(map_.next_value()?); } } } @@ -1034,25 +1030,25 @@ impl<'de> serde::Deserialize<'de> for AliasNode { formatter.write_str("struct datafusion.AliasNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; let mut alias__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::Alias => { if alias__.is_some() { return Err(serde::de::Error::duplicate_field("alias")); } - alias__ = Some(map.next_value()?); + alias__ = Some(map_.next_value()?); } } } @@ -1161,7 +1157,7 @@ impl<'de> serde::Deserialize<'de> for AnalyzeExecNode { formatter.write_str("struct datafusion.AnalyzeExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -1169,31 +1165,31 @@ impl<'de> serde::Deserialize<'de> for AnalyzeExecNode { let mut show_statistics__ = None; let mut input__ = None; let mut schema__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Verbose => { if verbose__.is_some() { return Err(serde::de::Error::duplicate_field("verbose")); } - verbose__ = Some(map.next_value()?); + verbose__ = Some(map_.next_value()?); } GeneratedField::ShowStatistics => { if show_statistics__.is_some() { return Err(serde::de::Error::duplicate_field("showStatistics")); } - show_statistics__ = Some(map.next_value()?); + show_statistics__ = Some(map_.next_value()?); } GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = map.next_value()?; + schema__ = map_.next_value()?; } } } @@ -1285,25 +1281,25 @@ impl<'de> serde::Deserialize<'de> for AnalyzeNode { formatter.write_str("struct datafusion.AnalyzeNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut verbose__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Verbose => { if verbose__.is_some() { return Err(serde::de::Error::duplicate_field("verbose")); } - verbose__ = Some(map.next_value()?); + verbose__ = Some(map_.next_value()?); } } } @@ -1385,18 +1381,18 @@ impl<'de> serde::Deserialize<'de> for AnalyzedLogicalPlanType { formatter.write_str("struct datafusion.AnalyzedLogicalPlanType") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut analyzer_name__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::AnalyzerName => { if analyzer_name__.is_some() { return Err(serde::de::Error::duplicate_field("analyzerName")); } - analyzer_name__ = Some(map.next_value()?); + analyzer_name__ = Some(map_.next_value()?); } } } @@ -1483,26 +1479,26 @@ impl serde::Serialize for ArrowType { struct_ser.serialize_field("DATE64", v)?; } arrow_type::ArrowTypeEnum::Duration(v) => { - let v = TimeUnit::from_i32(*v) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; + let v = TimeUnit::try_from(*v) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; struct_ser.serialize_field("DURATION", &v)?; } arrow_type::ArrowTypeEnum::Timestamp(v) => { struct_ser.serialize_field("TIMESTAMP", v)?; } arrow_type::ArrowTypeEnum::Time32(v) => { - let v = TimeUnit::from_i32(*v) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; + let v = TimeUnit::try_from(*v) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; struct_ser.serialize_field("TIME32", &v)?; } arrow_type::ArrowTypeEnum::Time64(v) => { - let v = TimeUnit::from_i32(*v) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; + let v = TimeUnit::try_from(*v) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; struct_ser.serialize_field("TIME64", &v)?; } arrow_type::ArrowTypeEnum::Interval(v) => { - let v = IntervalUnit::from_i32(*v) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; + let v = IntervalUnit::try_from(*v) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; struct_ser.serialize_field("INTERVAL", &v)?; } arrow_type::ArrowTypeEnum::Decimal(v) => { @@ -1685,237 +1681,237 @@ impl<'de> serde::Deserialize<'de> for ArrowType { formatter.write_str("struct datafusion.ArrowType") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut arrow_type_enum__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::None => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("NONE")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::None) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::None) ; } GeneratedField::Bool => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("BOOL")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Bool) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Bool) ; } GeneratedField::Uint8 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("UINT8")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Uint8) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Uint8) ; } GeneratedField::Int8 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("INT8")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Int8) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Int8) ; } GeneratedField::Uint16 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("UINT16")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Uint16) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Uint16) ; } GeneratedField::Int16 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("INT16")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Int16) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Int16) ; } GeneratedField::Uint32 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("UINT32")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Uint32) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Uint32) ; } GeneratedField::Int32 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("INT32")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Int32) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Int32) ; } GeneratedField::Uint64 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("UINT64")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Uint64) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Uint64) ; } GeneratedField::Int64 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("INT64")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Int64) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Int64) ; } GeneratedField::Float16 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("FLOAT16")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Float16) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Float16) ; } GeneratedField::Float32 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("FLOAT32")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Float32) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Float32) ; } GeneratedField::Float64 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("FLOAT64")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Float64) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Float64) ; } GeneratedField::Utf8 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("UTF8")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Utf8) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Utf8) ; } GeneratedField::LargeUtf8 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("LARGEUTF8")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::LargeUtf8) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::LargeUtf8) ; } GeneratedField::Binary => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("BINARY")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Binary) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Binary) ; } GeneratedField::FixedSizeBinary => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("FIXEDSIZEBINARY")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| arrow_type::ArrowTypeEnum::FixedSizeBinary(x.0)); + arrow_type_enum__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| arrow_type::ArrowTypeEnum::FixedSizeBinary(x.0)); } GeneratedField::LargeBinary => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("LARGEBINARY")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::LargeBinary) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::LargeBinary) ; } GeneratedField::Date32 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("DATE32")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Date32) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Date32) ; } GeneratedField::Date64 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("DATE64")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Date64) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Date64) ; } GeneratedField::Duration => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("DURATION")); } - arrow_type_enum__ = map.next_value::<::std::option::Option>()?.map(|x| arrow_type::ArrowTypeEnum::Duration(x as i32)); + arrow_type_enum__ = map_.next_value::<::std::option::Option>()?.map(|x| arrow_type::ArrowTypeEnum::Duration(x as i32)); } GeneratedField::Timestamp => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("TIMESTAMP")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Timestamp) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Timestamp) ; } GeneratedField::Time32 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("TIME32")); } - arrow_type_enum__ = map.next_value::<::std::option::Option>()?.map(|x| arrow_type::ArrowTypeEnum::Time32(x as i32)); + arrow_type_enum__ = map_.next_value::<::std::option::Option>()?.map(|x| arrow_type::ArrowTypeEnum::Time32(x as i32)); } GeneratedField::Time64 => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("TIME64")); } - arrow_type_enum__ = map.next_value::<::std::option::Option>()?.map(|x| arrow_type::ArrowTypeEnum::Time64(x as i32)); + arrow_type_enum__ = map_.next_value::<::std::option::Option>()?.map(|x| arrow_type::ArrowTypeEnum::Time64(x as i32)); } GeneratedField::Interval => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("INTERVAL")); } - arrow_type_enum__ = map.next_value::<::std::option::Option>()?.map(|x| arrow_type::ArrowTypeEnum::Interval(x as i32)); + arrow_type_enum__ = map_.next_value::<::std::option::Option>()?.map(|x| arrow_type::ArrowTypeEnum::Interval(x as i32)); } GeneratedField::Decimal => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("DECIMAL")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Decimal) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Decimal) ; } GeneratedField::List => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("LIST")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::List) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::List) ; } GeneratedField::LargeList => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("LARGELIST")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::LargeList) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::LargeList) ; } GeneratedField::FixedSizeList => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("FIXEDSIZELIST")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::FixedSizeList) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::FixedSizeList) ; } GeneratedField::Struct => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("STRUCT")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Struct) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Struct) ; } GeneratedField::Union => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("UNION")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Union) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Union) ; } GeneratedField::Dictionary => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("DICTIONARY")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Dictionary) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Dictionary) ; } GeneratedField::Map => { if arrow_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("MAP")); } - arrow_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Map) + arrow_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(arrow_type::ArrowTypeEnum::Map) ; } } @@ -1985,12 +1981,12 @@ impl<'de> serde::Deserialize<'de> for AvroFormat { formatter.write_str("struct datafusion.AvroFormat") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { - while map.next_key::()?.is_some() { - let _ = map.next_value::()?; + while map_.next_key::()?.is_some() { + let _ = map_.next_value::()?; } Ok(AvroFormat { }) @@ -2068,18 +2064,18 @@ impl<'de> serde::Deserialize<'de> for AvroScanExecNode { formatter.write_str("struct datafusion.AvroScanExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut base_conf__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::BaseConf => { if base_conf__.is_some() { return Err(serde::de::Error::duplicate_field("baseConf")); } - base_conf__ = map.next_value()?; + base_conf__ = map_.next_value()?; } } } @@ -2159,18 +2155,18 @@ impl<'de> serde::Deserialize<'de> for BareTableReference { formatter.write_str("struct datafusion.BareTableReference") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut table__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Table => { if table__.is_some() { return Err(serde::de::Error::duplicate_field("table")); } - table__ = Some(map.next_value()?); + table__ = Some(map_.next_value()?); } } } @@ -2277,7 +2273,7 @@ impl<'de> serde::Deserialize<'de> for BetweenNode { formatter.write_str("struct datafusion.BetweenNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -2285,31 +2281,31 @@ impl<'de> serde::Deserialize<'de> for BetweenNode { let mut negated__ = None; let mut low__ = None; let mut high__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::Negated => { if negated__.is_some() { return Err(serde::de::Error::duplicate_field("negated")); } - negated__ = Some(map.next_value()?); + negated__ = Some(map_.next_value()?); } GeneratedField::Low => { if low__.is_some() { return Err(serde::de::Error::duplicate_field("low")); } - low__ = map.next_value()?; + low__ = map_.next_value()?; } GeneratedField::High => { if high__.is_some() { return Err(serde::de::Error::duplicate_field("high")); } - high__ = map.next_value()?; + high__ = map_.next_value()?; } } } @@ -2401,25 +2397,25 @@ impl<'de> serde::Deserialize<'de> for BinaryExprNode { formatter.write_str("struct datafusion.BinaryExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut operands__ = None; let mut op__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Operands => { if operands__.is_some() { return Err(serde::de::Error::duplicate_field("operands")); } - operands__ = Some(map.next_value()?); + operands__ = Some(map_.next_value()?); } GeneratedField::Op => { if op__.is_some() { return Err(serde::de::Error::duplicate_field("op")); } - op__ = Some(map.next_value()?); + op__ = Some(map_.next_value()?); } } } @@ -2487,10 +2483,9 @@ impl<'de> serde::Deserialize<'de> for BuiltInWindowFunction { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(BuiltInWindowFunction::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -2500,10 +2495,9 @@ impl<'de> serde::Deserialize<'de> for BuiltInWindowFunction { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(BuiltInWindowFunction::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -2620,32 +2614,32 @@ impl<'de> serde::Deserialize<'de> for CaseNode { formatter.write_str("struct datafusion.CaseNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; let mut when_then_expr__ = None; let mut else_expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::WhenThenExpr => { if when_then_expr__.is_some() { return Err(serde::de::Error::duplicate_field("whenThenExpr")); } - when_then_expr__ = Some(map.next_value()?); + when_then_expr__ = Some(map_.next_value()?); } GeneratedField::ElseExpr => { if else_expr__.is_some() { return Err(serde::de::Error::duplicate_field("elseExpr")); } - else_expr__ = map.next_value()?; + else_expr__ = map_.next_value()?; } } } @@ -2737,25 +2731,25 @@ impl<'de> serde::Deserialize<'de> for CastNode { formatter.write_str("struct datafusion.CastNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; let mut arrow_type__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::ArrowType => { if arrow_type__.is_some() { return Err(serde::de::Error::duplicate_field("arrowType")); } - arrow_type__ = map.next_value()?; + arrow_type__ = map_.next_value()?; } } } @@ -2846,26 +2840,26 @@ impl<'de> serde::Deserialize<'de> for CoalesceBatchesExecNode { formatter.write_str("struct datafusion.CoalesceBatchesExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut target_batch_size__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::TargetBatchSize => { if target_batch_size__.is_some() { return Err(serde::de::Error::duplicate_field("targetBatchSize")); } target_batch_size__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -2947,18 +2941,18 @@ impl<'de> serde::Deserialize<'de> for CoalescePartitionsExecNode { formatter.write_str("struct datafusion.CoalescePartitionsExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } } } @@ -3047,25 +3041,25 @@ impl<'de> serde::Deserialize<'de> for Column { formatter.write_str("struct datafusion.Column") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut name__ = None; let mut relation__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { if name__.is_some() { return Err(serde::de::Error::duplicate_field("name")); } - name__ = Some(map.next_value()?); + name__ = Some(map_.next_value()?); } GeneratedField::Relation => { if relation__.is_some() { return Err(serde::de::Error::duplicate_field("relation")); } - relation__ = map.next_value()?; + relation__ = map_.next_value()?; } } } @@ -3097,8 +3091,8 @@ impl serde::Serialize for ColumnIndex { struct_ser.serialize_field("index", &self.index)?; } if self.side != 0 { - let v = JoinSide::from_i32(self.side) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.side)))?; + let v = JoinSide::try_from(self.side) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.side)))?; struct_ser.serialize_field("side", &v)?; } struct_ser.end() @@ -3157,27 +3151,27 @@ impl<'de> serde::Deserialize<'de> for ColumnIndex { formatter.write_str("struct datafusion.ColumnIndex") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut index__ = None; let mut side__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Index => { if index__.is_some() { return Err(serde::de::Error::duplicate_field("index")); } index__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::Side => { if side__.is_some() { return Err(serde::de::Error::duplicate_field("side")); } - side__ = Some(map.next_value::()? as i32); + side__ = Some(map_.next_value::()? as i32); } } } @@ -3258,18 +3252,18 @@ impl<'de> serde::Deserialize<'de> for ColumnRelation { formatter.write_str("struct datafusion.ColumnRelation") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut relation__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Relation => { if relation__.is_some() { return Err(serde::de::Error::duplicate_field("relation")); } - relation__ = Some(map.next_value()?); + relation__ = Some(map_.next_value()?); } } } @@ -3380,7 +3374,7 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { formatter.write_str("struct datafusion.ColumnStats") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -3388,31 +3382,31 @@ impl<'de> serde::Deserialize<'de> for ColumnStats { let mut max_value__ = None; let mut null_count__ = None; let mut distinct_count__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::MinValue => { if min_value__.is_some() { return Err(serde::de::Error::duplicate_field("minValue")); } - min_value__ = map.next_value()?; + min_value__ = map_.next_value()?; } GeneratedField::MaxValue => { if max_value__.is_some() { return Err(serde::de::Error::duplicate_field("maxValue")); } - max_value__ = map.next_value()?; + max_value__ = map_.next_value()?; } GeneratedField::NullCount => { if null_count__.is_some() { return Err(serde::de::Error::duplicate_field("nullCount")); } - null_count__ = map.next_value()?; + null_count__ = map_.next_value()?; } GeneratedField::DistinctCount => { if distinct_count__.is_some() { return Err(serde::de::Error::duplicate_field("distinctCount")); } - distinct_count__ = map.next_value()?; + distinct_count__ = map_.next_value()?; } } } @@ -3515,32 +3509,32 @@ impl<'de> serde::Deserialize<'de> for CreateCatalogNode { formatter.write_str("struct datafusion.CreateCatalogNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut catalog_name__ = None; let mut if_not_exists__ = None; let mut schema__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::CatalogName => { if catalog_name__.is_some() { return Err(serde::de::Error::duplicate_field("catalogName")); } - catalog_name__ = Some(map.next_value()?); + catalog_name__ = Some(map_.next_value()?); } GeneratedField::IfNotExists => { if if_not_exists__.is_some() { return Err(serde::de::Error::duplicate_field("ifNotExists")); } - if_not_exists__ = Some(map.next_value()?); + if_not_exists__ = Some(map_.next_value()?); } GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = map.next_value()?; + schema__ = map_.next_value()?; } } } @@ -3642,32 +3636,32 @@ impl<'de> serde::Deserialize<'de> for CreateCatalogSchemaNode { formatter.write_str("struct datafusion.CreateCatalogSchemaNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut schema_name__ = None; let mut if_not_exists__ = None; let mut schema__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::SchemaName => { if schema_name__.is_some() { return Err(serde::de::Error::duplicate_field("schemaName")); } - schema_name__ = Some(map.next_value()?); + schema_name__ = Some(map_.next_value()?); } GeneratedField::IfNotExists => { if if_not_exists__.is_some() { return Err(serde::de::Error::duplicate_field("ifNotExists")); } - if_not_exists__ = Some(map.next_value()?); + if_not_exists__ = Some(map_.next_value()?); } GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = map.next_value()?; + schema__ = map_.next_value()?; } } } @@ -3863,7 +3857,7 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { formatter.write_str("struct datafusion.CreateExternalTableNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -3880,86 +3874,86 @@ impl<'de> serde::Deserialize<'de> for CreateExternalTableNode { let mut order_exprs__ = None; let mut unbounded__ = None; let mut options__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { if name__.is_some() { return Err(serde::de::Error::duplicate_field("name")); } - name__ = map.next_value()?; + name__ = map_.next_value()?; } GeneratedField::Location => { if location__.is_some() { return Err(serde::de::Error::duplicate_field("location")); } - location__ = Some(map.next_value()?); + location__ = Some(map_.next_value()?); } GeneratedField::FileType => { if file_type__.is_some() { return Err(serde::de::Error::duplicate_field("fileType")); } - file_type__ = Some(map.next_value()?); + file_type__ = Some(map_.next_value()?); } GeneratedField::HasHeader => { if has_header__.is_some() { return Err(serde::de::Error::duplicate_field("hasHeader")); } - has_header__ = Some(map.next_value()?); + has_header__ = Some(map_.next_value()?); } GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = map.next_value()?; + schema__ = map_.next_value()?; } GeneratedField::TablePartitionCols => { if table_partition_cols__.is_some() { return Err(serde::de::Error::duplicate_field("tablePartitionCols")); } - table_partition_cols__ = Some(map.next_value()?); + table_partition_cols__ = Some(map_.next_value()?); } GeneratedField::IfNotExists => { if if_not_exists__.is_some() { return Err(serde::de::Error::duplicate_field("ifNotExists")); } - if_not_exists__ = Some(map.next_value()?); + if_not_exists__ = Some(map_.next_value()?); } GeneratedField::Delimiter => { if delimiter__.is_some() { return Err(serde::de::Error::duplicate_field("delimiter")); } - delimiter__ = Some(map.next_value()?); + delimiter__ = Some(map_.next_value()?); } GeneratedField::Definition => { if definition__.is_some() { return Err(serde::de::Error::duplicate_field("definition")); } - definition__ = Some(map.next_value()?); + definition__ = Some(map_.next_value()?); } GeneratedField::FileCompressionType => { if file_compression_type__.is_some() { return Err(serde::de::Error::duplicate_field("fileCompressionType")); } - file_compression_type__ = Some(map.next_value()?); + file_compression_type__ = Some(map_.next_value()?); } GeneratedField::OrderExprs => { if order_exprs__.is_some() { return Err(serde::de::Error::duplicate_field("orderExprs")); } - order_exprs__ = Some(map.next_value()?); + order_exprs__ = Some(map_.next_value()?); } GeneratedField::Unbounded => { if unbounded__.is_some() { return Err(serde::de::Error::duplicate_field("unbounded")); } - unbounded__ = Some(map.next_value()?); + unbounded__ = Some(map_.next_value()?); } GeneratedField::Options => { if options__.is_some() { return Err(serde::de::Error::duplicate_field("options")); } options__ = Some( - map.next_value::>()? + map_.next_value::>()? ); } } @@ -4080,7 +4074,7 @@ impl<'de> serde::Deserialize<'de> for CreateViewNode { formatter.write_str("struct datafusion.CreateViewNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -4088,31 +4082,31 @@ impl<'de> serde::Deserialize<'de> for CreateViewNode { let mut input__ = None; let mut or_replace__ = None; let mut definition__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { if name__.is_some() { return Err(serde::de::Error::duplicate_field("name")); } - name__ = map.next_value()?; + name__ = map_.next_value()?; } GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::OrReplace => { if or_replace__.is_some() { return Err(serde::de::Error::duplicate_field("orReplace")); } - or_replace__ = Some(map.next_value()?); + or_replace__ = Some(map_.next_value()?); } GeneratedField::Definition => { if definition__.is_some() { return Err(serde::de::Error::duplicate_field("definition")); } - definition__ = Some(map.next_value()?); + definition__ = Some(map_.next_value()?); } } } @@ -4204,25 +4198,25 @@ impl<'de> serde::Deserialize<'de> for CrossJoinExecNode { formatter.write_str("struct datafusion.CrossJoinExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut left__ = None; let mut right__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Left => { if left__.is_some() { return Err(serde::de::Error::duplicate_field("left")); } - left__ = map.next_value()?; + left__ = map_.next_value()?; } GeneratedField::Right => { if right__.is_some() { return Err(serde::de::Error::duplicate_field("right")); } - right__ = map.next_value()?; + right__ = map_.next_value()?; } } } @@ -4312,25 +4306,25 @@ impl<'de> serde::Deserialize<'de> for CrossJoinNode { formatter.write_str("struct datafusion.CrossJoinNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut left__ = None; let mut right__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Left => { if left__.is_some() { return Err(serde::de::Error::duplicate_field("left")); } - left__ = map.next_value()?; + left__ = map_.next_value()?; } GeneratedField::Right => { if right__.is_some() { return Err(serde::de::Error::duplicate_field("right")); } - right__ = map.next_value()?; + right__ = map_.next_value()?; } } } @@ -4443,7 +4437,7 @@ impl<'de> serde::Deserialize<'de> for CsvFormat { formatter.write_str("struct datafusion.CsvFormat") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -4451,31 +4445,31 @@ impl<'de> serde::Deserialize<'de> for CsvFormat { let mut delimiter__ = None; let mut quote__ = None; let mut optional_escape__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::HasHeader => { if has_header__.is_some() { return Err(serde::de::Error::duplicate_field("hasHeader")); } - has_header__ = Some(map.next_value()?); + has_header__ = Some(map_.next_value()?); } GeneratedField::Delimiter => { if delimiter__.is_some() { return Err(serde::de::Error::duplicate_field("delimiter")); } - delimiter__ = Some(map.next_value()?); + delimiter__ = Some(map_.next_value()?); } GeneratedField::Quote => { if quote__.is_some() { return Err(serde::de::Error::duplicate_field("quote")); } - quote__ = Some(map.next_value()?); + quote__ = Some(map_.next_value()?); } GeneratedField::Escape => { if optional_escape__.is_some() { return Err(serde::de::Error::duplicate_field("escape")); } - optional_escape__ = map.next_value::<::std::option::Option<_>>()?.map(csv_format::OptionalEscape::Escape); + optional_escape__ = map_.next_value::<::std::option::Option<_>>()?.map(csv_format::OptionalEscape::Escape); } } } @@ -4600,7 +4594,7 @@ impl<'de> serde::Deserialize<'de> for CsvScanExecNode { formatter.write_str("struct datafusion.CsvScanExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -4609,37 +4603,37 @@ impl<'de> serde::Deserialize<'de> for CsvScanExecNode { let mut delimiter__ = None; let mut quote__ = None; let mut optional_escape__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::BaseConf => { if base_conf__.is_some() { return Err(serde::de::Error::duplicate_field("baseConf")); } - base_conf__ = map.next_value()?; + base_conf__ = map_.next_value()?; } GeneratedField::HasHeader => { if has_header__.is_some() { return Err(serde::de::Error::duplicate_field("hasHeader")); } - has_header__ = Some(map.next_value()?); + has_header__ = Some(map_.next_value()?); } GeneratedField::Delimiter => { if delimiter__.is_some() { return Err(serde::de::Error::duplicate_field("delimiter")); } - delimiter__ = Some(map.next_value()?); + delimiter__ = Some(map_.next_value()?); } GeneratedField::Quote => { if quote__.is_some() { return Err(serde::de::Error::duplicate_field("quote")); } - quote__ = Some(map.next_value()?); + quote__ = Some(map_.next_value()?); } GeneratedField::Escape => { if optional_escape__.is_some() { return Err(serde::de::Error::duplicate_field("escape")); } - optional_escape__ = map.next_value::<::std::option::Option<_>>()?.map(csv_scan_exec_node::OptionalEscape::Escape); + optional_escape__ = map_.next_value::<::std::option::Option<_>>()?.map(csv_scan_exec_node::OptionalEscape::Escape); } } } @@ -4723,18 +4717,18 @@ impl<'de> serde::Deserialize<'de> for CubeNode { formatter.write_str("struct datafusion.CubeNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = Some(map.next_value()?); + expr__ = Some(map_.next_value()?); } } } @@ -4783,6 +4777,7 @@ impl serde::Serialize for CustomTableScanNode { struct_ser.serialize_field("filters", &self.filters)?; } if !self.custom_table_data.is_empty() { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("customTableData", pbjson::private::base64::encode(&self.custom_table_data).as_str())?; } struct_ser.end() @@ -4852,7 +4847,7 @@ impl<'de> serde::Deserialize<'de> for CustomTableScanNode { formatter.write_str("struct datafusion.CustomTableScanNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -4861,38 +4856,38 @@ impl<'de> serde::Deserialize<'de> for CustomTableScanNode { let mut schema__ = None; let mut filters__ = None; let mut custom_table_data__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::TableName => { if table_name__.is_some() { return Err(serde::de::Error::duplicate_field("tableName")); } - table_name__ = map.next_value()?; + table_name__ = map_.next_value()?; } GeneratedField::Projection => { if projection__.is_some() { return Err(serde::de::Error::duplicate_field("projection")); } - projection__ = map.next_value()?; + projection__ = map_.next_value()?; } GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = map.next_value()?; + schema__ = map_.next_value()?; } GeneratedField::Filters => { if filters__.is_some() { return Err(serde::de::Error::duplicate_field("filters")); } - filters__ = Some(map.next_value()?); + filters__ = Some(map_.next_value()?); } GeneratedField::CustomTableData => { if custom_table_data__.is_some() { return Err(serde::de::Error::duplicate_field("customTableData")); } custom_table_data__ = - Some(map.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) ; } } @@ -4946,10 +4941,9 @@ impl<'de> serde::Deserialize<'de> for DateUnit { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(DateUnit::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -4959,10 +4953,9 @@ impl<'de> serde::Deserialize<'de> for DateUnit { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(DateUnit::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -5059,20 +5052,20 @@ impl<'de> serde::Deserialize<'de> for Decimal { formatter.write_str("struct datafusion.Decimal") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut precision__ = None; let mut scale__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Precision => { if precision__.is_some() { return Err(serde::de::Error::duplicate_field("precision")); } precision__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::Scale => { @@ -5080,7 +5073,7 @@ impl<'de> serde::Deserialize<'de> for Decimal { return Err(serde::de::Error::duplicate_field("scale")); } scale__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -5113,12 +5106,15 @@ impl serde::Serialize for Decimal128 { } let mut struct_ser = serializer.serialize_struct("datafusion.Decimal128", len)?; if !self.value.is_empty() { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("value", pbjson::private::base64::encode(&self.value).as_str())?; } if self.p != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("p", ToString::to_string(&self.p).as_str())?; } if self.s != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("s", ToString::to_string(&self.s).as_str())?; } struct_ser.end() @@ -5180,21 +5176,21 @@ impl<'de> serde::Deserialize<'de> for Decimal128 { formatter.write_str("struct datafusion.Decimal128") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut value__ = None; let mut p__ = None; let mut s__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("value")); } value__ = - Some(map.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) ; } GeneratedField::P => { @@ -5202,7 +5198,7 @@ impl<'de> serde::Deserialize<'de> for Decimal128 { return Err(serde::de::Error::duplicate_field("p")); } p__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::S => { @@ -5210,7 +5206,7 @@ impl<'de> serde::Deserialize<'de> for Decimal128 { return Err(serde::de::Error::duplicate_field("s")); } s__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -5244,12 +5240,15 @@ impl serde::Serialize for Decimal256 { } let mut struct_ser = serializer.serialize_struct("datafusion.Decimal256", len)?; if !self.value.is_empty() { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("value", pbjson::private::base64::encode(&self.value).as_str())?; } if self.p != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("p", ToString::to_string(&self.p).as_str())?; } if self.s != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("s", ToString::to_string(&self.s).as_str())?; } struct_ser.end() @@ -5311,21 +5310,21 @@ impl<'de> serde::Deserialize<'de> for Decimal256 { formatter.write_str("struct datafusion.Decimal256") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut value__ = None; let mut p__ = None; let mut s__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("value")); } value__ = - Some(map.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) ; } GeneratedField::P => { @@ -5333,7 +5332,7 @@ impl<'de> serde::Deserialize<'de> for Decimal256 { return Err(serde::de::Error::duplicate_field("p")); } p__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::S => { @@ -5341,7 +5340,7 @@ impl<'de> serde::Deserialize<'de> for Decimal256 { return Err(serde::de::Error::duplicate_field("s")); } s__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -5433,25 +5432,25 @@ impl<'de> serde::Deserialize<'de> for DfField { formatter.write_str("struct datafusion.DfField") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut field__ = None; let mut qualifier__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Field => { if field__.is_some() { return Err(serde::de::Error::duplicate_field("field")); } - field__ = map.next_value()?; + field__ = map_.next_value()?; } GeneratedField::Qualifier => { if qualifier__.is_some() { return Err(serde::de::Error::duplicate_field("qualifier")); } - qualifier__ = map.next_value()?; + qualifier__ = map_.next_value()?; } } } @@ -5541,26 +5540,26 @@ impl<'de> serde::Deserialize<'de> for DfSchema { formatter.write_str("struct datafusion.DfSchema") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut columns__ = None; let mut metadata__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Columns => { if columns__.is_some() { return Err(serde::de::Error::duplicate_field("columns")); } - columns__ = Some(map.next_value()?); + columns__ = Some(map_.next_value()?); } GeneratedField::Metadata => { if metadata__.is_some() { return Err(serde::de::Error::duplicate_field("metadata")); } metadata__ = Some( - map.next_value::>()? + map_.next_value::>()? ); } } @@ -5651,25 +5650,25 @@ impl<'de> serde::Deserialize<'de> for Dictionary { formatter.write_str("struct datafusion.Dictionary") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut key__ = None; let mut value__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Key => { if key__.is_some() { return Err(serde::de::Error::duplicate_field("key")); } - key__ = map.next_value()?; + key__ = map_.next_value()?; } GeneratedField::Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("value")); } - value__ = map.next_value()?; + value__ = map_.next_value()?; } } } @@ -5750,18 +5749,18 @@ impl<'de> serde::Deserialize<'de> for DistinctNode { formatter.write_str("struct datafusion.DistinctNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } } } @@ -5860,32 +5859,32 @@ impl<'de> serde::Deserialize<'de> for DropViewNode { formatter.write_str("struct datafusion.DropViewNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut name__ = None; let mut if_exists__ = None; let mut schema__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { if name__.is_some() { return Err(serde::de::Error::duplicate_field("name")); } - name__ = map.next_value()?; + name__ = map_.next_value()?; } GeneratedField::IfExists => { if if_exists__.is_some() { return Err(serde::de::Error::duplicate_field("ifExists")); } - if_exists__ = Some(map.next_value()?); + if_exists__ = Some(map_.next_value()?); } GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = map.next_value()?; + schema__ = map_.next_value()?; } } } @@ -5977,25 +5976,25 @@ impl<'de> serde::Deserialize<'de> for EmptyExecNode { formatter.write_str("struct datafusion.EmptyExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut produce_one_row__ = None; let mut schema__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::ProduceOneRow => { if produce_one_row__.is_some() { return Err(serde::de::Error::duplicate_field("produceOneRow")); } - produce_one_row__ = Some(map.next_value()?); + produce_one_row__ = Some(map_.next_value()?); } GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = map.next_value()?; + schema__ = map_.next_value()?; } } } @@ -6065,12 +6064,12 @@ impl<'de> serde::Deserialize<'de> for EmptyMessage { formatter.write_str("struct datafusion.EmptyMessage") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { - while map.next_key::()?.is_some() { - let _ = map.next_value::()?; + while map_.next_key::()?.is_some() { + let _ = map_.next_value::()?; } Ok(EmptyMessage { }) @@ -6148,18 +6147,18 @@ impl<'de> serde::Deserialize<'de> for EmptyRelationNode { formatter.write_str("struct datafusion.EmptyRelationNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut produce_one_row__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::ProduceOneRow => { if produce_one_row__.is_some() { return Err(serde::de::Error::duplicate_field("produceOneRow")); } - produce_one_row__ = Some(map.next_value()?); + produce_one_row__ = Some(map_.next_value()?); } } } @@ -6258,32 +6257,32 @@ impl<'de> serde::Deserialize<'de> for ExplainExecNode { formatter.write_str("struct datafusion.ExplainExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut schema__ = None; let mut stringified_plans__ = None; let mut verbose__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = map.next_value()?; + schema__ = map_.next_value()?; } GeneratedField::StringifiedPlans => { if stringified_plans__.is_some() { return Err(serde::de::Error::duplicate_field("stringifiedPlans")); } - stringified_plans__ = Some(map.next_value()?); + stringified_plans__ = Some(map_.next_value()?); } GeneratedField::Verbose => { if verbose__.is_some() { return Err(serde::de::Error::duplicate_field("verbose")); } - verbose__ = Some(map.next_value()?); + verbose__ = Some(map_.next_value()?); } } } @@ -6374,25 +6373,25 @@ impl<'de> serde::Deserialize<'de> for ExplainNode { formatter.write_str("struct datafusion.ExplainNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut verbose__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Verbose => { if verbose__.is_some() { return Err(serde::de::Error::duplicate_field("verbose")); } - verbose__ = Some(map.next_value()?); + verbose__ = Some(map_.next_value()?); } } } @@ -6510,7 +6509,7 @@ impl<'de> serde::Deserialize<'de> for Field { formatter.write_str("struct datafusion.Field") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -6519,38 +6518,38 @@ impl<'de> serde::Deserialize<'de> for Field { let mut nullable__ = None; let mut children__ = None; let mut metadata__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { if name__.is_some() { return Err(serde::de::Error::duplicate_field("name")); } - name__ = Some(map.next_value()?); + name__ = Some(map_.next_value()?); } GeneratedField::ArrowType => { if arrow_type__.is_some() { return Err(serde::de::Error::duplicate_field("arrowType")); } - arrow_type__ = map.next_value()?; + arrow_type__ = map_.next_value()?; } GeneratedField::Nullable => { if nullable__.is_some() { return Err(serde::de::Error::duplicate_field("nullable")); } - nullable__ = Some(map.next_value()?); + nullable__ = Some(map_.next_value()?); } GeneratedField::Children => { if children__.is_some() { return Err(serde::de::Error::duplicate_field("children")); } - children__ = Some(map.next_value()?); + children__ = Some(map_.next_value()?); } GeneratedField::Metadata => { if metadata__.is_some() { return Err(serde::de::Error::duplicate_field("metadata")); } metadata__ = Some( - map.next_value::>()? + map_.next_value::>()? ); } } @@ -6635,18 +6634,18 @@ impl<'de> serde::Deserialize<'de> for FileGroup { formatter.write_str("struct datafusion.FileGroup") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut files__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Files => { if files__.is_some() { return Err(serde::de::Error::duplicate_field("files")); } - files__ = Some(map.next_value()?); + files__ = Some(map_.next_value()?); } } } @@ -6674,9 +6673,11 @@ impl serde::Serialize for FileRange { } let mut struct_ser = serializer.serialize_struct("datafusion.FileRange", len)?; if self.start != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("start", ToString::to_string(&self.start).as_str())?; } if self.end != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("end", ToString::to_string(&self.end).as_str())?; } struct_ser.end() @@ -6735,20 +6736,20 @@ impl<'de> serde::Deserialize<'de> for FileRange { formatter.write_str("struct datafusion.FileRange") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut start__ = None; let mut end__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Start => { if start__.is_some() { return Err(serde::de::Error::duplicate_field("start")); } start__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::End => { @@ -6756,7 +6757,7 @@ impl<'de> serde::Deserialize<'de> for FileRange { return Err(serde::de::Error::duplicate_field("end")); } end__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -6905,7 +6906,7 @@ impl<'de> serde::Deserialize<'de> for FileScanExecConf { formatter.write_str("struct datafusion.FileScanExecConf") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -6917,26 +6918,26 @@ impl<'de> serde::Deserialize<'de> for FileScanExecConf { let mut table_partition_cols__ = None; let mut object_store_url__ = None; let mut output_ordering__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::FileGroups => { if file_groups__.is_some() { return Err(serde::de::Error::duplicate_field("fileGroups")); } - file_groups__ = Some(map.next_value()?); + file_groups__ = Some(map_.next_value()?); } GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = map.next_value()?; + schema__ = map_.next_value()?; } GeneratedField::Projection => { if projection__.is_some() { return Err(serde::de::Error::duplicate_field("projection")); } projection__ = - Some(map.next_value::>>()? + Some(map_.next_value::>>()? .into_iter().map(|x| x.0).collect()) ; } @@ -6944,31 +6945,31 @@ impl<'de> serde::Deserialize<'de> for FileScanExecConf { if limit__.is_some() { return Err(serde::de::Error::duplicate_field("limit")); } - limit__ = map.next_value()?; + limit__ = map_.next_value()?; } GeneratedField::Statistics => { if statistics__.is_some() { return Err(serde::de::Error::duplicate_field("statistics")); } - statistics__ = map.next_value()?; + statistics__ = map_.next_value()?; } GeneratedField::TablePartitionCols => { if table_partition_cols__.is_some() { return Err(serde::de::Error::duplicate_field("tablePartitionCols")); } - table_partition_cols__ = Some(map.next_value()?); + table_partition_cols__ = Some(map_.next_value()?); } GeneratedField::ObjectStoreUrl => { if object_store_url__.is_some() { return Err(serde::de::Error::duplicate_field("objectStoreUrl")); } - object_store_url__ = Some(map.next_value()?); + object_store_url__ = Some(map_.next_value()?); } GeneratedField::OutputOrdering => { if output_ordering__.is_some() { return Err(serde::de::Error::duplicate_field("outputOrdering")); } - output_ordering__ = Some(map.next_value()?); + output_ordering__ = Some(map_.next_value()?); } } } @@ -7064,25 +7065,25 @@ impl<'de> serde::Deserialize<'de> for FilterExecNode { formatter.write_str("struct datafusion.FilterExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -7163,19 +7164,19 @@ impl<'de> serde::Deserialize<'de> for FixedSizeBinary { formatter.write_str("struct datafusion.FixedSizeBinary") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut length__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Length => { if length__.is_some() { return Err(serde::de::Error::duplicate_field("length")); } length__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -7267,26 +7268,26 @@ impl<'de> serde::Deserialize<'de> for FixedSizeList { formatter.write_str("struct datafusion.FixedSizeList") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut field_type__ = None; let mut list_size__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::FieldType => { if field_type__.is_some() { return Err(serde::de::Error::duplicate_field("fieldType")); } - field_type__ = map.next_value()?; + field_type__ = map_.next_value()?; } GeneratedField::ListSize => { if list_size__.is_some() { return Err(serde::de::Error::duplicate_field("listSize")); } list_size__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -7386,32 +7387,32 @@ impl<'de> serde::Deserialize<'de> for FullTableReference { formatter.write_str("struct datafusion.FullTableReference") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut catalog__ = None; let mut schema__ = None; let mut table__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Catalog => { if catalog__.is_some() { return Err(serde::de::Error::duplicate_field("catalog")); } - catalog__ = Some(map.next_value()?); + catalog__ = Some(map_.next_value()?); } GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = Some(map.next_value()?); + schema__ = Some(map_.next_value()?); } GeneratedField::Table => { if table__.is_some() { return Err(serde::de::Error::duplicate_field("table")); } - table__ = Some(map.next_value()?); + table__ = Some(map_.next_value()?); } } } @@ -7521,39 +7522,39 @@ impl<'de> serde::Deserialize<'de> for GetIndexedField { formatter.write_str("struct datafusion.GetIndexedField") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; let mut field__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::NamedStructField => { if field__.is_some() { return Err(serde::de::Error::duplicate_field("namedStructField")); } - field__ = map.next_value::<::std::option::Option<_>>()?.map(get_indexed_field::Field::NamedStructField) + field__ = map_.next_value::<::std::option::Option<_>>()?.map(get_indexed_field::Field::NamedStructField) ; } GeneratedField::ListIndex => { if field__.is_some() { return Err(serde::de::Error::duplicate_field("listIndex")); } - field__ = map.next_value::<::std::option::Option<_>>()?.map(get_indexed_field::Field::ListIndex) + field__ = map_.next_value::<::std::option::Option<_>>()?.map(get_indexed_field::Field::ListIndex) ; } GeneratedField::ListRange => { if field__.is_some() { return Err(serde::de::Error::duplicate_field("listRange")); } - field__ = map.next_value::<::std::option::Option<_>>()?.map(get_indexed_field::Field::ListRange) + field__ = map_.next_value::<::std::option::Option<_>>()?.map(get_indexed_field::Field::ListRange) ; } } @@ -7592,6 +7593,7 @@ impl serde::Serialize for GlobalLimitExecNode { struct_ser.serialize_field("skip", &self.skip)?; } if self.fetch != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } struct_ser.end() @@ -7653,27 +7655,27 @@ impl<'de> serde::Deserialize<'de> for GlobalLimitExecNode { formatter.write_str("struct datafusion.GlobalLimitExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut skip__ = None; let mut fetch__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Skip => { if skip__.is_some() { return Err(serde::de::Error::duplicate_field("skip")); } skip__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::Fetch => { @@ -7681,7 +7683,7 @@ impl<'de> serde::Deserialize<'de> for GlobalLimitExecNode { return Err(serde::de::Error::duplicate_field("fetch")); } fetch__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -7764,18 +7766,18 @@ impl<'de> serde::Deserialize<'de> for GroupingSetNode { formatter.write_str("struct datafusion.GroupingSetNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = Some(map.next_value()?); + expr__ = Some(map_.next_value()?); } } } @@ -7827,13 +7829,13 @@ impl serde::Serialize for HashJoinExecNode { struct_ser.serialize_field("on", &self.on)?; } if self.join_type != 0 { - let v = JoinType::from_i32(self.join_type) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.join_type)))?; + let v = JoinType::try_from(self.join_type) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.join_type)))?; struct_ser.serialize_field("joinType", &v)?; } if self.partition_mode != 0 { - let v = PartitionMode::from_i32(self.partition_mode) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.partition_mode)))?; + let v = PartitionMode::try_from(self.partition_mode) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.partition_mode)))?; struct_ser.serialize_field("partitionMode", &v)?; } if self.null_equals_null { @@ -7916,7 +7918,7 @@ impl<'de> serde::Deserialize<'de> for HashJoinExecNode { formatter.write_str("struct datafusion.HashJoinExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -7927,49 +7929,49 @@ impl<'de> serde::Deserialize<'de> for HashJoinExecNode { let mut partition_mode__ = None; let mut null_equals_null__ = None; let mut filter__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Left => { if left__.is_some() { return Err(serde::de::Error::duplicate_field("left")); } - left__ = map.next_value()?; + left__ = map_.next_value()?; } GeneratedField::Right => { if right__.is_some() { return Err(serde::de::Error::duplicate_field("right")); } - right__ = map.next_value()?; + right__ = map_.next_value()?; } GeneratedField::On => { if on__.is_some() { return Err(serde::de::Error::duplicate_field("on")); } - on__ = Some(map.next_value()?); + on__ = Some(map_.next_value()?); } GeneratedField::JoinType => { if join_type__.is_some() { return Err(serde::de::Error::duplicate_field("joinType")); } - join_type__ = Some(map.next_value::()? as i32); + join_type__ = Some(map_.next_value::()? as i32); } GeneratedField::PartitionMode => { if partition_mode__.is_some() { return Err(serde::de::Error::duplicate_field("partitionMode")); } - partition_mode__ = Some(map.next_value::()? as i32); + partition_mode__ = Some(map_.next_value::()? as i32); } GeneratedField::NullEqualsNull => { if null_equals_null__.is_some() { return Err(serde::de::Error::duplicate_field("nullEqualsNull")); } - null_equals_null__ = Some(map.next_value()?); + null_equals_null__ = Some(map_.next_value()?); } GeneratedField::Filter => { if filter__.is_some() { return Err(serde::de::Error::duplicate_field("filter")); } - filter__ = map.next_value()?; + filter__ = map_.next_value()?; } } } @@ -8006,6 +8008,7 @@ impl serde::Serialize for HashRepartition { struct_ser.serialize_field("hashExpr", &self.hash_expr)?; } if self.partition_count != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("partitionCount", ToString::to_string(&self.partition_count).as_str())?; } struct_ser.end() @@ -8066,26 +8069,26 @@ impl<'de> serde::Deserialize<'de> for HashRepartition { formatter.write_str("struct datafusion.HashRepartition") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut hash_expr__ = None; let mut partition_count__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::HashExpr => { if hash_expr__.is_some() { return Err(serde::de::Error::duplicate_field("hashExpr")); } - hash_expr__ = Some(map.next_value()?); + hash_expr__ = Some(map_.next_value()?); } GeneratedField::PartitionCount => { if partition_count__.is_some() { return Err(serde::de::Error::duplicate_field("partitionCount")); } partition_count__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -8195,7 +8198,7 @@ impl<'de> serde::Deserialize<'de> for ILikeNode { formatter.write_str("struct datafusion.ILikeNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -8203,31 +8206,31 @@ impl<'de> serde::Deserialize<'de> for ILikeNode { let mut expr__ = None; let mut pattern__ = None; let mut escape_char__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Negated => { if negated__.is_some() { return Err(serde::de::Error::duplicate_field("negated")); } - negated__ = Some(map.next_value()?); + negated__ = Some(map_.next_value()?); } GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::Pattern => { if pattern__.is_some() { return Err(serde::de::Error::duplicate_field("pattern")); } - pattern__ = map.next_value()?; + pattern__ = map_.next_value()?; } GeneratedField::EscapeChar => { if escape_char__.is_some() { return Err(serde::de::Error::duplicate_field("escapeChar")); } - escape_char__ = Some(map.next_value()?); + escape_char__ = Some(map_.next_value()?); } } } @@ -8328,32 +8331,32 @@ impl<'de> serde::Deserialize<'de> for InListNode { formatter.write_str("struct datafusion.InListNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; let mut list__ = None; let mut negated__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::List => { if list__.is_some() { return Err(serde::de::Error::duplicate_field("list")); } - list__ = Some(map.next_value()?); + list__ = Some(map_.next_value()?); } GeneratedField::Negated => { if negated__.is_some() { return Err(serde::de::Error::duplicate_field("negated")); } - negated__ = Some(map.next_value()?); + negated__ = Some(map_.next_value()?); } } } @@ -8392,6 +8395,7 @@ impl serde::Serialize for IntervalMonthDayNanoValue { struct_ser.serialize_field("days", &self.days)?; } if self.nanos != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("nanos", ToString::to_string(&self.nanos).as_str())?; } struct_ser.end() @@ -8453,21 +8457,21 @@ impl<'de> serde::Deserialize<'de> for IntervalMonthDayNanoValue { formatter.write_str("struct datafusion.IntervalMonthDayNanoValue") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut months__ = None; let mut days__ = None; let mut nanos__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Months => { if months__.is_some() { return Err(serde::de::Error::duplicate_field("months")); } months__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::Days => { @@ -8475,7 +8479,7 @@ impl<'de> serde::Deserialize<'de> for IntervalMonthDayNanoValue { return Err(serde::de::Error::duplicate_field("days")); } days__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::Nanos => { @@ -8483,7 +8487,7 @@ impl<'de> serde::Deserialize<'de> for IntervalMonthDayNanoValue { return Err(serde::de::Error::duplicate_field("nanos")); } nanos__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -8537,10 +8541,9 @@ impl<'de> serde::Deserialize<'de> for IntervalUnit { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(IntervalUnit::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -8550,10 +8553,9 @@ impl<'de> serde::Deserialize<'de> for IntervalUnit { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(IntervalUnit::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -8642,18 +8644,18 @@ impl<'de> serde::Deserialize<'de> for IsFalse { formatter.write_str("struct datafusion.IsFalse") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -8733,18 +8735,18 @@ impl<'de> serde::Deserialize<'de> for IsNotFalse { formatter.write_str("struct datafusion.IsNotFalse") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -8824,18 +8826,18 @@ impl<'de> serde::Deserialize<'de> for IsNotNull { formatter.write_str("struct datafusion.IsNotNull") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -8915,18 +8917,18 @@ impl<'de> serde::Deserialize<'de> for IsNotTrue { formatter.write_str("struct datafusion.IsNotTrue") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -9006,18 +9008,18 @@ impl<'de> serde::Deserialize<'de> for IsNotUnknown { formatter.write_str("struct datafusion.IsNotUnknown") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -9097,18 +9099,18 @@ impl<'de> serde::Deserialize<'de> for IsNull { formatter.write_str("struct datafusion.IsNull") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -9188,18 +9190,18 @@ impl<'de> serde::Deserialize<'de> for IsTrue { formatter.write_str("struct datafusion.IsTrue") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -9279,18 +9281,18 @@ impl<'de> serde::Deserialize<'de> for IsUnknown { formatter.write_str("struct datafusion.IsUnknown") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -9339,10 +9341,9 @@ impl<'de> serde::Deserialize<'de> for JoinConstraint { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(JoinConstraint::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -9352,10 +9353,9 @@ impl<'de> serde::Deserialize<'de> for JoinConstraint { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(JoinConstraint::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -9462,32 +9462,32 @@ impl<'de> serde::Deserialize<'de> for JoinFilter { formatter.write_str("struct datafusion.JoinFilter") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expression__ = None; let mut column_indices__ = None; let mut schema__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expression => { if expression__.is_some() { return Err(serde::de::Error::duplicate_field("expression")); } - expression__ = map.next_value()?; + expression__ = map_.next_value()?; } GeneratedField::ColumnIndices => { if column_indices__.is_some() { return Err(serde::de::Error::duplicate_field("columnIndices")); } - column_indices__ = Some(map.next_value()?); + column_indices__ = Some(map_.next_value()?); } GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = map.next_value()?; + schema__ = map_.next_value()?; } } } @@ -9541,13 +9541,13 @@ impl serde::Serialize for JoinNode { struct_ser.serialize_field("right", v)?; } if self.join_type != 0 { - let v = JoinType::from_i32(self.join_type) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.join_type)))?; + let v = JoinType::try_from(self.join_type) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.join_type)))?; struct_ser.serialize_field("joinType", &v)?; } if self.join_constraint != 0 { - let v = JoinConstraint::from_i32(self.join_constraint) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.join_constraint)))?; + let v = JoinConstraint::try_from(self.join_constraint) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.join_constraint)))?; struct_ser.serialize_field("joinConstraint", &v)?; } if !self.left_join_key.is_empty() { @@ -9641,7 +9641,7 @@ impl<'de> serde::Deserialize<'de> for JoinNode { formatter.write_str("struct datafusion.JoinNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -9653,55 +9653,55 @@ impl<'de> serde::Deserialize<'de> for JoinNode { let mut right_join_key__ = None; let mut null_equals_null__ = None; let mut filter__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Left => { if left__.is_some() { return Err(serde::de::Error::duplicate_field("left")); } - left__ = map.next_value()?; + left__ = map_.next_value()?; } GeneratedField::Right => { if right__.is_some() { return Err(serde::de::Error::duplicate_field("right")); } - right__ = map.next_value()?; + right__ = map_.next_value()?; } GeneratedField::JoinType => { if join_type__.is_some() { return Err(serde::de::Error::duplicate_field("joinType")); } - join_type__ = Some(map.next_value::()? as i32); + join_type__ = Some(map_.next_value::()? as i32); } GeneratedField::JoinConstraint => { if join_constraint__.is_some() { return Err(serde::de::Error::duplicate_field("joinConstraint")); } - join_constraint__ = Some(map.next_value::()? as i32); + join_constraint__ = Some(map_.next_value::()? as i32); } GeneratedField::LeftJoinKey => { if left_join_key__.is_some() { return Err(serde::de::Error::duplicate_field("leftJoinKey")); } - left_join_key__ = Some(map.next_value()?); + left_join_key__ = Some(map_.next_value()?); } GeneratedField::RightJoinKey => { if right_join_key__.is_some() { return Err(serde::de::Error::duplicate_field("rightJoinKey")); } - right_join_key__ = Some(map.next_value()?); + right_join_key__ = Some(map_.next_value()?); } GeneratedField::NullEqualsNull => { if null_equals_null__.is_some() { return Err(serde::de::Error::duplicate_field("nullEqualsNull")); } - null_equals_null__ = Some(map.next_value()?); + null_equals_null__ = Some(map_.next_value()?); } GeneratedField::Filter => { if filter__.is_some() { return Err(serde::de::Error::duplicate_field("filter")); } - filter__ = map.next_value()?; + filter__ = map_.next_value()?; } } } @@ -9797,25 +9797,25 @@ impl<'de> serde::Deserialize<'de> for JoinOn { formatter.write_str("struct datafusion.JoinOn") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut left__ = None; let mut right__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Left => { if left__.is_some() { return Err(serde::de::Error::duplicate_field("left")); } - left__ = map.next_value()?; + left__ = map_.next_value()?; } GeneratedField::Right => { if right__.is_some() { return Err(serde::de::Error::duplicate_field("right")); } - right__ = map.next_value()?; + right__ = map_.next_value()?; } } } @@ -9865,10 +9865,9 @@ impl<'de> serde::Deserialize<'de> for JoinSide { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(JoinSide::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -9878,10 +9877,9 @@ impl<'de> serde::Deserialize<'de> for JoinSide { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(JoinSide::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -9950,10 +9948,9 @@ impl<'de> serde::Deserialize<'de> for JoinType { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(JoinType::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -9963,10 +9960,9 @@ impl<'de> serde::Deserialize<'de> for JoinType { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(JoinType::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -10088,7 +10084,7 @@ impl<'de> serde::Deserialize<'de> for LikeNode { formatter.write_str("struct datafusion.LikeNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -10096,31 +10092,31 @@ impl<'de> serde::Deserialize<'de> for LikeNode { let mut expr__ = None; let mut pattern__ = None; let mut escape_char__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Negated => { if negated__.is_some() { return Err(serde::de::Error::duplicate_field("negated")); } - negated__ = Some(map.next_value()?); + negated__ = Some(map_.next_value()?); } GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::Pattern => { if pattern__.is_some() { return Err(serde::de::Error::duplicate_field("pattern")); } - pattern__ = map.next_value()?; + pattern__ = map_.next_value()?; } GeneratedField::EscapeChar => { if escape_char__.is_some() { return Err(serde::de::Error::duplicate_field("escapeChar")); } - escape_char__ = Some(map.next_value()?); + escape_char__ = Some(map_.next_value()?); } } } @@ -10157,9 +10153,11 @@ impl serde::Serialize for LimitNode { struct_ser.serialize_field("input", v)?; } if self.skip != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("skip", ToString::to_string(&self.skip).as_str())?; } if self.fetch != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } struct_ser.end() @@ -10221,27 +10219,27 @@ impl<'de> serde::Deserialize<'de> for LimitNode { formatter.write_str("struct datafusion.LimitNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut skip__ = None; let mut fetch__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Skip => { if skip__.is_some() { return Err(serde::de::Error::duplicate_field("skip")); } skip__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::Fetch => { @@ -10249,7 +10247,7 @@ impl<'de> serde::Deserialize<'de> for LimitNode { return Err(serde::de::Error::duplicate_field("fetch")); } fetch__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -10333,18 +10331,18 @@ impl<'de> serde::Deserialize<'de> for List { formatter.write_str("struct datafusion.List") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut field_type__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::FieldType => { if field_type__.is_some() { return Err(serde::de::Error::duplicate_field("fieldType")); } - field_type__ = map.next_value()?; + field_type__ = map_.next_value()?; } } } @@ -10424,18 +10422,18 @@ impl<'de> serde::Deserialize<'de> for ListIndex { formatter.write_str("struct datafusion.ListIndex") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut key__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Key => { if key__.is_some() { return Err(serde::de::Error::duplicate_field("key")); } - key__ = map.next_value()?; + key__ = map_.next_value()?; } } } @@ -10515,18 +10513,18 @@ impl<'de> serde::Deserialize<'de> for ListIndexExpr { formatter.write_str("struct datafusion.ListIndexExpr") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut key__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Key => { if key__.is_some() { return Err(serde::de::Error::duplicate_field("key")); } - key__ = map.next_value()?; + key__ = map_.next_value()?; } } } @@ -10615,25 +10613,25 @@ impl<'de> serde::Deserialize<'de> for ListRange { formatter.write_str("struct datafusion.ListRange") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut start__ = None; let mut stop__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Start => { if start__.is_some() { return Err(serde::de::Error::duplicate_field("start")); } - start__ = map.next_value()?; + start__ = map_.next_value()?; } GeneratedField::Stop => { if stop__.is_some() { return Err(serde::de::Error::duplicate_field("stop")); } - stop__ = map.next_value()?; + stop__ = map_.next_value()?; } } } @@ -10723,25 +10721,25 @@ impl<'de> serde::Deserialize<'de> for ListRangeExpr { formatter.write_str("struct datafusion.ListRangeExpr") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut start__ = None; let mut stop__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Start => { if start__.is_some() { return Err(serde::de::Error::duplicate_field("start")); } - start__ = map.next_value()?; + start__ = map_.next_value()?; } GeneratedField::Stop => { if stop__.is_some() { return Err(serde::de::Error::duplicate_field("stop")); } - stop__ = map.next_value()?; + stop__ = map_.next_value()?; } } } @@ -10934,7 +10932,7 @@ impl<'de> serde::Deserialize<'de> for ListingTableScanNode { formatter.write_str("struct datafusion.ListingTableScanNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -10949,89 +10947,89 @@ impl<'de> serde::Deserialize<'de> for ListingTableScanNode { let mut target_partitions__ = None; let mut file_sort_order__ = None; let mut file_format_type__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::TableName => { if table_name__.is_some() { return Err(serde::de::Error::duplicate_field("tableName")); } - table_name__ = map.next_value()?; + table_name__ = map_.next_value()?; } GeneratedField::Paths => { if paths__.is_some() { return Err(serde::de::Error::duplicate_field("paths")); } - paths__ = Some(map.next_value()?); + paths__ = Some(map_.next_value()?); } GeneratedField::FileExtension => { if file_extension__.is_some() { return Err(serde::de::Error::duplicate_field("fileExtension")); } - file_extension__ = Some(map.next_value()?); + file_extension__ = Some(map_.next_value()?); } GeneratedField::Projection => { if projection__.is_some() { return Err(serde::de::Error::duplicate_field("projection")); } - projection__ = map.next_value()?; + projection__ = map_.next_value()?; } GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = map.next_value()?; + schema__ = map_.next_value()?; } GeneratedField::Filters => { if filters__.is_some() { return Err(serde::de::Error::duplicate_field("filters")); } - filters__ = Some(map.next_value()?); + filters__ = Some(map_.next_value()?); } GeneratedField::TablePartitionCols => { if table_partition_cols__.is_some() { return Err(serde::de::Error::duplicate_field("tablePartitionCols")); } - table_partition_cols__ = Some(map.next_value()?); + table_partition_cols__ = Some(map_.next_value()?); } GeneratedField::CollectStat => { if collect_stat__.is_some() { return Err(serde::de::Error::duplicate_field("collectStat")); } - collect_stat__ = Some(map.next_value()?); + collect_stat__ = Some(map_.next_value()?); } GeneratedField::TargetPartitions => { if target_partitions__.is_some() { return Err(serde::de::Error::duplicate_field("targetPartitions")); } target_partitions__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::FileSortOrder => { if file_sort_order__.is_some() { return Err(serde::de::Error::duplicate_field("fileSortOrder")); } - file_sort_order__ = Some(map.next_value()?); + file_sort_order__ = Some(map_.next_value()?); } GeneratedField::Csv => { if file_format_type__.is_some() { return Err(serde::de::Error::duplicate_field("csv")); } - file_format_type__ = map.next_value::<::std::option::Option<_>>()?.map(listing_table_scan_node::FileFormatType::Csv) + file_format_type__ = map_.next_value::<::std::option::Option<_>>()?.map(listing_table_scan_node::FileFormatType::Csv) ; } GeneratedField::Parquet => { if file_format_type__.is_some() { return Err(serde::de::Error::duplicate_field("parquet")); } - file_format_type__ = map.next_value::<::std::option::Option<_>>()?.map(listing_table_scan_node::FileFormatType::Parquet) + file_format_type__ = map_.next_value::<::std::option::Option<_>>()?.map(listing_table_scan_node::FileFormatType::Parquet) ; } GeneratedField::Avro => { if file_format_type__.is_some() { return Err(serde::de::Error::duplicate_field("avro")); } - file_format_type__ = map.next_value::<::std::option::Option<_>>()?.map(listing_table_scan_node::FileFormatType::Avro) + file_format_type__ = map_.next_value::<::std::option::Option<_>>()?.map(listing_table_scan_node::FileFormatType::Avro) ; } } @@ -11131,26 +11129,26 @@ impl<'de> serde::Deserialize<'de> for LocalLimitExecNode { formatter.write_str("struct datafusion.LocalLimitExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut fetch__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Fetch => { if fetch__.is_some() { return Err(serde::de::Error::duplicate_field("fetch")); } fetch__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -11232,18 +11230,18 @@ impl<'de> serde::Deserialize<'de> for LogicalExprList { formatter.write_str("struct datafusion.LogicalExprList") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = Some(map.next_value()?); + expr__ = Some(map_.next_value()?); } } } @@ -11546,248 +11544,248 @@ impl<'de> serde::Deserialize<'de> for LogicalExprNode { formatter.write_str("struct datafusion.LogicalExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr_type__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Column => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("column")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Column) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Column) ; } GeneratedField::Alias => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("alias")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Alias) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Alias) ; } GeneratedField::Literal => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("literal")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Literal) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Literal) ; } GeneratedField::BinaryExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("binaryExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::BinaryExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::BinaryExpr) ; } GeneratedField::AggregateExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("aggregateExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::AggregateExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::AggregateExpr) ; } GeneratedField::IsNullExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("isNullExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsNullExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsNullExpr) ; } GeneratedField::IsNotNullExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("isNotNullExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsNotNullExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsNotNullExpr) ; } GeneratedField::NotExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("notExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::NotExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::NotExpr) ; } GeneratedField::Between => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("between")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Between) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Between) ; } GeneratedField::Case => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("case")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Case) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Case) ; } GeneratedField::Cast => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("cast")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::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) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Sort) ; } GeneratedField::Negative => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("negative")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Negative) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Negative) ; } GeneratedField::InList => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("inList")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::InList) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::InList) ; } GeneratedField::Wildcard => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("wildcard")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Wildcard); + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Wildcard); } GeneratedField::ScalarFunction => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("scalarFunction")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::ScalarFunction) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::ScalarFunction) ; } GeneratedField::TryCast => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("tryCast")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::TryCast) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::TryCast) ; } GeneratedField::WindowExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("windowExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::WindowExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::WindowExpr) ; } GeneratedField::AggregateUdfExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("aggregateUdfExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::AggregateUdfExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::AggregateUdfExpr) ; } GeneratedField::ScalarUdfExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("scalarUdfExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::ScalarUdfExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::ScalarUdfExpr) ; } GeneratedField::GetIndexedField => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("getIndexedField")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::GetIndexedField) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::GetIndexedField) ; } GeneratedField::GroupingSet => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("groupingSet")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::GroupingSet) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::GroupingSet) ; } GeneratedField::Cube => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("cube")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Cube) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Cube) ; } GeneratedField::Rollup => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("rollup")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Rollup) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Rollup) ; } GeneratedField::IsTrue => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("isTrue")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsTrue) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsTrue) ; } GeneratedField::IsFalse => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("isFalse")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsFalse) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsFalse) ; } GeneratedField::IsUnknown => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("isUnknown")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsUnknown) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsUnknown) ; } GeneratedField::IsNotTrue => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("isNotTrue")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsNotTrue) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsNotTrue) ; } GeneratedField::IsNotFalse => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("isNotFalse")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsNotFalse) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsNotFalse) ; } GeneratedField::IsNotUnknown => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("isNotUnknown")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsNotUnknown) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::IsNotUnknown) ; } GeneratedField::Like => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("like")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Like) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Like) ; } GeneratedField::Ilike => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("ilike")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Ilike) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Ilike) ; } GeneratedField::SimilarTo => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("similarTo")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::SimilarTo) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::SimilarTo) ; } GeneratedField::Placeholder => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("placeholder")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Placeholder) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_expr_node::ExprType::Placeholder) ; } } @@ -11869,18 +11867,18 @@ impl<'de> serde::Deserialize<'de> for LogicalExprNodeCollection { formatter.write_str("struct datafusion.LogicalExprNodeCollection") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut logical_expr_nodes__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::LogicalExprNodes => { if logical_expr_nodes__.is_some() { return Err(serde::de::Error::duplicate_field("logicalExprNodes")); } - logical_expr_nodes__ = Some(map.next_value()?); + logical_expr_nodes__ = Some(map_.next_value()?); } } } @@ -11908,6 +11906,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)] struct_ser.serialize_field("node", pbjson::private::base64::encode(&self.node).as_str())?; } if !self.inputs.is_empty() { @@ -11969,27 +11968,27 @@ impl<'de> serde::Deserialize<'de> for LogicalExtensionNode { formatter.write_str("struct datafusion.LogicalExtensionNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut node__ = None; let mut inputs__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Node => { if node__.is_some() { return Err(serde::de::Error::duplicate_field("node")); } node__ = - Some(map.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) ; } GeneratedField::Inputs => { if inputs__.is_some() { return Err(serde::de::Error::duplicate_field("inputs")); } - inputs__ = Some(map.next_value()?); + inputs__ = Some(map_.next_value()?); } } } @@ -12235,193 +12234,193 @@ impl<'de> serde::Deserialize<'de> for LogicalPlanNode { formatter.write_str("struct datafusion.LogicalPlanNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut logical_plan_type__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::ListingScan => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("listingScan")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::ListingScan) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::ListingScan) ; } GeneratedField::Projection => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("projection")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Projection) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Projection) ; } GeneratedField::Selection => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("selection")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Selection) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Selection) ; } GeneratedField::Limit => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("limit")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Limit) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Limit) ; } GeneratedField::Aggregate => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("aggregate")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Aggregate) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Aggregate) ; } GeneratedField::Join => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("join")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Join) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Join) ; } GeneratedField::Sort => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("sort")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Sort) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Sort) ; } GeneratedField::Repartition => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("repartition")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Repartition) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Repartition) ; } GeneratedField::EmptyRelation => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("emptyRelation")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::EmptyRelation) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::EmptyRelation) ; } GeneratedField::CreateExternalTable => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("createExternalTable")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::CreateExternalTable) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::CreateExternalTable) ; } GeneratedField::Explain => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("explain")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Explain) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Explain) ; } GeneratedField::Window => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("window")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Window) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Window) ; } GeneratedField::Analyze => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("analyze")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Analyze) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Analyze) ; } GeneratedField::CrossJoin => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("crossJoin")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::CrossJoin) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::CrossJoin) ; } GeneratedField::Values => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("values")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Values) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Values) ; } GeneratedField::Extension => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("extension")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Extension) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Extension) ; } GeneratedField::CreateCatalogSchema => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("createCatalogSchema")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::CreateCatalogSchema) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::CreateCatalogSchema) ; } GeneratedField::Union => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("union")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Union) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Union) ; } GeneratedField::CreateCatalog => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("createCatalog")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::CreateCatalog) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::CreateCatalog) ; } GeneratedField::SubqueryAlias => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("subqueryAlias")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::SubqueryAlias) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::SubqueryAlias) ; } GeneratedField::CreateView => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("createView")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::CreateView) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::CreateView) ; } GeneratedField::Distinct => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("distinct")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Distinct) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Distinct) ; } GeneratedField::ViewScan => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("viewScan")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::ViewScan) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::ViewScan) ; } GeneratedField::CustomScan => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("customScan")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::CustomScan) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::CustomScan) ; } GeneratedField::Prepare => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("prepare")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Prepare) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::Prepare) ; } GeneratedField::DropView => { if logical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("dropView")); } - logical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::DropView) + logical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(logical_plan_node::LogicalPlanType::DropView) ; } } @@ -12513,25 +12512,25 @@ impl<'de> serde::Deserialize<'de> for Map { formatter.write_str("struct datafusion.Map") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut field_type__ = None; let mut keys_sorted__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::FieldType => { if field_type__.is_some() { return Err(serde::de::Error::duplicate_field("fieldType")); } - field_type__ = map.next_value()?; + field_type__ = map_.next_value()?; } GeneratedField::KeysSorted => { if keys_sorted__.is_some() { return Err(serde::de::Error::duplicate_field("keysSorted")); } - keys_sorted__ = Some(map.next_value()?); + keys_sorted__ = Some(map_.next_value()?); } } } @@ -12612,18 +12611,18 @@ impl<'de> serde::Deserialize<'de> for MaybeFilter { formatter.write_str("struct datafusion.MaybeFilter") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -12704,18 +12703,18 @@ impl<'de> serde::Deserialize<'de> for MaybePhysicalSortExprs { formatter.write_str("struct datafusion.MaybePhysicalSortExprs") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut sort_expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::SortExpr => { if sort_expr__.is_some() { return Err(serde::de::Error::duplicate_field("sortExpr")); } - sort_expr__ = Some(map.next_value()?); + sort_expr__ = Some(map_.next_value()?); } } } @@ -12795,18 +12794,18 @@ impl<'de> serde::Deserialize<'de> for NamedStructField { formatter.write_str("struct datafusion.NamedStructField") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut name__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { if name__.is_some() { return Err(serde::de::Error::duplicate_field("name")); } - name__ = map.next_value()?; + name__ = map_.next_value()?; } } } @@ -12886,18 +12885,18 @@ impl<'de> serde::Deserialize<'de> for NamedStructFieldExpr { formatter.write_str("struct datafusion.NamedStructFieldExpr") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut name__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { if name__.is_some() { return Err(serde::de::Error::duplicate_field("name")); } - name__ = map.next_value()?; + name__ = map_.next_value()?; } } } @@ -12977,18 +12976,18 @@ impl<'de> serde::Deserialize<'de> for NegativeNode { formatter.write_str("struct datafusion.NegativeNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -13028,8 +13027,8 @@ impl serde::Serialize for NestedLoopJoinExecNode { struct_ser.serialize_field("right", v)?; } if self.join_type != 0 { - let v = JoinType::from_i32(self.join_type) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.join_type)))?; + let v = JoinType::try_from(self.join_type) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.join_type)))?; struct_ser.serialize_field("joinType", &v)?; } if let Some(v) = self.filter.as_ref() { @@ -13098,7 +13097,7 @@ impl<'de> serde::Deserialize<'de> for NestedLoopJoinExecNode { formatter.write_str("struct datafusion.NestedLoopJoinExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -13106,31 +13105,31 @@ impl<'de> serde::Deserialize<'de> for NestedLoopJoinExecNode { let mut right__ = None; let mut join_type__ = None; let mut filter__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Left => { if left__.is_some() { return Err(serde::de::Error::duplicate_field("left")); } - left__ = map.next_value()?; + left__ = map_.next_value()?; } GeneratedField::Right => { if right__.is_some() { return Err(serde::de::Error::duplicate_field("right")); } - right__ = map.next_value()?; + right__ = map_.next_value()?; } GeneratedField::JoinType => { if join_type__.is_some() { return Err(serde::de::Error::duplicate_field("joinType")); } - join_type__ = Some(map.next_value::()? as i32); + join_type__ = Some(map_.next_value::()? as i32); } GeneratedField::Filter => { if filter__.is_some() { return Err(serde::de::Error::duplicate_field("filter")); } - filter__ = map.next_value()?; + filter__ = map_.next_value()?; } } } @@ -13213,18 +13212,18 @@ impl<'de> serde::Deserialize<'de> for Not { formatter.write_str("struct datafusion.Not") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -13305,18 +13304,18 @@ impl<'de> serde::Deserialize<'de> for OptimizedLogicalPlanType { formatter.write_str("struct datafusion.OptimizedLogicalPlanType") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut optimizer_name__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::OptimizerName => { if optimizer_name__.is_some() { return Err(serde::de::Error::duplicate_field("optimizerName")); } - optimizer_name__ = Some(map.next_value()?); + optimizer_name__ = Some(map_.next_value()?); } } } @@ -13397,18 +13396,18 @@ impl<'de> serde::Deserialize<'de> for OptimizedPhysicalPlanType { formatter.write_str("struct datafusion.OptimizedPhysicalPlanType") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut optimizer_name__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::OptimizerName => { if optimizer_name__.is_some() { return Err(serde::de::Error::duplicate_field("optimizerName")); } - optimizer_name__ = Some(map.next_value()?); + optimizer_name__ = Some(map_.next_value()?); } } } @@ -13504,32 +13503,32 @@ impl<'de> serde::Deserialize<'de> for OwnedTableReference { formatter.write_str("struct datafusion.OwnedTableReference") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut table_reference_enum__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Bare => { if table_reference_enum__.is_some() { return Err(serde::de::Error::duplicate_field("bare")); } - table_reference_enum__ = map.next_value::<::std::option::Option<_>>()?.map(owned_table_reference::TableReferenceEnum::Bare) + table_reference_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(owned_table_reference::TableReferenceEnum::Bare) ; } GeneratedField::Partial => { if table_reference_enum__.is_some() { return Err(serde::de::Error::duplicate_field("partial")); } - table_reference_enum__ = map.next_value::<::std::option::Option<_>>()?.map(owned_table_reference::TableReferenceEnum::Partial) + table_reference_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(owned_table_reference::TableReferenceEnum::Partial) ; } GeneratedField::Full => { if table_reference_enum__.is_some() { return Err(serde::de::Error::duplicate_field("full")); } - table_reference_enum__ = map.next_value::<::std::option::Option<_>>()?.map(owned_table_reference::TableReferenceEnum::Full) + table_reference_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(owned_table_reference::TableReferenceEnum::Full) ; } } @@ -13599,12 +13598,12 @@ impl<'de> serde::Deserialize<'de> for ParquetFormat { formatter.write_str("struct datafusion.ParquetFormat") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { - while map.next_key::()?.is_some() { - let _ = map.next_value::()?; + while map_.next_key::()?.is_some() { + let _ = map_.next_value::()?; } Ok(ParquetFormat { }) @@ -13691,25 +13690,25 @@ impl<'de> serde::Deserialize<'de> for ParquetScanExecNode { formatter.write_str("struct datafusion.ParquetScanExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut base_conf__ = None; let mut predicate__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::BaseConf => { if base_conf__.is_some() { return Err(serde::de::Error::duplicate_field("baseConf")); } - base_conf__ = map.next_value()?; + base_conf__ = map_.next_value()?; } GeneratedField::Predicate => { if predicate__.is_some() { return Err(serde::de::Error::duplicate_field("predicate")); } - predicate__ = map.next_value()?; + predicate__ = map_.next_value()?; } } } @@ -13799,25 +13798,25 @@ impl<'de> serde::Deserialize<'de> for PartialTableReference { formatter.write_str("struct datafusion.PartialTableReference") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut schema__ = None; let mut table__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = Some(map.next_value()?); + schema__ = Some(map_.next_value()?); } GeneratedField::Table => { if table__.is_some() { return Err(serde::de::Error::duplicate_field("table")); } - table__ = Some(map.next_value()?); + table__ = Some(map_.next_value()?); } } } @@ -13898,19 +13897,19 @@ impl<'de> serde::Deserialize<'de> for PartiallySortedPartitionSearchMode { formatter.write_str("struct datafusion.PartiallySortedPartitionSearchMode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut columns__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Columns => { if columns__.is_some() { return Err(serde::de::Error::duplicate_field("columns")); } columns__ = - Some(map.next_value::>>()? + Some(map_.next_value::>>()? .into_iter().map(|x| x.0).collect()) ; } @@ -13963,10 +13962,9 @@ impl<'de> serde::Deserialize<'de> for PartitionMode { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(PartitionMode::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -13976,10 +13974,9 @@ impl<'de> serde::Deserialize<'de> for PartitionMode { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(PartitionMode::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -14022,12 +14019,15 @@ impl serde::Serialize for PartitionStats { } let mut struct_ser = serializer.serialize_struct("datafusion.PartitionStats", len)?; if self.num_rows != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("numRows", ToString::to_string(&self.num_rows).as_str())?; } if self.num_batches != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("numBatches", ToString::to_string(&self.num_batches).as_str())?; } if self.num_bytes != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("numBytes", ToString::to_string(&self.num_bytes).as_str())?; } if !self.column_stats.is_empty() { @@ -14099,7 +14099,7 @@ impl<'de> serde::Deserialize<'de> for PartitionStats { formatter.write_str("struct datafusion.PartitionStats") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -14107,14 +14107,14 @@ impl<'de> serde::Deserialize<'de> for PartitionStats { let mut num_batches__ = None; let mut num_bytes__ = None; let mut column_stats__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::NumRows => { if num_rows__.is_some() { return Err(serde::de::Error::duplicate_field("numRows")); } num_rows__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::NumBatches => { @@ -14122,7 +14122,7 @@ impl<'de> serde::Deserialize<'de> for PartitionStats { return Err(serde::de::Error::duplicate_field("numBatches")); } num_batches__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::NumBytes => { @@ -14130,14 +14130,14 @@ impl<'de> serde::Deserialize<'de> for PartitionStats { return Err(serde::de::Error::duplicate_field("numBytes")); } num_bytes__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::ColumnStats => { if column_stats__.is_some() { return Err(serde::de::Error::duplicate_field("columnStats")); } - column_stats__ = Some(map.next_value()?); + column_stats__ = Some(map_.next_value()?); } } } @@ -14180,9 +14180,11 @@ impl serde::Serialize for PartitionedFile { struct_ser.serialize_field("path", &self.path)?; } if self.size != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("size", ToString::to_string(&self.size).as_str())?; } if self.last_modified_ns != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("lastModifiedNs", ToString::to_string(&self.last_modified_ns).as_str())?; } if !self.partition_values.is_empty() { @@ -14258,7 +14260,7 @@ impl<'de> serde::Deserialize<'de> for PartitionedFile { formatter.write_str("struct datafusion.PartitionedFile") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -14267,20 +14269,20 @@ impl<'de> serde::Deserialize<'de> for PartitionedFile { let mut last_modified_ns__ = None; let mut partition_values__ = None; let mut range__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Path => { if path__.is_some() { return Err(serde::de::Error::duplicate_field("path")); } - path__ = Some(map.next_value()?); + path__ = Some(map_.next_value()?); } GeneratedField::Size => { if size__.is_some() { return Err(serde::de::Error::duplicate_field("size")); } size__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::LastModifiedNs => { @@ -14288,20 +14290,20 @@ impl<'de> serde::Deserialize<'de> for PartitionedFile { return Err(serde::de::Error::duplicate_field("lastModifiedNs")); } last_modified_ns__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::PartitionValues => { if partition_values__.is_some() { return Err(serde::de::Error::duplicate_field("partitionValues")); } - partition_values__ = Some(map.next_value()?); + partition_values__ = Some(map_.next_value()?); } GeneratedField::Range => { if range__.is_some() { return Err(serde::de::Error::duplicate_field("range")); } - range__ = map.next_value()?; + range__ = map_.next_value()?; } } } @@ -14350,8 +14352,8 @@ impl serde::Serialize for PhysicalAggregateExprNode { if let Some(v) = self.aggregate_function.as_ref() { match v { physical_aggregate_expr_node::AggregateFunction::AggrFunction(v) => { - let v = AggregateFunction::from_i32(*v) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; + let v = AggregateFunction::try_from(*v) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; struct_ser.serialize_field("aggrFunction", &v)?; } physical_aggregate_expr_node::AggregateFunction::UserDefinedAggrFunction(v) => { @@ -14427,7 +14429,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalAggregateExprNode { formatter.write_str("struct datafusion.PhysicalAggregateExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -14435,37 +14437,37 @@ impl<'de> serde::Deserialize<'de> for PhysicalAggregateExprNode { let mut ordering_req__ = None; let mut distinct__ = None; let mut aggregate_function__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = Some(map.next_value()?); + expr__ = Some(map_.next_value()?); } GeneratedField::OrderingReq => { if ordering_req__.is_some() { return Err(serde::de::Error::duplicate_field("orderingReq")); } - ordering_req__ = Some(map.next_value()?); + ordering_req__ = Some(map_.next_value()?); } GeneratedField::Distinct => { if distinct__.is_some() { return Err(serde::de::Error::duplicate_field("distinct")); } - distinct__ = Some(map.next_value()?); + distinct__ = Some(map_.next_value()?); } GeneratedField::AggrFunction => { if aggregate_function__.is_some() { return Err(serde::de::Error::duplicate_field("aggrFunction")); } - aggregate_function__ = map.next_value::<::std::option::Option>()?.map(|x| physical_aggregate_expr_node::AggregateFunction::AggrFunction(x as i32)); + aggregate_function__ = map_.next_value::<::std::option::Option>()?.map(|x| physical_aggregate_expr_node::AggregateFunction::AggrFunction(x as i32)); } GeneratedField::UserDefinedAggrFunction => { if aggregate_function__.is_some() { return Err(serde::de::Error::duplicate_field("userDefinedAggrFunction")); } - aggregate_function__ = map.next_value::<::std::option::Option<_>>()?.map(physical_aggregate_expr_node::AggregateFunction::UserDefinedAggrFunction); + aggregate_function__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_aggregate_expr_node::AggregateFunction::UserDefinedAggrFunction); } } } @@ -14557,25 +14559,25 @@ impl<'de> serde::Deserialize<'de> for PhysicalAliasNode { formatter.write_str("struct datafusion.PhysicalAliasNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; let mut alias__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::Alias => { if alias__.is_some() { return Err(serde::de::Error::duplicate_field("alias")); } - alias__ = Some(map.next_value()?); + alias__ = Some(map_.next_value()?); } } } @@ -14674,32 +14676,32 @@ impl<'de> serde::Deserialize<'de> for PhysicalBinaryExprNode { formatter.write_str("struct datafusion.PhysicalBinaryExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut l__ = None; let mut r__ = None; let mut op__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::L => { if l__.is_some() { return Err(serde::de::Error::duplicate_field("l")); } - l__ = map.next_value()?; + l__ = map_.next_value()?; } GeneratedField::R => { if r__.is_some() { return Err(serde::de::Error::duplicate_field("r")); } - r__ = map.next_value()?; + r__ = map_.next_value()?; } GeneratedField::Op => { if op__.is_some() { return Err(serde::de::Error::duplicate_field("op")); } - op__ = Some(map.next_value()?); + op__ = Some(map_.next_value()?); } } } @@ -14801,32 +14803,32 @@ impl<'de> serde::Deserialize<'de> for PhysicalCaseNode { formatter.write_str("struct datafusion.PhysicalCaseNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; let mut when_then_expr__ = None; let mut else_expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::WhenThenExpr => { if when_then_expr__.is_some() { return Err(serde::de::Error::duplicate_field("whenThenExpr")); } - when_then_expr__ = Some(map.next_value()?); + when_then_expr__ = Some(map_.next_value()?); } GeneratedField::ElseExpr => { if else_expr__.is_some() { return Err(serde::de::Error::duplicate_field("elseExpr")); } - else_expr__ = map.next_value()?; + else_expr__ = map_.next_value()?; } } } @@ -14918,25 +14920,25 @@ impl<'de> serde::Deserialize<'de> for PhysicalCastNode { formatter.write_str("struct datafusion.PhysicalCastNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; let mut arrow_type__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::ArrowType => { if arrow_type__.is_some() { return Err(serde::de::Error::duplicate_field("arrowType")); } - arrow_type__ = map.next_value()?; + arrow_type__ = map_.next_value()?; } } } @@ -15026,26 +15028,26 @@ impl<'de> serde::Deserialize<'de> for PhysicalColumn { formatter.write_str("struct datafusion.PhysicalColumn") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut name__ = None; let mut index__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { if name__.is_some() { return Err(serde::de::Error::duplicate_field("name")); } - name__ = Some(map.next_value()?); + name__ = Some(map_.next_value()?); } GeneratedField::Index => { if index__.is_some() { return Err(serde::de::Error::duplicate_field("index")); } index__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -15145,32 +15147,32 @@ impl<'de> serde::Deserialize<'de> for PhysicalDateTimeIntervalExprNode { formatter.write_str("struct datafusion.PhysicalDateTimeIntervalExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut l__ = None; let mut r__ = None; let mut op__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::L => { if l__.is_some() { return Err(serde::de::Error::duplicate_field("l")); } - l__ = map.next_value()?; + l__ = map_.next_value()?; } GeneratedField::R => { if r__.is_some() { return Err(serde::de::Error::duplicate_field("r")); } - r__ = map.next_value()?; + r__ = map_.next_value()?; } GeneratedField::Op => { if op__.is_some() { return Err(serde::de::Error::duplicate_field("op")); } - op__ = Some(map.next_value()?); + op__ = Some(map_.next_value()?); } } } @@ -15371,137 +15373,137 @@ impl<'de> serde::Deserialize<'de> for PhysicalExprNode { formatter.write_str("struct datafusion.PhysicalExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr_type__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Column => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("column")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::Column) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::Column) ; } GeneratedField::Literal => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("literal")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::Literal) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::Literal) ; } GeneratedField::BinaryExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("binaryExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::BinaryExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::BinaryExpr) ; } GeneratedField::AggregateExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("aggregateExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::AggregateExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::AggregateExpr) ; } GeneratedField::IsNullExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("isNullExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::IsNullExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::IsNullExpr) ; } GeneratedField::IsNotNullExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("isNotNullExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::IsNotNullExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::IsNotNullExpr) ; } GeneratedField::NotExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("notExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::NotExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::NotExpr) ; } GeneratedField::Case => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("case")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::Case) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::Case) ; } GeneratedField::Cast => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("cast")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::Cast) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_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(physical_expr_node::ExprType::Sort) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::Sort) ; } GeneratedField::Negative => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("negative")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::Negative) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::Negative) ; } GeneratedField::InList => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("inList")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::InList) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::InList) ; } GeneratedField::ScalarFunction => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("scalarFunction")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::ScalarFunction) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::ScalarFunction) ; } GeneratedField::TryCast => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("tryCast")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::TryCast) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::TryCast) ; } GeneratedField::WindowExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("windowExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::WindowExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::WindowExpr) ; } GeneratedField::ScalarUdf => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("scalarUdf")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::ScalarUdf) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::ScalarUdf) ; } GeneratedField::LikeExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("likeExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::LikeExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::LikeExpr) ; } GeneratedField::GetIndexedFieldExpr => { if expr_type__.is_some() { return Err(serde::de::Error::duplicate_field("getIndexedFieldExpr")); } - expr_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::GetIndexedFieldExpr) + expr_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_expr_node::ExprType::GetIndexedFieldExpr) ; } } @@ -15530,6 +15532,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)] struct_ser.serialize_field("node", pbjson::private::base64::encode(&self.node).as_str())?; } if !self.inputs.is_empty() { @@ -15591,27 +15594,27 @@ impl<'de> serde::Deserialize<'de> for PhysicalExtensionNode { formatter.write_str("struct datafusion.PhysicalExtensionNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut node__ = None; let mut inputs__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Node => { if node__.is_some() { return Err(serde::de::Error::duplicate_field("node")); } node__ = - Some(map.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) ; } GeneratedField::Inputs => { if inputs__.is_some() { return Err(serde::de::Error::duplicate_field("inputs")); } - inputs__ = Some(map.next_value()?); + inputs__ = Some(map_.next_value()?); } } } @@ -15720,39 +15723,39 @@ impl<'de> serde::Deserialize<'de> for PhysicalGetIndexedFieldExprNode { formatter.write_str("struct datafusion.PhysicalGetIndexedFieldExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut arg__ = None; let mut field__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Arg => { if arg__.is_some() { return Err(serde::de::Error::duplicate_field("arg")); } - arg__ = map.next_value()?; + arg__ = map_.next_value()?; } GeneratedField::NamedStructFieldExpr => { if field__.is_some() { return Err(serde::de::Error::duplicate_field("namedStructFieldExpr")); } - field__ = map.next_value::<::std::option::Option<_>>()?.map(physical_get_indexed_field_expr_node::Field::NamedStructFieldExpr) + field__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_get_indexed_field_expr_node::Field::NamedStructFieldExpr) ; } GeneratedField::ListIndexExpr => { if field__.is_some() { return Err(serde::de::Error::duplicate_field("listIndexExpr")); } - field__ = map.next_value::<::std::option::Option<_>>()?.map(physical_get_indexed_field_expr_node::Field::ListIndexExpr) + field__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_get_indexed_field_expr_node::Field::ListIndexExpr) ; } GeneratedField::ListRangeExpr => { if field__.is_some() { return Err(serde::de::Error::duplicate_field("listRangeExpr")); } - field__ = map.next_value::<::std::option::Option<_>>()?.map(physical_get_indexed_field_expr_node::Field::ListRangeExpr) + field__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_get_indexed_field_expr_node::Field::ListRangeExpr) ; } } @@ -15785,6 +15788,7 @@ impl serde::Serialize for PhysicalHashRepartition { struct_ser.serialize_field("hashExpr", &self.hash_expr)?; } if self.partition_count != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("partitionCount", ToString::to_string(&self.partition_count).as_str())?; } struct_ser.end() @@ -15845,26 +15849,26 @@ impl<'de> serde::Deserialize<'de> for PhysicalHashRepartition { formatter.write_str("struct datafusion.PhysicalHashRepartition") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut hash_expr__ = None; let mut partition_count__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::HashExpr => { if hash_expr__.is_some() { return Err(serde::de::Error::duplicate_field("hashExpr")); } - hash_expr__ = Some(map.next_value()?); + hash_expr__ = Some(map_.next_value()?); } GeneratedField::PartitionCount => { if partition_count__.is_some() { return Err(serde::de::Error::duplicate_field("partitionCount")); } partition_count__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -15964,32 +15968,32 @@ impl<'de> serde::Deserialize<'de> for PhysicalInListNode { formatter.write_str("struct datafusion.PhysicalInListNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; let mut list__ = None; let mut negated__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::List => { if list__.is_some() { return Err(serde::de::Error::duplicate_field("list")); } - list__ = Some(map.next_value()?); + list__ = Some(map_.next_value()?); } GeneratedField::Negated => { if negated__.is_some() { return Err(serde::de::Error::duplicate_field("negated")); } - negated__ = Some(map.next_value()?); + negated__ = Some(map_.next_value()?); } } } @@ -16071,18 +16075,18 @@ impl<'de> serde::Deserialize<'de> for PhysicalIsNotNull { formatter.write_str("struct datafusion.PhysicalIsNotNull") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -16162,18 +16166,18 @@ impl<'de> serde::Deserialize<'de> for PhysicalIsNull { formatter.write_str("struct datafusion.PhysicalIsNull") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -16281,7 +16285,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalLikeExprNode { formatter.write_str("struct datafusion.PhysicalLikeExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -16289,31 +16293,31 @@ impl<'de> serde::Deserialize<'de> for PhysicalLikeExprNode { let mut case_insensitive__ = None; let mut expr__ = None; let mut pattern__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Negated => { if negated__.is_some() { return Err(serde::de::Error::duplicate_field("negated")); } - negated__ = Some(map.next_value()?); + negated__ = Some(map_.next_value()?); } GeneratedField::CaseInsensitive => { if case_insensitive__.is_some() { return Err(serde::de::Error::duplicate_field("caseInsensitive")); } - case_insensitive__ = Some(map.next_value()?); + case_insensitive__ = Some(map_.next_value()?); } GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::Pattern => { if pattern__.is_some() { return Err(serde::de::Error::duplicate_field("pattern")); } - pattern__ = map.next_value()?; + pattern__ = map_.next_value()?; } } } @@ -16396,18 +16400,18 @@ impl<'de> serde::Deserialize<'de> for PhysicalNegativeNode { formatter.write_str("struct datafusion.PhysicalNegativeNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -16487,18 +16491,18 @@ impl<'de> serde::Deserialize<'de> for PhysicalNot { formatter.write_str("struct datafusion.PhysicalNot") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -16718,165 +16722,165 @@ impl<'de> serde::Deserialize<'de> for PhysicalPlanNode { formatter.write_str("struct datafusion.PhysicalPlanNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut physical_plan_type__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::ParquetScan => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("parquetScan")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::ParquetScan) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::ParquetScan) ; } GeneratedField::CsvScan => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("csvScan")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::CsvScan) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::CsvScan) ; } GeneratedField::Empty => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("empty")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Empty) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Empty) ; } GeneratedField::Projection => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("projection")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Projection) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Projection) ; } GeneratedField::GlobalLimit => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("globalLimit")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::GlobalLimit) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::GlobalLimit) ; } GeneratedField::LocalLimit => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("localLimit")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::LocalLimit) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::LocalLimit) ; } GeneratedField::Aggregate => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("aggregate")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Aggregate) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Aggregate) ; } GeneratedField::HashJoin => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("hashJoin")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::HashJoin) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::HashJoin) ; } GeneratedField::Sort => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("sort")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Sort) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Sort) ; } GeneratedField::CoalesceBatches => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("coalesceBatches")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::CoalesceBatches) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::CoalesceBatches) ; } GeneratedField::Filter => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("filter")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Filter) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Filter) ; } GeneratedField::Merge => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("merge")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Merge) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Merge) ; } GeneratedField::Repartition => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("repartition")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Repartition) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Repartition) ; } GeneratedField::Window => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("window")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Window) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Window) ; } GeneratedField::CrossJoin => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("crossJoin")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::CrossJoin) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::CrossJoin) ; } GeneratedField::AvroScan => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("avroScan")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::AvroScan) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::AvroScan) ; } GeneratedField::Extension => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("extension")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Extension) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Extension) ; } GeneratedField::Union => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("union")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Union) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Union) ; } GeneratedField::Explain => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("explain")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Explain) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Explain) ; } GeneratedField::SortPreservingMerge => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("sortPreservingMerge")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::SortPreservingMerge) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::SortPreservingMerge) ; } GeneratedField::NestedLoopJoin => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("nestedLoopJoin")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::NestedLoopJoin) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::NestedLoopJoin) ; } GeneratedField::Analyze => { if physical_plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("analyze")); } - physical_plan_type__ = map.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Analyze) + physical_plan_type__ = map_.next_value::<::std::option::Option<_>>()?.map(physical_plan_node::PhysicalPlanType::Analyze) ; } } @@ -16914,8 +16918,8 @@ impl serde::Serialize for PhysicalScalarFunctionNode { struct_ser.serialize_field("name", &self.name)?; } if self.fun != 0 { - let v = ScalarFunction::from_i32(self.fun) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.fun)))?; + let v = ScalarFunction::try_from(self.fun) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.fun)))?; struct_ser.serialize_field("fun", &v)?; } if !self.args.is_empty() { @@ -16987,7 +16991,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalScalarFunctionNode { formatter.write_str("struct datafusion.PhysicalScalarFunctionNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -16995,31 +16999,31 @@ impl<'de> serde::Deserialize<'de> for PhysicalScalarFunctionNode { let mut fun__ = None; let mut args__ = None; let mut return_type__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { if name__.is_some() { return Err(serde::de::Error::duplicate_field("name")); } - name__ = Some(map.next_value()?); + name__ = Some(map_.next_value()?); } GeneratedField::Fun => { if fun__.is_some() { return Err(serde::de::Error::duplicate_field("fun")); } - fun__ = Some(map.next_value::()? as i32); + fun__ = Some(map_.next_value::()? as i32); } GeneratedField::Args => { if args__.is_some() { return Err(serde::de::Error::duplicate_field("args")); } - args__ = Some(map.next_value()?); + args__ = Some(map_.next_value()?); } GeneratedField::ReturnType => { if return_type__.is_some() { return Err(serde::de::Error::duplicate_field("returnType")); } - return_type__ = map.next_value()?; + return_type__ = map_.next_value()?; } } } @@ -17121,32 +17125,32 @@ impl<'de> serde::Deserialize<'de> for PhysicalScalarUdfNode { formatter.write_str("struct datafusion.PhysicalScalarUdfNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut name__ = None; let mut args__ = None; let mut return_type__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { if name__.is_some() { return Err(serde::de::Error::duplicate_field("name")); } - name__ = Some(map.next_value()?); + name__ = Some(map_.next_value()?); } GeneratedField::Args => { if args__.is_some() { return Err(serde::de::Error::duplicate_field("args")); } - args__ = Some(map.next_value()?); + args__ = Some(map_.next_value()?); } GeneratedField::ReturnType => { if return_type__.is_some() { return Err(serde::de::Error::duplicate_field("returnType")); } - return_type__ = map.next_value()?; + return_type__ = map_.next_value()?; } } } @@ -17247,32 +17251,32 @@ impl<'de> serde::Deserialize<'de> for PhysicalSortExprNode { formatter.write_str("struct datafusion.PhysicalSortExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; let mut asc__ = None; let mut nulls_first__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::Asc => { if asc__.is_some() { return Err(serde::de::Error::duplicate_field("asc")); } - asc__ = Some(map.next_value()?); + asc__ = Some(map_.next_value()?); } GeneratedField::NullsFirst => { if nulls_first__.is_some() { return Err(serde::de::Error::duplicate_field("nullsFirst")); } - nulls_first__ = Some(map.next_value()?); + nulls_first__ = Some(map_.next_value()?); } } } @@ -17355,18 +17359,18 @@ impl<'de> serde::Deserialize<'de> for PhysicalSortExprNodeCollection { formatter.write_str("struct datafusion.PhysicalSortExprNodeCollection") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut physical_sort_expr_nodes__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::PhysicalSortExprNodes => { if physical_sort_expr_nodes__.is_some() { return Err(serde::de::Error::duplicate_field("physicalSortExprNodes")); } - physical_sort_expr_nodes__ = Some(map.next_value()?); + physical_sort_expr_nodes__ = Some(map_.next_value()?); } } } @@ -17456,25 +17460,25 @@ impl<'de> serde::Deserialize<'de> for PhysicalTryCastNode { formatter.write_str("struct datafusion.PhysicalTryCastNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; let mut arrow_type__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::ArrowType => { if arrow_type__.is_some() { return Err(serde::de::Error::duplicate_field("arrowType")); } - arrow_type__ = map.next_value()?; + arrow_type__ = map_.next_value()?; } } } @@ -17566,25 +17570,25 @@ impl<'de> serde::Deserialize<'de> for PhysicalWhenThen { formatter.write_str("struct datafusion.PhysicalWhenThen") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut when_expr__ = None; let mut then_expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::WhenExpr => { if when_expr__.is_some() { return Err(serde::de::Error::duplicate_field("whenExpr")); } - when_expr__ = map.next_value()?; + when_expr__ = map_.next_value()?; } GeneratedField::ThenExpr => { if then_expr__.is_some() { return Err(serde::de::Error::duplicate_field("thenExpr")); } - then_expr__ = map.next_value()?; + then_expr__ = map_.next_value()?; } } } @@ -17642,13 +17646,13 @@ impl serde::Serialize for PhysicalWindowExprNode { if let Some(v) = self.window_function.as_ref() { match v { physical_window_expr_node::WindowFunction::AggrFunction(v) => { - let v = AggregateFunction::from_i32(*v) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; + let v = AggregateFunction::try_from(*v) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; struct_ser.serialize_field("aggrFunction", &v)?; } physical_window_expr_node::WindowFunction::BuiltInFunction(v) => { - let v = BuiltInWindowFunction::from_i32(*v) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; + let v = BuiltInWindowFunction::try_from(*v) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; struct_ser.serialize_field("builtInFunction", &v)?; } } @@ -17729,7 +17733,7 @@ impl<'de> serde::Deserialize<'de> for PhysicalWindowExprNode { formatter.write_str("struct datafusion.PhysicalWindowExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -17739,49 +17743,49 @@ impl<'de> serde::Deserialize<'de> for PhysicalWindowExprNode { let mut window_frame__ = None; let mut name__ = None; let mut window_function__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Args => { if args__.is_some() { return Err(serde::de::Error::duplicate_field("args")); } - args__ = Some(map.next_value()?); + args__ = Some(map_.next_value()?); } GeneratedField::PartitionBy => { if partition_by__.is_some() { return Err(serde::de::Error::duplicate_field("partitionBy")); } - partition_by__ = Some(map.next_value()?); + partition_by__ = Some(map_.next_value()?); } GeneratedField::OrderBy => { if order_by__.is_some() { return Err(serde::de::Error::duplicate_field("orderBy")); } - order_by__ = Some(map.next_value()?); + order_by__ = Some(map_.next_value()?); } GeneratedField::WindowFrame => { if window_frame__.is_some() { return Err(serde::de::Error::duplicate_field("windowFrame")); } - window_frame__ = map.next_value()?; + window_frame__ = map_.next_value()?; } GeneratedField::Name => { if name__.is_some() { return Err(serde::de::Error::duplicate_field("name")); } - name__ = Some(map.next_value()?); + name__ = Some(map_.next_value()?); } GeneratedField::AggrFunction => { if window_function__.is_some() { return Err(serde::de::Error::duplicate_field("aggrFunction")); } - window_function__ = map.next_value::<::std::option::Option>()?.map(|x| physical_window_expr_node::WindowFunction::AggrFunction(x as i32)); + window_function__ = map_.next_value::<::std::option::Option>()?.map(|x| physical_window_expr_node::WindowFunction::AggrFunction(x as i32)); } GeneratedField::BuiltInFunction => { if window_function__.is_some() { return Err(serde::de::Error::duplicate_field("builtInFunction")); } - window_function__ = map.next_value::<::std::option::Option>()?.map(|x| physical_window_expr_node::WindowFunction::BuiltInFunction(x as i32)); + window_function__ = map_.next_value::<::std::option::Option>()?.map(|x| physical_window_expr_node::WindowFunction::BuiltInFunction(x as i32)); } } } @@ -17876,25 +17880,25 @@ impl<'de> serde::Deserialize<'de> for PlaceholderNode { formatter.write_str("struct datafusion.PlaceholderNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut id__ = None; let mut data_type__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Id => { if id__.is_some() { return Err(serde::de::Error::duplicate_field("id")); } - id__ = Some(map.next_value()?); + id__ = Some(map_.next_value()?); } GeneratedField::DataType => { if data_type__.is_some() { return Err(serde::de::Error::duplicate_field("dataType")); } - data_type__ = map.next_value()?; + data_type__ = map_.next_value()?; } } } @@ -18021,67 +18025,67 @@ impl<'de> serde::Deserialize<'de> for PlanType { formatter.write_str("struct datafusion.PlanType") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut plan_type_enum__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::InitialLogicalPlan => { if plan_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("InitialLogicalPlan")); } - plan_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::InitialLogicalPlan) + plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::InitialLogicalPlan) ; } GeneratedField::AnalyzedLogicalPlan => { if plan_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("AnalyzedLogicalPlan")); } - plan_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::AnalyzedLogicalPlan) + plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::AnalyzedLogicalPlan) ; } GeneratedField::FinalAnalyzedLogicalPlan => { if plan_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("FinalAnalyzedLogicalPlan")); } - plan_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::FinalAnalyzedLogicalPlan) + plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::FinalAnalyzedLogicalPlan) ; } GeneratedField::OptimizedLogicalPlan => { if plan_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("OptimizedLogicalPlan")); } - plan_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::OptimizedLogicalPlan) + plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::OptimizedLogicalPlan) ; } GeneratedField::FinalLogicalPlan => { if plan_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("FinalLogicalPlan")); } - plan_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::FinalLogicalPlan) + plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::FinalLogicalPlan) ; } GeneratedField::InitialPhysicalPlan => { if plan_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("InitialPhysicalPlan")); } - plan_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::InitialPhysicalPlan) + plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::InitialPhysicalPlan) ; } GeneratedField::OptimizedPhysicalPlan => { if plan_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("OptimizedPhysicalPlan")); } - plan_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::OptimizedPhysicalPlan) + plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::OptimizedPhysicalPlan) ; } GeneratedField::FinalPhysicalPlan => { if plan_type_enum__.is_some() { return Err(serde::de::Error::duplicate_field("FinalPhysicalPlan")); } - plan_type_enum__ = map.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::FinalPhysicalPlan) + plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::FinalPhysicalPlan) ; } } @@ -18181,32 +18185,32 @@ impl<'de> serde::Deserialize<'de> for PrepareNode { formatter.write_str("struct datafusion.PrepareNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut name__ = None; let mut data_types__ = None; let mut input__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Name => { if name__.is_some() { return Err(serde::de::Error::duplicate_field("name")); } - name__ = Some(map.next_value()?); + name__ = Some(map_.next_value()?); } GeneratedField::DataTypes => { if data_types__.is_some() { return Err(serde::de::Error::duplicate_field("dataTypes")); } - data_types__ = Some(map.next_value()?); + data_types__ = Some(map_.next_value()?); } GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } } } @@ -18288,18 +18292,18 @@ impl<'de> serde::Deserialize<'de> for ProjectionColumns { formatter.write_str("struct datafusion.ProjectionColumns") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut columns__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Columns => { if columns__.is_some() { return Err(serde::de::Error::duplicate_field("columns")); } - columns__ = Some(map.next_value()?); + columns__ = Some(map_.next_value()?); } } } @@ -18398,32 +18402,32 @@ impl<'de> serde::Deserialize<'de> for ProjectionExecNode { formatter.write_str("struct datafusion.ProjectionExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut expr__ = None; let mut expr_name__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = Some(map.next_value()?); + expr__ = Some(map_.next_value()?); } GeneratedField::ExprName => { if expr_name__.is_some() { return Err(serde::de::Error::duplicate_field("exprName")); } - expr_name__ = Some(map.next_value()?); + expr_name__ = Some(map_.next_value()?); } } } @@ -18527,32 +18531,32 @@ impl<'de> serde::Deserialize<'de> for ProjectionNode { formatter.write_str("struct datafusion.ProjectionNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut expr__ = None; let mut optional_alias__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = Some(map.next_value()?); + expr__ = Some(map_.next_value()?); } GeneratedField::Alias => { if optional_alias__.is_some() { return Err(serde::de::Error::duplicate_field("alias")); } - optional_alias__ = map.next_value::<::std::option::Option<_>>()?.map(projection_node::OptionalAlias::Alias); + optional_alias__ = map_.next_value::<::std::option::Option<_>>()?.map(projection_node::OptionalAlias::Alias); } } } @@ -18587,12 +18591,14 @@ impl serde::Serialize for RepartitionExecNode { if let Some(v) = self.partition_method.as_ref() { match v { repartition_exec_node::PartitionMethod::RoundRobin(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("roundRobin", ToString::to_string(&v).as_str())?; } repartition_exec_node::PartitionMethod::Hash(v) => { struct_ser.serialize_field("hash", v)?; } repartition_exec_node::PartitionMethod::Unknown(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("unknown", ToString::to_string(&v).as_str())?; } } @@ -18660,38 +18666,38 @@ impl<'de> serde::Deserialize<'de> for RepartitionExecNode { formatter.write_str("struct datafusion.RepartitionExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut partition_method__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::RoundRobin => { if partition_method__.is_some() { return Err(serde::de::Error::duplicate_field("roundRobin")); } - partition_method__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| repartition_exec_node::PartitionMethod::RoundRobin(x.0)); + partition_method__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| repartition_exec_node::PartitionMethod::RoundRobin(x.0)); } GeneratedField::Hash => { if partition_method__.is_some() { return Err(serde::de::Error::duplicate_field("hash")); } - partition_method__ = map.next_value::<::std::option::Option<_>>()?.map(repartition_exec_node::PartitionMethod::Hash) + partition_method__ = map_.next_value::<::std::option::Option<_>>()?.map(repartition_exec_node::PartitionMethod::Hash) ; } GeneratedField::Unknown => { if partition_method__.is_some() { return Err(serde::de::Error::duplicate_field("unknown")); } - partition_method__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| repartition_exec_node::PartitionMethod::Unknown(x.0)); + partition_method__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| repartition_exec_node::PartitionMethod::Unknown(x.0)); } } } @@ -18725,6 +18731,7 @@ impl serde::Serialize for RepartitionNode { if let Some(v) = self.partition_method.as_ref() { match v { repartition_node::PartitionMethod::RoundRobin(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("roundRobin", ToString::to_string(&v).as_str())?; } repartition_node::PartitionMethod::Hash(v) => { @@ -18792,31 +18799,31 @@ impl<'de> serde::Deserialize<'de> for RepartitionNode { formatter.write_str("struct datafusion.RepartitionNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut partition_method__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::RoundRobin => { if partition_method__.is_some() { return Err(serde::de::Error::duplicate_field("roundRobin")); } - partition_method__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| repartition_node::PartitionMethod::RoundRobin(x.0)); + partition_method__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| repartition_node::PartitionMethod::RoundRobin(x.0)); } GeneratedField::Hash => { if partition_method__.is_some() { return Err(serde::de::Error::duplicate_field("hash")); } - partition_method__ = map.next_value::<::std::option::Option<_>>()?.map(repartition_node::PartitionMethod::Hash) + partition_method__ = map_.next_value::<::std::option::Option<_>>()?.map(repartition_node::PartitionMethod::Hash) ; } } @@ -18898,18 +18905,18 @@ impl<'de> serde::Deserialize<'de> for RollupNode { formatter.write_str("struct datafusion.RollupNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = Some(map.next_value()?); + expr__ = Some(map_.next_value()?); } } } @@ -18999,25 +19006,25 @@ impl<'de> serde::Deserialize<'de> for ScalarDictionaryValue { formatter.write_str("struct datafusion.ScalarDictionaryValue") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut index_type__ = None; let mut value__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::IndexType => { if index_type__.is_some() { return Err(serde::de::Error::duplicate_field("indexType")); } - index_type__ = map.next_value()?; + index_type__ = map_.next_value()?; } GeneratedField::Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("value")); } - value__ = map.next_value()?; + value__ = map_.next_value()?; } } } @@ -19046,6 +19053,7 @@ impl serde::Serialize for ScalarFixedSizeBinary { } let mut struct_ser = serializer.serialize_struct("datafusion.ScalarFixedSizeBinary", len)?; if !self.values.is_empty() { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("values", pbjson::private::base64::encode(&self.values).as_str())?; } if self.length != 0 { @@ -19107,20 +19115,20 @@ impl<'de> serde::Deserialize<'de> for ScalarFixedSizeBinary { formatter.write_str("struct datafusion.ScalarFixedSizeBinary") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut values__ = None; let mut length__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Values => { if values__.is_some() { return Err(serde::de::Error::duplicate_field("values")); } values__ = - Some(map.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::BytesDeserialize<_>>()?.0) ; } GeneratedField::Length => { @@ -19128,7 +19136,7 @@ impl<'de> serde::Deserialize<'de> for ScalarFixedSizeBinary { return Err(serde::de::Error::duplicate_field("length")); } length__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -19411,10 +19419,9 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(ScalarFunction::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -19424,10 +19431,9 @@ impl<'de> serde::Deserialize<'de> for ScalarFunction { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(ScalarFunction::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -19579,8 +19585,8 @@ impl serde::Serialize for ScalarFunctionNode { } let mut struct_ser = serializer.serialize_struct("datafusion.ScalarFunctionNode", len)?; if self.fun != 0 { - let v = ScalarFunction::from_i32(self.fun) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.fun)))?; + let v = ScalarFunction::try_from(self.fun) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.fun)))?; struct_ser.serialize_field("fun", &v)?; } if !self.args.is_empty() { @@ -19642,25 +19648,25 @@ impl<'de> serde::Deserialize<'de> for ScalarFunctionNode { formatter.write_str("struct datafusion.ScalarFunctionNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut fun__ = None; let mut args__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Fun => { if fun__.is_some() { return Err(serde::de::Error::duplicate_field("fun")); } - fun__ = Some(map.next_value::()? as i32); + fun__ = Some(map_.next_value::()? as i32); } GeneratedField::Args => { if args__.is_some() { return Err(serde::de::Error::duplicate_field("args")); } - args__ = Some(map.next_value()?); + args__ = Some(map_.next_value()?); } } } @@ -19760,32 +19766,32 @@ impl<'de> serde::Deserialize<'de> for ScalarListValue { formatter.write_str("struct datafusion.ScalarListValue") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut is_null__ = None; let mut field__ = None; let mut values__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::IsNull => { if is_null__.is_some() { return Err(serde::de::Error::duplicate_field("isNull")); } - is_null__ = Some(map.next_value()?); + is_null__ = Some(map_.next_value()?); } GeneratedField::Field => { if field__.is_some() { return Err(serde::de::Error::duplicate_field("field")); } - field__ = map.next_value()?; + field__ = map_.next_value()?; } GeneratedField::Values => { if values__.is_some() { return Err(serde::de::Error::duplicate_field("values")); } - values__ = Some(map.next_value()?); + values__ = Some(map_.next_value()?); } } } @@ -19879,24 +19885,24 @@ impl<'de> serde::Deserialize<'de> for ScalarTime32Value { formatter.write_str("struct datafusion.ScalarTime32Value") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut value__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Time32SecondValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("time32SecondValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_time32_value::Value::Time32SecondValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_time32_value::Value::Time32SecondValue(x.0)); } GeneratedField::Time32MillisecondValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("time32MillisecondValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_time32_value::Value::Time32MillisecondValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_time32_value::Value::Time32MillisecondValue(x.0)); } } } @@ -19923,9 +19929,11 @@ impl serde::Serialize for ScalarTime64Value { if let Some(v) = self.value.as_ref() { match v { scalar_time64_value::Value::Time64MicrosecondValue(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("time64MicrosecondValue", ToString::to_string(&v).as_str())?; } scalar_time64_value::Value::Time64NanosecondValue(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("time64NanosecondValue", ToString::to_string(&v).as_str())?; } } @@ -19988,24 +19996,24 @@ impl<'de> serde::Deserialize<'de> for ScalarTime64Value { formatter.write_str("struct datafusion.ScalarTime64Value") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut value__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Time64MicrosecondValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("time64MicrosecondValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_time64_value::Value::Time64MicrosecondValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_time64_value::Value::Time64MicrosecondValue(x.0)); } GeneratedField::Time64NanosecondValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("time64NanosecondValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_time64_value::Value::Time64NanosecondValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_time64_value::Value::Time64NanosecondValue(x.0)); } } } @@ -20038,15 +20046,19 @@ impl serde::Serialize for ScalarTimestampValue { if let Some(v) = self.value.as_ref() { match v { scalar_timestamp_value::Value::TimeMicrosecondValue(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("timeMicrosecondValue", ToString::to_string(&v).as_str())?; } scalar_timestamp_value::Value::TimeNanosecondValue(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("timeNanosecondValue", ToString::to_string(&v).as_str())?; } scalar_timestamp_value::Value::TimeSecondValue(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("timeSecondValue", ToString::to_string(&v).as_str())?; } scalar_timestamp_value::Value::TimeMillisecondValue(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("timeMillisecondValue", ToString::to_string(&v).as_str())?; } } @@ -20120,43 +20132,43 @@ impl<'de> serde::Deserialize<'de> for ScalarTimestampValue { formatter.write_str("struct datafusion.ScalarTimestampValue") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut timezone__ = None; let mut value__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Timezone => { if timezone__.is_some() { return Err(serde::de::Error::duplicate_field("timezone")); } - timezone__ = Some(map.next_value()?); + timezone__ = Some(map_.next_value()?); } GeneratedField::TimeMicrosecondValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("timeMicrosecondValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_timestamp_value::Value::TimeMicrosecondValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_timestamp_value::Value::TimeMicrosecondValue(x.0)); } GeneratedField::TimeNanosecondValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("timeNanosecondValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_timestamp_value::Value::TimeNanosecondValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_timestamp_value::Value::TimeNanosecondValue(x.0)); } GeneratedField::TimeSecondValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("timeSecondValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_timestamp_value::Value::TimeSecondValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_timestamp_value::Value::TimeSecondValue(x.0)); } GeneratedField::TimeMillisecondValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("timeMillisecondValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_timestamp_value::Value::TimeMillisecondValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_timestamp_value::Value::TimeMillisecondValue(x.0)); } } } @@ -20247,25 +20259,25 @@ impl<'de> serde::Deserialize<'de> for ScalarUdfExprNode { formatter.write_str("struct datafusion.ScalarUDFExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut fun_name__ = None; let mut args__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::FunName => { if fun_name__.is_some() { return Err(serde::de::Error::duplicate_field("funName")); } - fun_name__ = Some(map.next_value()?); + fun_name__ = Some(map_.next_value()?); } GeneratedField::Args => { if args__.is_some() { return Err(serde::de::Error::duplicate_field("args")); } - args__ = Some(map.next_value()?); + args__ = Some(map_.next_value()?); } } } @@ -20314,6 +20326,7 @@ impl serde::Serialize for ScalarValue { struct_ser.serialize_field("int32Value", v)?; } scalar_value::Value::Int64Value(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("int64Value", ToString::to_string(&v).as_str())?; } scalar_value::Value::Uint8Value(v) => { @@ -20326,6 +20339,7 @@ impl serde::Serialize for ScalarValue { struct_ser.serialize_field("uint32Value", v)?; } scalar_value::Value::Uint64Value(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("uint64Value", ToString::to_string(&v).as_str())?; } scalar_value::Value::Float32Value(v) => { @@ -20350,24 +20364,30 @@ impl serde::Serialize for ScalarValue { struct_ser.serialize_field("decimal256Value", v)?; } scalar_value::Value::Date64Value(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("date64Value", ToString::to_string(&v).as_str())?; } scalar_value::Value::IntervalYearmonthValue(v) => { struct_ser.serialize_field("intervalYearmonthValue", v)?; } scalar_value::Value::IntervalDaytimeValue(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("intervalDaytimeValue", ToString::to_string(&v).as_str())?; } scalar_value::Value::DurationSecondValue(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("durationSecondValue", ToString::to_string(&v).as_str())?; } scalar_value::Value::DurationMillisecondValue(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("durationMillisecondValue", ToString::to_string(&v).as_str())?; } scalar_value::Value::DurationMicrosecondValue(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("durationMicrosecondValue", ToString::to_string(&v).as_str())?; } scalar_value::Value::DurationNanosecondValue(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("durationNanosecondValue", ToString::to_string(&v).as_str())?; } scalar_value::Value::TimestampValue(v) => { @@ -20377,9 +20397,11 @@ impl serde::Serialize for ScalarValue { struct_ser.serialize_field("dictionaryValue", v)?; } scalar_value::Value::BinaryValue(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("binaryValue", pbjson::private::base64::encode(&v).as_str())?; } scalar_value::Value::LargeBinaryValue(v) => { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("largeBinaryValue", pbjson::private::base64::encode(&v).as_str())?; } scalar_value::Value::Time64Value(v) => { @@ -20582,226 +20604,226 @@ impl<'de> serde::Deserialize<'de> for ScalarValue { formatter.write_str("struct datafusion.ScalarValue") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut value__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::NullValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("nullValue")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::NullValue) + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::NullValue) ; } GeneratedField::BoolValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("boolValue")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::BoolValue); + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::BoolValue); } GeneratedField::Utf8Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("utf8Value")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::Utf8Value); + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::Utf8Value); } GeneratedField::LargeUtf8Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("largeUtf8Value")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::LargeUtf8Value); + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::LargeUtf8Value); } GeneratedField::Int8Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("int8Value")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Int8Value(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Int8Value(x.0)); } GeneratedField::Int16Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("int16Value")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Int16Value(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Int16Value(x.0)); } GeneratedField::Int32Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("int32Value")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Int32Value(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Int32Value(x.0)); } GeneratedField::Int64Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("int64Value")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Int64Value(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Int64Value(x.0)); } GeneratedField::Uint8Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("uint8Value")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Uint8Value(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Uint8Value(x.0)); } GeneratedField::Uint16Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("uint16Value")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Uint16Value(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Uint16Value(x.0)); } GeneratedField::Uint32Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("uint32Value")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Uint32Value(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Uint32Value(x.0)); } GeneratedField::Uint64Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("uint64Value")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Uint64Value(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Uint64Value(x.0)); } GeneratedField::Float32Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("float32Value")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Float32Value(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Float32Value(x.0)); } GeneratedField::Float64Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("float64Value")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Float64Value(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Float64Value(x.0)); } GeneratedField::Date32Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("date32Value")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Date32Value(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Date32Value(x.0)); } GeneratedField::Time32Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("time32Value")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::Time32Value) + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::Time32Value) ; } GeneratedField::ListValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("listValue")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::ListValue) + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::ListValue) ; } GeneratedField::Decimal128Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("decimal128Value")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::Decimal128Value) + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::Decimal128Value) ; } GeneratedField::Decimal256Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("decimal256Value")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::Decimal256Value) + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::Decimal256Value) ; } GeneratedField::Date64Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("date64Value")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Date64Value(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::Date64Value(x.0)); } GeneratedField::IntervalYearmonthValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("intervalYearmonthValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::IntervalYearmonthValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::IntervalYearmonthValue(x.0)); } GeneratedField::IntervalDaytimeValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("intervalDaytimeValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::IntervalDaytimeValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::IntervalDaytimeValue(x.0)); } GeneratedField::DurationSecondValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("durationSecondValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::DurationSecondValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::DurationSecondValue(x.0)); } GeneratedField::DurationMillisecondValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("durationMillisecondValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::DurationMillisecondValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::DurationMillisecondValue(x.0)); } GeneratedField::DurationMicrosecondValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("durationMicrosecondValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::DurationMicrosecondValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::DurationMicrosecondValue(x.0)); } GeneratedField::DurationNanosecondValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("durationNanosecondValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::DurationNanosecondValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| scalar_value::Value::DurationNanosecondValue(x.0)); } GeneratedField::TimestampValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("timestampValue")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::TimestampValue) + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::TimestampValue) ; } GeneratedField::DictionaryValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("dictionaryValue")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::DictionaryValue) + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::DictionaryValue) ; } GeneratedField::BinaryValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("binaryValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::BytesDeserialize<_>>>()?.map(|x| scalar_value::Value::BinaryValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::BytesDeserialize<_>>>()?.map(|x| scalar_value::Value::BinaryValue(x.0)); } GeneratedField::LargeBinaryValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("largeBinaryValue")); } - value__ = map.next_value::<::std::option::Option<::pbjson::private::BytesDeserialize<_>>>()?.map(|x| scalar_value::Value::LargeBinaryValue(x.0)); + value__ = map_.next_value::<::std::option::Option<::pbjson::private::BytesDeserialize<_>>>()?.map(|x| scalar_value::Value::LargeBinaryValue(x.0)); } GeneratedField::Time64Value => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("time64Value")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::Time64Value) + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::Time64Value) ; } GeneratedField::IntervalMonthDayNano => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("intervalMonthDayNano")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::IntervalMonthDayNano) + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::IntervalMonthDayNano) ; } GeneratedField::StructValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("structValue")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::StructValue) + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::StructValue) ; } GeneratedField::FixedSizeBinaryValue => { if value__.is_some() { return Err(serde::de::Error::duplicate_field("fixedSizeBinaryValue")); } - value__ = map.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::FixedSizeBinaryValue) + value__ = map_.next_value::<::std::option::Option<_>>()?.map(scalar_value::Value::FixedSizeBinaryValue) ; } } @@ -20882,19 +20904,19 @@ impl<'de> serde::Deserialize<'de> for ScanLimit { formatter.write_str("struct datafusion.ScanLimit") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut limit__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Limit => { if limit__.is_some() { return Err(serde::de::Error::duplicate_field("limit")); } limit__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -20984,26 +21006,26 @@ impl<'de> serde::Deserialize<'de> for Schema { formatter.write_str("struct datafusion.Schema") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut columns__ = None; let mut metadata__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Columns => { if columns__.is_some() { return Err(serde::de::Error::duplicate_field("columns")); } - columns__ = Some(map.next_value()?); + columns__ = Some(map_.next_value()?); } GeneratedField::Metadata => { if metadata__.is_some() { return Err(serde::de::Error::duplicate_field("metadata")); } metadata__ = Some( - map.next_value::>()? + map_.next_value::>()? ); } } @@ -21085,18 +21107,18 @@ impl<'de> serde::Deserialize<'de> for SelectionExecNode { formatter.write_str("struct datafusion.SelectionExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -21185,25 +21207,25 @@ impl<'de> serde::Deserialize<'de> for SelectionNode { formatter.write_str("struct datafusion.SelectionNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } } } @@ -21232,8 +21254,8 @@ impl serde::Serialize for Sharpness { } let mut struct_ser = serializer.serialize_struct("datafusion.Sharpness", len)?; if self.sharpness_info != 0 { - let v = SharpnessInfo::from_i32(self.sharpness_info) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.sharpness_info)))?; + let v = SharpnessInfo::try_from(self.sharpness_info) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.sharpness_info)))?; struct_ser.serialize_field("sharpnessInfo", &v)?; } if let Some(v) = self.val.as_ref() { @@ -21296,25 +21318,25 @@ impl<'de> serde::Deserialize<'de> for Sharpness { formatter.write_str("struct datafusion.Sharpness") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut sharpness_info__ = None; let mut val__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::SharpnessInfo => { if sharpness_info__.is_some() { return Err(serde::de::Error::duplicate_field("sharpnessInfo")); } - sharpness_info__ = Some(map.next_value::()? as i32); + sharpness_info__ = Some(map_.next_value::()? as i32); } GeneratedField::Val => { if val__.is_some() { return Err(serde::de::Error::duplicate_field("val")); } - val__ = map.next_value()?; + val__ = map_.next_value()?; } } } @@ -21366,10 +21388,9 @@ impl<'de> serde::Deserialize<'de> for SharpnessInfo { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(SharpnessInfo::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -21379,10 +21400,9 @@ impl<'de> serde::Deserialize<'de> for SharpnessInfo { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(SharpnessInfo::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -21499,7 +21519,7 @@ impl<'de> serde::Deserialize<'de> for SimilarToNode { formatter.write_str("struct datafusion.SimilarToNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -21507,31 +21527,31 @@ impl<'de> serde::Deserialize<'de> for SimilarToNode { let mut expr__ = None; let mut pattern__ = None; let mut escape_char__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Negated => { if negated__.is_some() { return Err(serde::de::Error::duplicate_field("negated")); } - negated__ = Some(map.next_value()?); + negated__ = Some(map_.next_value()?); } GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::Pattern => { if pattern__.is_some() { return Err(serde::de::Error::duplicate_field("pattern")); } - pattern__ = map.next_value()?; + pattern__ = map_.next_value()?; } GeneratedField::EscapeChar => { if escape_char__.is_some() { return Err(serde::de::Error::duplicate_field("escapeChar")); } - escape_char__ = Some(map.next_value()?); + escape_char__ = Some(map_.next_value()?); } } } @@ -21574,6 +21594,7 @@ impl serde::Serialize for SortExecNode { struct_ser.serialize_field("expr", &self.expr)?; } if self.fetch != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } if self.preserve_partitioning { @@ -21642,7 +21663,7 @@ impl<'de> serde::Deserialize<'de> for SortExecNode { formatter.write_str("struct datafusion.SortExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -21650,33 +21671,33 @@ impl<'de> serde::Deserialize<'de> for SortExecNode { let mut expr__ = None; let mut fetch__ = None; let mut preserve_partitioning__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = Some(map.next_value()?); + expr__ = Some(map_.next_value()?); } GeneratedField::Fetch => { if fetch__.is_some() { return Err(serde::de::Error::duplicate_field("fetch")); } fetch__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::PreservePartitioning => { if preserve_partitioning__.is_some() { return Err(serde::de::Error::duplicate_field("preservePartitioning")); } - preserve_partitioning__ = Some(map.next_value()?); + preserve_partitioning__ = Some(map_.next_value()?); } } } @@ -21778,32 +21799,32 @@ impl<'de> serde::Deserialize<'de> for SortExprNode { formatter.write_str("struct datafusion.SortExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; let mut asc__ = None; let mut nulls_first__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::Asc => { if asc__.is_some() { return Err(serde::de::Error::duplicate_field("asc")); } - asc__ = Some(map.next_value()?); + asc__ = Some(map_.next_value()?); } GeneratedField::NullsFirst => { if nulls_first__.is_some() { return Err(serde::de::Error::duplicate_field("nullsFirst")); } - nulls_first__ = Some(map.next_value()?); + nulls_first__ = Some(map_.next_value()?); } } } @@ -21842,6 +21863,7 @@ impl serde::Serialize for SortNode { struct_ser.serialize_field("expr", &self.expr)?; } if self.fetch != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } struct_ser.end() @@ -21903,33 +21925,33 @@ impl<'de> serde::Deserialize<'de> for SortNode { formatter.write_str("struct datafusion.SortNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut expr__ = None; let mut fetch__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = Some(map.next_value()?); + expr__ = Some(map_.next_value()?); } GeneratedField::Fetch => { if fetch__.is_some() { return Err(serde::de::Error::duplicate_field("fetch")); } fetch__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -21969,6 +21991,7 @@ impl serde::Serialize for SortPreservingMergeExecNode { struct_ser.serialize_field("expr", &self.expr)?; } if self.fetch != 0 { + #[allow(clippy::needless_borrow)] struct_ser.serialize_field("fetch", ToString::to_string(&self.fetch).as_str())?; } struct_ser.end() @@ -22030,33 +22053,33 @@ impl<'de> serde::Deserialize<'de> for SortPreservingMergeExecNode { formatter.write_str("struct datafusion.SortPreservingMergeExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut expr__ = None; let mut fetch__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = Some(map.next_value()?); + expr__ = Some(map_.next_value()?); } GeneratedField::Fetch => { if fetch__.is_some() { return Err(serde::de::Error::duplicate_field("fetch")); } fetch__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } } @@ -22160,32 +22183,32 @@ impl<'de> serde::Deserialize<'de> for Statistics { formatter.write_str("struct datafusion.Statistics") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut num_rows__ = None; let mut total_byte_size__ = None; let mut column_stats__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::NumRows => { if num_rows__.is_some() { return Err(serde::de::Error::duplicate_field("numRows")); } - num_rows__ = map.next_value()?; + num_rows__ = map_.next_value()?; } GeneratedField::TotalByteSize => { if total_byte_size__.is_some() { return Err(serde::de::Error::duplicate_field("totalByteSize")); } - total_byte_size__ = map.next_value()?; + total_byte_size__ = map_.next_value()?; } GeneratedField::ColumnStats => { if column_stats__.is_some() { return Err(serde::de::Error::duplicate_field("columnStats")); } - column_stats__ = Some(map.next_value()?); + column_stats__ = Some(map_.next_value()?); } } } @@ -22277,25 +22300,25 @@ impl<'de> serde::Deserialize<'de> for StringifiedPlan { formatter.write_str("struct datafusion.StringifiedPlan") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut plan_type__ = None; let mut plan__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::PlanType => { if plan_type__.is_some() { return Err(serde::de::Error::duplicate_field("planType")); } - plan_type__ = map.next_value()?; + plan_type__ = map_.next_value()?; } GeneratedField::Plan => { if plan__.is_some() { return Err(serde::de::Error::duplicate_field("plan")); } - plan__ = Some(map.next_value()?); + plan__ = Some(map_.next_value()?); } } } @@ -22377,18 +22400,18 @@ impl<'de> serde::Deserialize<'de> for Struct { formatter.write_str("struct datafusion.Struct") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut sub_field_types__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::SubFieldTypes => { if sub_field_types__.is_some() { return Err(serde::de::Error::duplicate_field("subFieldTypes")); } - sub_field_types__ = Some(map.next_value()?); + sub_field_types__ = Some(map_.next_value()?); } } } @@ -22478,25 +22501,25 @@ impl<'de> serde::Deserialize<'de> for StructValue { formatter.write_str("struct datafusion.StructValue") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut field_values__ = None; let mut fields__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::FieldValues => { if field_values__.is_some() { return Err(serde::de::Error::duplicate_field("fieldValues")); } - field_values__ = Some(map.next_value()?); + field_values__ = Some(map_.next_value()?); } GeneratedField::Fields => { if fields__.is_some() { return Err(serde::de::Error::duplicate_field("fields")); } - fields__ = Some(map.next_value()?); + fields__ = Some(map_.next_value()?); } } } @@ -22586,25 +22609,25 @@ impl<'de> serde::Deserialize<'de> for SubqueryAliasNode { formatter.write_str("struct datafusion.SubqueryAliasNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut alias__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Alias => { if alias__.is_some() { return Err(serde::de::Error::duplicate_field("alias")); } - alias__ = map.next_value()?; + alias__ = map_.next_value()?; } } } @@ -22658,10 +22681,9 @@ impl<'de> serde::Deserialize<'de> for TimeUnit { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(TimeUnit::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -22671,10 +22693,9 @@ impl<'de> serde::Deserialize<'de> for TimeUnit { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(TimeUnit::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -22712,8 +22733,8 @@ impl serde::Serialize for Timestamp { } let mut struct_ser = serializer.serialize_struct("datafusion.Timestamp", len)?; if self.time_unit != 0 { - let v = TimeUnit::from_i32(self.time_unit) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.time_unit)))?; + let v = TimeUnit::try_from(self.time_unit) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.time_unit)))?; struct_ser.serialize_field("timeUnit", &v)?; } if !self.timezone.is_empty() { @@ -22776,25 +22797,25 @@ impl<'de> serde::Deserialize<'de> for Timestamp { formatter.write_str("struct datafusion.Timestamp") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut time_unit__ = None; let mut timezone__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::TimeUnit => { if time_unit__.is_some() { return Err(serde::de::Error::duplicate_field("timeUnit")); } - time_unit__ = Some(map.next_value::()? as i32); + time_unit__ = Some(map_.next_value::()? as i32); } GeneratedField::Timezone => { if timezone__.is_some() { return Err(serde::de::Error::duplicate_field("timezone")); } - timezone__ = Some(map.next_value()?); + timezone__ = Some(map_.next_value()?); } } } @@ -22885,25 +22906,25 @@ impl<'de> serde::Deserialize<'de> for TryCastNode { formatter.write_str("struct datafusion.TryCastNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut expr__ = None; let mut arrow_type__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::ArrowType => { if arrow_type__.is_some() { return Err(serde::de::Error::duplicate_field("arrowType")); } - arrow_type__ = map.next_value()?; + arrow_type__ = map_.next_value()?; } } } @@ -22938,8 +22959,8 @@ impl serde::Serialize for Union { struct_ser.serialize_field("unionTypes", &self.union_types)?; } if self.union_mode != 0 { - let v = UnionMode::from_i32(self.union_mode) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.union_mode)))?; + let v = UnionMode::try_from(self.union_mode) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.union_mode)))?; struct_ser.serialize_field("unionMode", &v)?; } if !self.type_ids.is_empty() { @@ -23007,33 +23028,33 @@ impl<'de> serde::Deserialize<'de> for Union { formatter.write_str("struct datafusion.Union") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut union_types__ = None; let mut union_mode__ = None; let mut type_ids__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::UnionTypes => { if union_types__.is_some() { return Err(serde::de::Error::duplicate_field("unionTypes")); } - union_types__ = Some(map.next_value()?); + union_types__ = Some(map_.next_value()?); } GeneratedField::UnionMode => { if union_mode__.is_some() { return Err(serde::de::Error::duplicate_field("unionMode")); } - union_mode__ = Some(map.next_value::()? as i32); + union_mode__ = Some(map_.next_value::()? as i32); } GeneratedField::TypeIds => { if type_ids__.is_some() { return Err(serde::de::Error::duplicate_field("typeIds")); } type_ids__ = - Some(map.next_value::>>()? + Some(map_.next_value::>>()? .into_iter().map(|x| x.0).collect()) ; } @@ -23117,18 +23138,18 @@ impl<'de> serde::Deserialize<'de> for UnionExecNode { formatter.write_str("struct datafusion.UnionExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut inputs__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Inputs => { if inputs__.is_some() { return Err(serde::de::Error::duplicate_field("inputs")); } - inputs__ = Some(map.next_value()?); + inputs__ = Some(map_.next_value()?); } } } @@ -23177,10 +23198,9 @@ impl<'de> serde::Deserialize<'de> for UnionMode { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(UnionMode::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -23190,10 +23210,9 @@ impl<'de> serde::Deserialize<'de> for UnionMode { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(UnionMode::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -23281,18 +23300,18 @@ impl<'de> serde::Deserialize<'de> for UnionNode { formatter.write_str("struct datafusion.UnionNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut inputs__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Inputs => { if inputs__.is_some() { return Err(serde::de::Error::duplicate_field("inputs")); } - inputs__ = Some(map.next_value()?); + inputs__ = Some(map_.next_value()?); } } } @@ -23320,6 +23339,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)] struct_ser.serialize_field("nCols", ToString::to_string(&self.n_cols).as_str())?; } if !self.values_list.is_empty() { @@ -23383,27 +23403,27 @@ impl<'de> serde::Deserialize<'de> for ValuesNode { formatter.write_str("struct datafusion.ValuesNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut n_cols__ = None; let mut values_list__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::NCols => { if n_cols__.is_some() { return Err(serde::de::Error::duplicate_field("nCols")); } n_cols__ = - Some(map.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) + Some(map_.next_value::<::pbjson::private::NumberDeserialize<_>>()?.0) ; } GeneratedField::ValuesList => { if values_list__.is_some() { return Err(serde::de::Error::duplicate_field("valuesList")); } - values_list__ = Some(map.next_value()?); + values_list__ = Some(map_.next_value()?); } } } @@ -23521,7 +23541,7 @@ impl<'de> serde::Deserialize<'de> for ViewTableScanNode { formatter.write_str("struct datafusion.ViewTableScanNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -23530,37 +23550,37 @@ impl<'de> serde::Deserialize<'de> for ViewTableScanNode { let mut schema__ = None; let mut projection__ = None; let mut definition__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::TableName => { if table_name__.is_some() { return Err(serde::de::Error::duplicate_field("tableName")); } - table_name__ = map.next_value()?; + table_name__ = map_.next_value()?; } GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::Schema => { if schema__.is_some() { return Err(serde::de::Error::duplicate_field("schema")); } - schema__ = map.next_value()?; + schema__ = map_.next_value()?; } GeneratedField::Projection => { if projection__.is_some() { return Err(serde::de::Error::duplicate_field("projection")); } - projection__ = map.next_value()?; + projection__ = map_.next_value()?; } GeneratedField::Definition => { if definition__.is_some() { return Err(serde::de::Error::duplicate_field("definition")); } - definition__ = Some(map.next_value()?); + definition__ = Some(map_.next_value()?); } } } @@ -23655,25 +23675,25 @@ impl<'de> serde::Deserialize<'de> for WhenThen { formatter.write_str("struct datafusion.WhenThen") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut when_expr__ = None; let mut then_expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::WhenExpr => { if when_expr__.is_some() { return Err(serde::de::Error::duplicate_field("whenExpr")); } - when_expr__ = map.next_value()?; + when_expr__ = map_.next_value()?; } GeneratedField::ThenExpr => { if then_expr__.is_some() { return Err(serde::de::Error::duplicate_field("thenExpr")); } - then_expr__ = map.next_value()?; + then_expr__ = map_.next_value()?; } } } @@ -23810,7 +23830,7 @@ impl<'de> serde::Deserialize<'de> for WindowAggExecNode { formatter.write_str("struct datafusion.WindowAggExecNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -23819,51 +23839,51 @@ impl<'de> serde::Deserialize<'de> for WindowAggExecNode { let mut input_schema__ = None; let mut partition_keys__ = None; let mut partition_search_mode__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::WindowExpr => { if window_expr__.is_some() { return Err(serde::de::Error::duplicate_field("windowExpr")); } - window_expr__ = Some(map.next_value()?); + window_expr__ = Some(map_.next_value()?); } GeneratedField::InputSchema => { if input_schema__.is_some() { return Err(serde::de::Error::duplicate_field("inputSchema")); } - input_schema__ = map.next_value()?; + input_schema__ = map_.next_value()?; } GeneratedField::PartitionKeys => { if partition_keys__.is_some() { return Err(serde::de::Error::duplicate_field("partitionKeys")); } - partition_keys__ = Some(map.next_value()?); + partition_keys__ = Some(map_.next_value()?); } GeneratedField::Linear => { if partition_search_mode__.is_some() { return Err(serde::de::Error::duplicate_field("linear")); } - partition_search_mode__ = map.next_value::<::std::option::Option<_>>()?.map(window_agg_exec_node::PartitionSearchMode::Linear) + partition_search_mode__ = map_.next_value::<::std::option::Option<_>>()?.map(window_agg_exec_node::PartitionSearchMode::Linear) ; } GeneratedField::PartiallySorted => { if partition_search_mode__.is_some() { return Err(serde::de::Error::duplicate_field("partiallySorted")); } - partition_search_mode__ = map.next_value::<::std::option::Option<_>>()?.map(window_agg_exec_node::PartitionSearchMode::PartiallySorted) + partition_search_mode__ = map_.next_value::<::std::option::Option<_>>()?.map(window_agg_exec_node::PartitionSearchMode::PartiallySorted) ; } GeneratedField::Sorted => { if partition_search_mode__.is_some() { return Err(serde::de::Error::duplicate_field("sorted")); } - partition_search_mode__ = map.next_value::<::std::option::Option<_>>()?.map(window_agg_exec_node::PartitionSearchMode::Sorted) + partition_search_mode__ = map_.next_value::<::std::option::Option<_>>()?.map(window_agg_exec_node::PartitionSearchMode::Sorted) ; } } @@ -23919,13 +23939,13 @@ impl serde::Serialize for WindowExprNode { if let Some(v) = self.window_function.as_ref() { match v { window_expr_node::WindowFunction::AggrFunction(v) => { - let v = AggregateFunction::from_i32(*v) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; + let v = AggregateFunction::try_from(*v) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; struct_ser.serialize_field("aggrFunction", &v)?; } window_expr_node::WindowFunction::BuiltInFunction(v) => { - let v = BuiltInWindowFunction::from_i32(*v) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; + let v = BuiltInWindowFunction::try_from(*v) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", *v)))?; struct_ser.serialize_field("builtInFunction", &v)?; } window_expr_node::WindowFunction::Udaf(v) => { @@ -24015,7 +24035,7 @@ impl<'de> serde::Deserialize<'de> for WindowExprNode { formatter.write_str("struct datafusion.WindowExprNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { @@ -24024,55 +24044,55 @@ impl<'de> serde::Deserialize<'de> for WindowExprNode { let mut order_by__ = None; let mut window_frame__ = None; let mut window_function__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Expr => { if expr__.is_some() { return Err(serde::de::Error::duplicate_field("expr")); } - expr__ = map.next_value()?; + expr__ = map_.next_value()?; } GeneratedField::PartitionBy => { if partition_by__.is_some() { return Err(serde::de::Error::duplicate_field("partitionBy")); } - partition_by__ = Some(map.next_value()?); + partition_by__ = Some(map_.next_value()?); } GeneratedField::OrderBy => { if order_by__.is_some() { return Err(serde::de::Error::duplicate_field("orderBy")); } - order_by__ = Some(map.next_value()?); + order_by__ = Some(map_.next_value()?); } GeneratedField::WindowFrame => { if window_frame__.is_some() { return Err(serde::de::Error::duplicate_field("windowFrame")); } - window_frame__ = map.next_value()?; + window_frame__ = map_.next_value()?; } GeneratedField::AggrFunction => { if window_function__.is_some() { return Err(serde::de::Error::duplicate_field("aggrFunction")); } - window_function__ = map.next_value::<::std::option::Option>()?.map(|x| window_expr_node::WindowFunction::AggrFunction(x as i32)); + window_function__ = map_.next_value::<::std::option::Option>()?.map(|x| window_expr_node::WindowFunction::AggrFunction(x as i32)); } GeneratedField::BuiltInFunction => { if window_function__.is_some() { return Err(serde::de::Error::duplicate_field("builtInFunction")); } - window_function__ = map.next_value::<::std::option::Option>()?.map(|x| window_expr_node::WindowFunction::BuiltInFunction(x as i32)); + window_function__ = map_.next_value::<::std::option::Option>()?.map(|x| window_expr_node::WindowFunction::BuiltInFunction(x as i32)); } GeneratedField::Udaf => { if window_function__.is_some() { return Err(serde::de::Error::duplicate_field("udaf")); } - window_function__ = map.next_value::<::std::option::Option<_>>()?.map(window_expr_node::WindowFunction::Udaf); + window_function__ = map_.next_value::<::std::option::Option<_>>()?.map(window_expr_node::WindowFunction::Udaf); } GeneratedField::Udwf => { if window_function__.is_some() { return Err(serde::de::Error::duplicate_field("udwf")); } - window_function__ = map.next_value::<::std::option::Option<_>>()?.map(window_expr_node::WindowFunction::Udwf); + window_function__ = map_.next_value::<::std::option::Option<_>>()?.map(window_expr_node::WindowFunction::Udwf); } } } @@ -24107,8 +24127,8 @@ impl serde::Serialize for WindowFrame { } let mut struct_ser = serializer.serialize_struct("datafusion.WindowFrame", len)?; if self.window_frame_units != 0 { - let v = WindowFrameUnits::from_i32(self.window_frame_units) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.window_frame_units)))?; + let v = WindowFrameUnits::try_from(self.window_frame_units) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.window_frame_units)))?; struct_ser.serialize_field("windowFrameUnits", &v)?; } if let Some(v) = self.start_bound.as_ref() { @@ -24182,32 +24202,32 @@ impl<'de> serde::Deserialize<'de> for WindowFrame { formatter.write_str("struct datafusion.WindowFrame") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut window_frame_units__ = None; let mut start_bound__ = None; let mut end_bound__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::WindowFrameUnits => { if window_frame_units__.is_some() { return Err(serde::de::Error::duplicate_field("windowFrameUnits")); } - window_frame_units__ = Some(map.next_value::()? as i32); + window_frame_units__ = Some(map_.next_value::()? as i32); } GeneratedField::StartBound => { if start_bound__.is_some() { return Err(serde::de::Error::duplicate_field("startBound")); } - start_bound__ = map.next_value()?; + start_bound__ = map_.next_value()?; } GeneratedField::Bound => { if end_bound__.is_some() { return Err(serde::de::Error::duplicate_field("bound")); } - end_bound__ = map.next_value::<::std::option::Option<_>>()?.map(window_frame::EndBound::Bound) + end_bound__ = map_.next_value::<::std::option::Option<_>>()?.map(window_frame::EndBound::Bound) ; } } @@ -24238,8 +24258,8 @@ impl serde::Serialize for WindowFrameBound { } let mut struct_ser = serializer.serialize_struct("datafusion.WindowFrameBound", len)?; if self.window_frame_bound_type != 0 { - let v = WindowFrameBoundType::from_i32(self.window_frame_bound_type) - .ok_or_else(|| serde::ser::Error::custom(format!("Invalid variant {}", self.window_frame_bound_type)))?; + let v = WindowFrameBoundType::try_from(self.window_frame_bound_type) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.window_frame_bound_type)))?; struct_ser.serialize_field("windowFrameBoundType", &v)?; } if let Some(v) = self.bound_value.as_ref() { @@ -24303,25 +24323,25 @@ impl<'de> serde::Deserialize<'de> for WindowFrameBound { formatter.write_str("struct datafusion.WindowFrameBound") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut window_frame_bound_type__ = None; let mut bound_value__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::WindowFrameBoundType => { if window_frame_bound_type__.is_some() { return Err(serde::de::Error::duplicate_field("windowFrameBoundType")); } - window_frame_bound_type__ = Some(map.next_value::()? as i32); + window_frame_bound_type__ = Some(map_.next_value::()? as i32); } GeneratedField::BoundValue => { if bound_value__.is_some() { return Err(serde::de::Error::duplicate_field("boundValue")); } - bound_value__ = map.next_value()?; + bound_value__ = map_.next_value()?; } } } @@ -24373,10 +24393,9 @@ impl<'de> serde::Deserialize<'de> for WindowFrameBoundType { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(WindowFrameBoundType::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -24386,10 +24405,9 @@ impl<'de> serde::Deserialize<'de> for WindowFrameBoundType { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(WindowFrameBoundType::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -24449,10 +24467,9 @@ impl<'de> serde::Deserialize<'de> for WindowFrameUnits { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(WindowFrameUnits::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) }) @@ -24462,10 +24479,9 @@ impl<'de> serde::Deserialize<'de> for WindowFrameUnits { where E: serde::de::Error, { - use std::convert::TryFrom; i32::try_from(v) .ok() - .and_then(WindowFrameUnits::from_i32) + .and_then(|x| x.try_into().ok()) .ok_or_else(|| { serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) }) @@ -24564,25 +24580,25 @@ impl<'de> serde::Deserialize<'de> for WindowNode { formatter.write_str("struct datafusion.WindowNode") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map_: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { let mut input__ = None; let mut window_expr__ = None; - while let Some(k) = map.next_key()? { + while let Some(k) = map_.next_key()? { match k { GeneratedField::Input => { if input__.is_some() { return Err(serde::de::Error::duplicate_field("input")); } - input__ = map.next_value()?; + input__ = map_.next_value()?; } GeneratedField::WindowExpr => { if window_expr__.is_some() { return Err(serde::de::Error::duplicate_field("windowExpr")); } - window_expr__ = Some(map.next_value()?); + window_expr__ = Some(map_.next_value()?); } } } diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 219a79a99109..67de2fd13153 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -609,12 +609,11 @@ impl From<&protobuf::ColumnStats> for ColumnStatistics { impl From for Sharpness { fn from(s: protobuf::Sharpness) -> Self { - let sharpness_type = - if let Some(s_type) = protobuf::SharpnessInfo::from_i32(s.sharpness_info) { - s_type - } else { - return Sharpness::Absent; - }; + let sharpness_type = if let Ok(s_type) = s.sharpness_info.try_into() { + s_type + } else { + return Sharpness::Absent; + }; match sharpness_type { protobuf::SharpnessInfo::Exact => { if s.val.is_none() { @@ -645,12 +644,11 @@ impl From for Sharpness { impl From for Sharpness { fn from(s: protobuf::Sharpness) -> Self { - let sharpness_type = - if let Some(s_type) = protobuf::SharpnessInfo::from_i32(s.sharpness_info) { - s_type - } else { - return Sharpness::Absent; - }; + let sharpness_type = if let Ok(s_type) = s.sharpness_info.try_into() { + s_type + } else { + return Sharpness::Absent; + }; match sharpness_type { protobuf::SharpnessInfo::Exact => { if s.val.is_none() { From e8fa61536f3147cda2d07cad093af12e898c7c91 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 29 Sep 2023 10:04:31 +0300 Subject: [PATCH 21/53] fix after merge --- datafusion/common/src/stats.rs | 2 +- datafusion/core/src/datasource/statistics.rs | 4 ++-- .../src/windows/bounded_window_agg_exec.rs | 3 +-- .../proto/tests/cases/roundtrip_physical_plan.rs | 10 ++++++---- 4 files changed, 10 insertions(+), 9 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index c6ef2e036dfc..fefd973e7bf0 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -33,7 +33,7 @@ pub enum Sharpness { } impl Sharpness { - /// If the information is known somehow, it return the value. Otherwise, it returns None. + /// If the information is known somehow, it returns the value. Otherwise, it returns None. pub fn get_value(&self) -> Option { match self { Sharpness::Exact(val) | Sharpness::Inexact(val) => Some(val.clone()), diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 3c1d340ecc3a..5e7f904333ae 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -146,8 +146,8 @@ pub async fn get_statistics_with_limit( let statistics = if all_files.next().await.is_some() { // if we still have files in the stream, it means that the limit kicked - // in and that the statistic could have been different if we processed - // the files in a different order. + // in and the statistic could have been different if we have + // processed the files in a different order. Statistics { num_rows, total_byte_size, 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 03af07549144..9a1f0e4cd6bf 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -50,8 +50,7 @@ use datafusion_common::utils::{ evaluate_partition_ranges, get_arrayref_at_indices, get_at_indices, get_record_batch_at_indices, get_row_at_idx, }; -use datafusion_common::{exec_err, plan_err, DataFusionError, Result}; -use datafusion_execution::TaskContext; +use datafusion_common::DataFusionError; use datafusion_expr::window_state::{PartitionBatchState, WindowAggState}; use datafusion_expr::ColumnarValue; use datafusion_physical_expr::hash_utils::create_hashes; diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 77e77630bcb2..62ecaab58df9 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -53,6 +53,7 @@ use datafusion::physical_plan::{functions, udaf}; use datafusion::physical_plan::{AggregateExpr, ExecutionPlan, PhysicalExpr, Statistics}; use datafusion::prelude::SessionContext; use datafusion::scalar::ScalarValue; +use datafusion_common::stats::Sharpness; use datafusion_common::Result; use datafusion_expr::{ Accumulator, AccumulatorFactoryFunction, AggregateUDF, ReturnTypeFunction, Signature, @@ -473,10 +474,11 @@ fn roundtrip_parquet_exec_with_pruning_predicate() -> Result<()> { 1024, )]], statistics: Statistics { - num_rows: Some(100), - total_byte_size: Some(1024), - column_statistics: None, - is_exact: false, + num_rows: Sharpness::Inexact(100), + total_byte_size: Sharpness::Inexact(1024), + column_statistics: Statistics::unbounded_column_statistics(&Arc::new( + Schema::new(vec![Field::new("col", DataType::Utf8, false)]), + )), }, projection: None, limit: None, From 91c9a6b9e0b5163f69fa7811dd1e12af50959b0f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 29 Sep 2023 15:35:43 +0300 Subject: [PATCH 22/53] Minor changes, simplifications --- datafusion/common/src/stats.rs | 82 +++++-------------- datafusion/core/src/datasource/statistics.rs | 52 +++--------- .../aggregate_statistics.rs | 47 ++++------- .../enforce_distribution.rs | 11 ++- 4 files changed, 51 insertions(+), 141 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index fefd973e7bf0..4fe5d42f3ab3 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -94,6 +94,14 @@ impl Sharpness { (_, _) => Sharpness::Absent, } } + + /// Convert Sharpness from exact to inexact. + pub fn to_inexact(self) -> Self { + match self { + Sharpness::Exact(val) => Sharpness::Inexact(val), + other => other, + } + } } impl Sharpness { @@ -189,71 +197,24 @@ impl Statistics { schema .fields() .iter() - .map(|field| { - let inf = ScalarValue::try_from(field.data_type()).ok(); - ColumnStatistics { - null_count: Sharpness::Absent, - max_value: inf - .clone() - .map(Sharpness::Inexact) - .unwrap_or(Sharpness::Absent), - min_value: inf.map(Sharpness::Inexact).unwrap_or(Sharpness::Absent), - distinct_count: Sharpness::Absent, - } - }) + .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) .collect() } - /// Returns true if all the statistical parameters contain exact information. - pub fn all_exact(&self) -> bool { - self.num_rows.is_exact().unwrap_or(false) - && self.total_byte_size.is_exact().unwrap_or(false) - && self.column_statistics.iter().all(|col_stat| { - col_stat.distinct_count.is_exact().unwrap_or(false) - && col_stat.max_value.is_exact().unwrap_or(false) - && col_stat.min_value.is_exact().unwrap_or(false) - && col_stat.null_count.is_exact().unwrap_or(false) - }) - } - /// If the exactness of a [`Statistics`] instance is lost, this function relaxes /// the exactness of all information by converting them [`Sharpness::Inexact`]. pub fn make_inexact(self) -> Self { Statistics { - num_rows: if let Sharpness::Exact(val) = &self.num_rows { - Sharpness::Inexact(*val) - } else { - self.num_rows - }, - total_byte_size: if let Sharpness::Exact(val) = &self.total_byte_size { - Sharpness::Inexact(*val) - } else { - self.total_byte_size - }, + num_rows: self.num_rows.to_inexact(), + total_byte_size: self.total_byte_size.to_inexact(), column_statistics: self .column_statistics - .iter() + .into_iter() .map(|cs| ColumnStatistics { - null_count: if let Sharpness::Exact(val) = &cs.null_count { - Sharpness::Inexact(*val) - } else { - cs.null_count.clone() - }, - max_value: if let Sharpness::Exact(val) = &cs.max_value { - Sharpness::Inexact(val.clone()) - } else { - cs.max_value.clone() - }, - min_value: if let Sharpness::Exact(val) = &cs.min_value { - Sharpness::Inexact(val.clone()) - } else { - cs.min_value.clone() - }, - distinct_count: if let Sharpness::Exact(val) = &cs.distinct_count { - Sharpness::Inexact(*val) - } else { - cs.distinct_count.clone() - }, + null_count: cs.null_count.to_inexact(), + max_value: cs.max_value.to_inexact(), + min_value: cs.min_value.to_inexact(), + distinct_count: cs.distinct_count.to_inexact(), }) .collect::>(), } @@ -297,14 +258,13 @@ impl ColumnStatistics { /// Returns the [`ColumnStatistics`] corresponding to the given datatype by assigning infinite bounds. pub fn new_with_unbounded_column(dt: &DataType) -> ColumnStatistics { - let null = ScalarValue::try_from(dt.clone()).ok(); + let inf = ScalarValue::try_from(dt.clone()) + .map(Sharpness::Inexact) + .unwrap_or(Sharpness::Absent); ColumnStatistics { null_count: Sharpness::Absent, - max_value: null - .clone() - .map(Sharpness::Inexact) - .unwrap_or(Sharpness::Absent), - min_value: null.map(Sharpness::Inexact).unwrap_or(Sharpness::Absent), + max_value: inf.clone(), + min_value: inf, distinct_count: Sharpness::Absent, } } diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 5e7f904333ae..0d7fdcef405d 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -51,53 +51,23 @@ pub async fn get_statistics_with_limit( while let Some(res) = all_files.next().await { let (file, file_stats) = res?; result_files.push(file); - num_rows = if let Some(exactness) = num_rows.is_exact() { - if exactness { - if file_stats.num_rows == Sharpness::Absent { - Sharpness::Exact(0_usize) - .add(&Sharpness::Exact(num_rows.get_value().unwrap())) - } else { - file_stats - .num_rows - .add(&Sharpness::Exact(num_rows.get_value().unwrap())) - } - } else if file_stats.num_rows == Sharpness::Absent { - Sharpness::Exact(0_usize) - .add(&Sharpness::Inexact(num_rows.get_value().unwrap())) - } else { - file_stats - .num_rows - .add(&Sharpness::Inexact(num_rows.get_value().unwrap())) - } - } else { - file_stats.num_rows + num_rows = match (file_stats.num_rows, num_rows.clone()) { + (Sharpness::Absent, rhs) => rhs.to_inexact(), + (lhs, Sharpness::Absent) => lhs.to_inexact(), + (lhs, rhs) => lhs.add(&rhs), }; - total_byte_size = if let Some(exactness) = total_byte_size.is_exact() { - if exactness { - if file_stats.total_byte_size == Sharpness::Absent { - Sharpness::Exact(0_usize) - .add(&Sharpness::Exact(total_byte_size.get_value().unwrap())) - } else { - file_stats - .total_byte_size - .add(&Sharpness::Exact(total_byte_size.get_value().unwrap())) - } - } else if file_stats.total_byte_size == Sharpness::Absent { - Sharpness::Exact(0_usize) - .add(&Sharpness::Inexact(total_byte_size.get_value().unwrap())) - } else { - file_stats - .total_byte_size - .add(&Sharpness::Inexact(total_byte_size.get_value().unwrap())) - } - } else { - file_stats.total_byte_size + total_byte_size = match (file_stats.total_byte_size, total_byte_size.clone()) { + (Sharpness::Absent, rhs) => rhs.to_inexact(), + (lhs, Sharpness::Absent) => lhs.to_inexact(), + (lhs, rhs) => lhs.add(&rhs), }; + if !file_stats.column_statistics.is_empty() { has_statistics = true; for (i, cs) in file_stats.column_statistics.iter().enumerate() { null_counts[i] = if cs.null_count == Sharpness::Absent { - null_counts[i].clone() + // Downcast to inexact + null_counts[i].clone().to_inexact() } else { null_counts[i].add(&cs.null_count) }; diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 617a2307c9e1..7a98b480e20f 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -16,6 +16,7 @@ // under the License. //! Utilizing exact statistics from sources to avoid scanning data +use datafusion_common::stats::Sharpness; use std::sync::Arc; use crate::config::ConfigOptions; @@ -141,14 +142,8 @@ fn take_optimizable_table_count( agg_expr: &dyn AggregateExpr, stats: &Statistics, ) -> Option<(ScalarValue, &'static str)> { - if let (Some(num_rows), Some(casted_expr)) = ( - stats.num_rows.is_exact().and_then(|exact| { - if exact { - stats.num_rows.get_value() - } else { - None - } - }), + if let (&Sharpness::Exact(num_rows), Some(casted_expr)) = ( + &stats.num_rows, agg_expr.as_any().downcast_ref::(), ) { // TODO implementing Eq on PhysicalExpr would help a lot here @@ -175,14 +170,8 @@ fn take_optimizable_column_count( stats: &Statistics, ) -> Option<(ScalarValue, String)> { let col_stats = &stats.column_statistics; - if let (Some(num_rows), Some(casted_expr)) = ( - stats.num_rows.is_exact().and_then(|exact| { - if exact { - stats.num_rows.get_value() - } else { - None - } - }), + if let (&Sharpness::Exact(num_rows), Some(casted_expr)) = ( + &stats.num_rows, agg_expr.as_any().downcast_ref::(), ) { if casted_expr.expressions().len() == 1 { @@ -192,13 +181,11 @@ fn take_optimizable_column_count( .downcast_ref::() { let current_val = &col_stats[col_expr.index()].null_count; - if let Some(val) = current_val.get_value() { - if current_val.is_exact().unwrap_or(false) { - return Some(( - ScalarValue::Int64(Some((num_rows - val) as i64)), - casted_expr.name().to_string(), - )); - } + if let &Sharpness::Exact(val) = current_val { + return Some(( + ScalarValue::Int64(Some((num_rows - val) as i64)), + casted_expr.name().to_string(), + )); } } } @@ -219,11 +206,8 @@ fn take_optimizable_min( .as_any() .downcast_ref::() { - let current_val = &col_stats[col_expr.index()].min_value; - if let Some(val) = current_val.get_value() { - if !val.is_null() && current_val.is_exact().unwrap_or(false) { - return Some((val.clone(), casted_expr.name().to_string())); - } + if let Sharpness::Exact(val) = &col_stats[col_expr.index()].min_value { + return Some((val.clone(), casted_expr.name().to_string())); } } } @@ -244,11 +228,8 @@ fn take_optimizable_max( .as_any() .downcast_ref::() { - let current_val = &col_stats[col_expr.index()].max_value; - if let Some(val) = current_val.get_value() { - if !val.is_null() && current_val.is_exact().unwrap_or(false) { - return Some((val.clone(), casted_expr.name().to_string())); - } + if let Sharpness::Exact(val) = &col_stats[col_expr.index()].max_value { + return Some((val.clone(), casted_expr.name().to_string())); } } } diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 5c86124b296b..0ea100dba761 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -57,6 +57,7 @@ use datafusion_physical_expr::{ }; use datafusion_physical_plan::unbounded_output; +use datafusion_common::stats::Sharpness; use itertools::izip; /// The `EnforceDistribution` rule ensures that distribution requirements are @@ -1271,12 +1272,10 @@ fn ensure_distribution( // Don't need to apply when the returned row count is not greater than 1: let stats = dist_context.plan.statistics()?; let mut repartition_beneficial_stat = true; - if stats.all_exact() { - repartition_beneficial_stat = stats - .num_rows - .get_value() - .map(|num_rows| num_rows > 1) - .unwrap_or(true); + if let Sharpness::Exact(num_rows) = stats.num_rows { + // when we are sure that number of rows is <=1 + // repartitioning is not beneficial + repartition_beneficial_stat = num_rows > 1; } // Remove unnecessary repartition from the physical plan if any From cfe6fca82b2aab7142c87887e4ff34087c95495d Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 29 Sep 2023 16:48:55 +0300 Subject: [PATCH 23/53] minor changes --- datafusion/common/src/stats.rs | 16 ++-- .../core/src/datasource/listing/table.rs | 4 +- datafusion/core/src/datasource/statistics.rs | 4 +- .../aggregate_statistics.rs | 8 +- .../src/physical_optimizer/join_selection.rs | 6 +- datafusion/core/src/physical_planner.rs | 4 +- datafusion/core/tests/sql/explain_analyze.rs | 2 +- datafusion/physical-plan/src/joins/utils.rs | 13 ++- .../proto/src/physical_plan/from_proto.rs | 84 ++++++++++--------- .../proto/src/physical_plan/to_proto.rs | 30 ++----- .../sqllogictest/test_files/explain.slt | 8 +- datafusion/sqllogictest/test_files/union.slt | 15 ++-- datafusion/sqllogictest/test_files/window.slt | 11 +-- 13 files changed, 97 insertions(+), 108 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 4fe5d42f3ab3..f34e7f8c963b 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -95,7 +95,7 @@ impl Sharpness { } } - /// Convert Sharpness from exact to inexact. + /// Converts Sharpness from exact to inexact. pub fn to_inexact(self) -> Self { match self { Sharpness::Exact(val) => Sharpness::Inexact(val), @@ -147,9 +147,9 @@ impl Debug { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { - Sharpness::Exact(inner) => write!(f, "Exact:({:?})", inner), - Sharpness::Inexact(inner) => write!(f, "Approximate:({:?})", inner), - Sharpness::Absent => write!(f, "Absent Info"), + Sharpness::Exact(inner) => write!(f, "Exact({:?})", inner), + Sharpness::Inexact(inner) => write!(f, "Inexact({:?})", inner), + Sharpness::Absent => write!(f, "Absent"), } } } @@ -159,9 +159,9 @@ impl Display { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { - Sharpness::Exact(inner) => write!(f, "Exact:({})", inner), - Sharpness::Inexact(inner) => write!(f, "Approximate:({})", inner), - Sharpness::Absent => write!(f, "Absent Info"), + Sharpness::Exact(inner) => write!(f, "Exact({})", inner), + Sharpness::Inexact(inner) => write!(f, "Inexact({})", inner), + Sharpness::Absent => write!(f, "Absent"), } } } @@ -225,7 +225,7 @@ impl Display for Statistics { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { write!( f, - "Number of Rows={}, Number of Bytes={}, Columns Statistics={:?}", + "Rows={}, Bytes={}, ColStats={:?}", self.num_rows, self.total_byte_size, self.column_statistics )?; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index b90617969b46..749faa203fc8 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1113,8 +1113,8 @@ mod tests { let table = ListingTable::try_new(config)?; let exec = table.scan(&state, None, &[], None).await?; - assert_eq!(exec.statistics()?.num_rows, Sharpness::Absent); - assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Absent); + assert_eq!(exec.statistics()?.num_rows, Sharpness::Inexact(0)); + assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Inexact(0)); Ok(()) } diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 0d7fdcef405d..d1ff486df071 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -43,8 +43,8 @@ pub async fn get_statistics_with_limit( // at least one file has them. If none of the files provide them, then they // will be omitted from the statistics. The missing values will be counted // as zero. - let mut num_rows = Sharpness::Absent; - let mut total_byte_size = Sharpness::Absent; + let mut num_rows = Sharpness::Exact(0); + let mut total_byte_size = Sharpness::Exact(0); // fusing the stream allows us to call next safely even once it is finished let mut all_files = Box::pin(all_files.fuse()); diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 7a98b480e20f..3e9750e10652 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -207,7 +207,9 @@ fn take_optimizable_min( .downcast_ref::() { if let Sharpness::Exact(val) = &col_stats[col_expr.index()].min_value { - return Some((val.clone(), casted_expr.name().to_string())); + if !val.is_null() { + return Some((val.clone(), casted_expr.name().to_string())); + } } } } @@ -229,7 +231,9 @@ fn take_optimizable_max( .downcast_ref::() { if let Sharpness::Exact(val) = &col_stats[col_expr.index()].max_value { - return Some((val.clone(), casted_expr.name().to_string())); + if !val.is_null() { + return Some((val.clone(), casted_expr.name().to_string())); + } } } } diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index f8f30e429124..2cc1f8c86e4f 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -925,9 +925,9 @@ mod tests_statistical { " HashJoinExec: mode=CollectLeft, join_type=Right, on=[(small_col@1, medium_col@0)]", " ProjectionExec: expr=[big_col@1 as big_col, small_col@0 as small_col]", " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(small_col@0, big_col@0)]", - " StatisticsExec: col_count=1, row_count=Approximate:(1000)", - " StatisticsExec: col_count=1, row_count=Approximate:(100000)", - " StatisticsExec: col_count=1, row_count=Approximate:(0)", + " StatisticsExec: col_count=1, row_count=Inexact(1000)", + " StatisticsExec: col_count=1, row_count=Inexact(100000)", + " StatisticsExec: col_count=1, row_count=Inexact(0)", "", ]; assert_optimized!(expected, join); diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 0193b1d6555b..bd7d1d8acfc9 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2755,10 +2755,8 @@ mod tests { digraph { 1[shape=box label="ProjectionExec: expr=[id@0 + 2 as employee.id + Int32(2)]", tooltip=""] - 2[shape=box label="RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", tooltip=""] + 2[shape=box label="EmptyExec: produce_one_row=false", tooltip=""] 1 -> 2 [arrowhead=none, arrowtail=normal, dir=back] - 3[shape=box label="EmptyExec: produce_one_row=false", tooltip=""] - 2 -> 3 [arrowhead=none, arrowtail=normal, dir=back] } // End DataFusion GraphViz Plan "#; diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 4bf1567099bb..e5a7f163221a 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -826,5 +826,5 @@ async fn csv_explain_analyze_with_statistics() { .to_string(); // should contain scan statistics - assert_contains!(&formatted, ", statistics=[Number of Rows=Absent Info, Number of Bytes=Absent Info, Columns Statistics=[ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Utf8(NULL)), min_value: Exact:(Utf8(NULL)), distinct_count: Absent Info }]]"); + assert_contains!(&formatted, ", statistics=[Rows=Inexact(0), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Exact(Utf8(NULL)), min_value: Exact(Utf8(NULL)), distinct_count: Absent }]]"); } diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 3b7fb7a892e9..1b2a0b4310c3 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -835,8 +835,8 @@ fn estimate_inner_join_cardinality( // this when the statistics are exact (since it is a very strong assumption). if left_stat.min_value.get_value()? > right_stat.max_value.get_value()? { return match ( - left_stat.min_value.is_exact().unwrap(), - right_stat.max_value.is_exact().unwrap(), + left_stat.min_value.is_exact().unwrap_or(false), + right_stat.max_value.is_exact().unwrap_or(false), ) { (true, true) => Some(Sharpness::Exact(0)), _ => Some(Sharpness::Inexact(0)), @@ -844,8 +844,8 @@ fn estimate_inner_join_cardinality( } if left_stat.max_value.get_value()? < right_stat.min_value.get_value()? { return match ( - left_stat.max_value.is_exact().unwrap(), - right_stat.min_value.is_exact().unwrap(), + left_stat.max_value.is_exact().unwrap_or(false), + right_stat.min_value.is_exact().unwrap_or(false), ) { (true, true) => Some(Sharpness::Exact(0)), _ => Some(Sharpness::Inexact(0)), @@ -914,9 +914,8 @@ fn max_distinct_count( let ceiling = num_rows.get_value()? - stats.null_count.get_value().unwrap_or(0); Some( - if num_rows.is_exact().unwrap() - && stats.max_value.is_exact().unwrap() - && stats.min_value.is_exact().unwrap() + if let (Sharpness::Exact(_), Sharpness::Exact(_), Sharpness::Exact(_)) = + (num_rows, stats.max_value, stats.min_value) { Sharpness::Exact(numeric_range.min(ceiling)) } else { diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 67de2fd13153..1fed39e7a582 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -583,25 +583,25 @@ impl TryFrom<&protobuf::FileGroup> for Vec { impl From<&protobuf::ColumnStats> for ColumnStatistics { fn from(cs: &protobuf::ColumnStats) -> ColumnStatistics { ColumnStatistics { - null_count: if cs.null_count.is_none() { - Sharpness::Absent + null_count: if let Some(nc) = &cs.null_count { + nc.clone().into() } else { - cs.null_count.clone().unwrap().into() - }, - max_value: if cs.max_value.is_none() { Sharpness::Absent - } else { - cs.max_value.clone().unwrap().into() }, - min_value: if cs.min_value.is_none() { - Sharpness::Absent + max_value: if let Some(max) = &cs.max_value { + max.clone().into() } else { - cs.min_value.clone().unwrap().into() + Sharpness::Absent }, - distinct_count: if cs.distinct_count.is_none() { + min_value: if let Some(min) = &cs.min_value { + min.clone().into() + } else { Sharpness::Absent + }, + distinct_count: if let Some(dc) = &cs.distinct_count { + dc.clone().into() } else { - cs.distinct_count.clone().unwrap().into() + Sharpness::Absent }, } } @@ -616,23 +616,27 @@ impl From for Sharpness { }; match sharpness_type { protobuf::SharpnessInfo::Exact => { - if s.val.is_none() { - Sharpness::Absent - } else if let Ok(ScalarValue::UInt64(Some(val))) = - ScalarValue::try_from(&s.val.unwrap()) - { - Sharpness::Exact(val as usize) + if let Some(val) = s.val { + if let Ok(ScalarValue::UInt64(Some(val))) = + ScalarValue::try_from(&val) + { + Sharpness::Exact(val as usize) + } else { + Sharpness::Absent + } } else { Sharpness::Absent } } protobuf::SharpnessInfo::Inexact => { - if s.val.is_none() { - Sharpness::Absent - } else if let Ok(ScalarValue::UInt64(Some(val))) = - ScalarValue::try_from(&s.val.unwrap()) - { - Sharpness::Inexact(val as usize) + if let Some(val) = s.val { + if let Ok(ScalarValue::UInt64(Some(val))) = + ScalarValue::try_from(&val) + { + Sharpness::Inexact(val as usize) + } else { + Sharpness::Absent + } } else { Sharpness::Absent } @@ -651,19 +655,23 @@ impl From for Sharpness { }; match sharpness_type { protobuf::SharpnessInfo::Exact => { - if s.val.is_none() { - Sharpness::Absent - } else if let Ok(val) = ScalarValue::try_from(&s.val.unwrap()) { - Sharpness::Exact(val) + if let Some(val) = s.val { + if let Ok(val) = ScalarValue::try_from(&val) { + Sharpness::Exact(val) + } else { + Sharpness::Absent + } } else { Sharpness::Absent } } protobuf::SharpnessInfo::Inexact => { - if s.val.is_none() { - Sharpness::Absent - } else if let Ok(val) = ScalarValue::try_from(&s.val.unwrap()) { - Sharpness::Inexact(val) + if let Some(val) = s.val { + if let Ok(val) = ScalarValue::try_from(&val) { + Sharpness::Inexact(val) + } else { + Sharpness::Absent + } } else { Sharpness::Absent } @@ -690,15 +698,15 @@ impl TryFrom<&protobuf::Statistics> for Statistics { let column_statistics = s.column_stats.iter().map(|s| s.into()).collect::>(); Ok(Statistics { - num_rows: if s.num_rows.is_none() { - Sharpness::Absent + num_rows: if let Some(nr) = &s.num_rows { + nr.clone().into() } else { - s.num_rows.clone().unwrap().into() - }, - total_byte_size: if s.total_byte_size.is_none() { Sharpness::Absent + }, + total_byte_size: if let Some(tbs) = &s.total_byte_size { + tbs.clone().into() } else { - s.total_byte_size.clone().unwrap().into() + Sharpness::Absent }, // No column statistic (None) is encoded with empty array column_statistics, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 290d93693db9..f8e6332a951d 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -672,28 +672,14 @@ impl From<&Sharpness> for protobuf::Sharpness { impl From<&Sharpness> for protobuf::Sharpness { fn from(s: &Sharpness) -> protobuf::Sharpness { match s { - Sharpness::Exact(val) | Sharpness::Inexact(val) => { - let res: Result = val.try_into().map_err(|_| { - DataFusionError::Internal("Undefined sharpness".to_owned()) - }); - if res.is_err() { - return protobuf::Sharpness { - sharpness_info: protobuf::SharpnessInfo::Absent.into(), - val: Some(ScalarValue { value: None }), - }; - }; - if s.is_exact().unwrap() { - protobuf::Sharpness { - sharpness_info: protobuf::SharpnessInfo::Exact.into(), - val: Some(val.try_into().unwrap()), - } - } else { - protobuf::Sharpness { - sharpness_info: protobuf::SharpnessInfo::Inexact.into(), - val: Some(val.try_into().unwrap()), - } - } - } + Sharpness::Exact(val) => protobuf::Sharpness { + sharpness_info: protobuf::SharpnessInfo::Exact.into(), + val: val.try_into().ok(), + }, + Sharpness::Inexact(val) => protobuf::Sharpness { + sharpness_info: protobuf::SharpnessInfo::Inexact.into(), + val: val.try_into().ok(), + }, Sharpness::Absent => protobuf::Sharpness { sharpness_info: protobuf::SharpnessInfo::Absent.into(), val: Some(ScalarValue { value: None }), diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 7110252c3f36..e0f769e486b1 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -265,8 +265,8 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Number of Rows=Exact:(10), Number of Bytes=Absent Info, Columns Statistics=[ColumnStatistics { null_count: Absent Info, max_value: Approximate:(Int32(NULL)), min_value: Approximate:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Absent Info, max_value: Approximate:(Int32(NULL)), min_value: Approximate:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Absent Info, max_value: Approximate:(Int32(NULL)), min_value: Approximate:(Int32(NULL)), distinct_count: Absent Info }]] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Number of Rows=Absent Info, Number of Bytes=Absent Info, Columns Statistics=[ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }]] +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(0), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }]] +--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Inexact(0), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }]] # Parquet scan with statistics collected statement ok @@ -279,8 +279,8 @@ query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Number of Rows=Exact:(8), Number of Bytes=Absent Info, Columns Statistics=[ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Boolean(NULL)), min_value: Exact:(Boolean(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int64(NULL)), min_value: Exact:(Int64(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Float32(NULL)), min_value: Exact:(Float32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Float64(NULL)), min_value: Exact:(Float64(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Binary(NULL)), min_value: Exact:(Binary(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Binary(NULL)), min_value: Exact:(Binary(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(TimestampNanosecond(NULL, None)), min_value: Exact:(TimestampNanosecond(NULL, None)), distinct_count: Absent Info }]] ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Number of Rows=Exact:(8), Number of Bytes=Absent Info, Columns Statistics=[ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Boolean(NULL)), min_value: Exact:(Boolean(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int32(NULL)), min_value: Exact:(Int32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Int64(NULL)), min_value: Exact:(Int64(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Float32(NULL)), min_value: Exact:(Float32(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Float64(NULL)), min_value: Exact:(Float64(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Binary(NULL)), min_value: Exact:(Binary(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(Binary(NULL)), min_value: Exact:(Binary(NULL)), distinct_count: Absent Info }, ColumnStatistics { null_count: Exact:(0), max_value: Exact:(TimestampNanosecond(NULL, None)), min_value: Exact:(TimestampNanosecond(NULL, None)), distinct_count: Absent Info }]] +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Boolean(NULL)), min_value: Exact(Boolean(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int64(NULL)), min_value: Exact(Int64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Float32(NULL)), min_value: Exact(Float32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Float64(NULL)), min_value: Exact(Float64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Binary(NULL)), min_value: Exact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Binary(NULL)), min_value: Exact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(TimestampNanosecond(NULL, None)), min_value: Exact(TimestampNanosecond(NULL, None)), distinct_count: Absent }]] +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Boolean(NULL)), min_value: Exact(Boolean(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int64(NULL)), min_value: Exact(Int64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Float32(NULL)), min_value: Exact(Float32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Float64(NULL)), min_value: Exact(Float64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Binary(NULL)), min_value: Exact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Binary(NULL)), min_value: Exact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(TimestampNanosecond(NULL, None)), min_value: Exact(TimestampNanosecond(NULL, None)), distinct_count: Absent }]] statement ok set datafusion.execution.collect_statistics = false; diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 05eaa10dabde..c8aaa36fbcb0 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -466,8 +466,8 @@ UnionExec ----AggregateExec: mode=FinalPartitioned, gby=[Int64(1)@0 as Int64(1)], aggr=[] ------CoalesceBatchesExec: target_batch_size=8192 --------RepartitionExec: partitioning=Hash([Int64(1)@0], 4), input_partitions=4 -----------AggregateExec: mode=Partial, gby=[1 as Int64(1)], aggr=[] -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------AggregateExec: mode=Partial, gby=[1 as Int64(1)], aggr=[] --------------EmptyExec: produce_one_row=true --ProjectionExec: expr=[2 as a] ----EmptyExec: produce_one_row=true @@ -498,14 +498,11 @@ UnionExec ----AggregateExec: mode=FinalPartitioned, gby=[n@0 as n], aggr=[COUNT(*)] ------CoalesceBatchesExec: target_batch_size=8192 --------RepartitionExec: partitioning=Hash([n@0], 4), input_partitions=4 -----------AggregateExec: mode=Partial, gby=[n@0 as n], aggr=[COUNT(*)] -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------AggregateExec: mode=Partial, gby=[n@0 as n], aggr=[COUNT(*)] --------------ProjectionExec: expr=[5 as n] ----------------EmptyExec: produce_one_row=true --ProjectionExec: expr=[x@0 as count, y@1 as n] ----ProjectionExec: expr=[1 as x, MAX(Int64(10))@0 as y] -------AggregateExec: mode=Final, gby=[], aggr=[MAX(Int64(10))] ---------CoalescePartitionsExec -----------AggregateExec: mode=Partial, gby=[], aggr=[MAX(Int64(10))] -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------EmptyExec: produce_one_row=true +------AggregateExec: mode=Single, gby=[], aggr=[MAX(Int64(10))] +--------EmptyExec: produce_one_row=true diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 3d9f7511be26..26cb77378fee 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2010,13 +2010,10 @@ Projection: ARRAY_AGG(aggregate_test_100.c13) AS array_agg1 --------TableScan: aggregate_test_100 projection=[c13] physical_plan ProjectionExec: expr=[ARRAY_AGG(aggregate_test_100.c13)@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------GlobalLimitExec: skip=0, fetch=1 -------------SortExec: fetch=1, expr=[c13@0 ASC NULLS LAST] ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true +--AggregateExec: mode=Single, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] +----GlobalLimitExec: skip=0, fetch=1 +------SortExec: fetch=1, expr=[c13@0 ASC NULLS LAST] +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true query ? From 8e95696655ac5921b10cf461884041bd9effae6d Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 2 Oct 2023 11:47:24 +0300 Subject: [PATCH 24/53] min max accs removed --- datafusion/common/src/scalar.rs | 3 +- datafusion/common/src/stats.rs | 42 ++++-- .../src/datasource/file_format/parquet.rs | 5 +- datafusion/core/src/datasource/mod.rs | 2 +- datafusion/core/src/datasource/statistics.rs | 134 +++++++++++++----- datafusion/core/tests/parquet/page_pruning.rs | 9 +- 6 files changed, 142 insertions(+), 53 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 32343b98fa24..f7ade11b58fb 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -1004,7 +1004,8 @@ impl ScalarValue { | ScalarValue::Int16(None) | ScalarValue::Int32(None) | ScalarValue::Int64(None) - | ScalarValue::Float32(None) => Ok(self.clone()), + | ScalarValue::Float32(None) + | ScalarValue::Float64(None) => Ok(self.clone()), ScalarValue::Float64(Some(v)) => Ok(ScalarValue::Float64(Some(-v))), ScalarValue::Float32(Some(v)) => Ok(ScalarValue::Float32(Some(-v))), ScalarValue::Int8(Some(v)) => Ok(ScalarValue::Int8(Some(-v))), diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index f34e7f8c963b..9fba07818363 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -25,14 +25,14 @@ use std::fmt::{self, Display}; /// To deal with information whose exactness is not guaranteed, it can be wrapped with [`Sharpness`] /// to express its reliability, such as in Statistics. #[derive(Clone, PartialEq, Eq, Default)] -pub enum Sharpness { +pub enum Sharpness { Exact(T), Inexact(T), #[default] Absent, } -impl Sharpness { +impl Sharpness { /// If the information is known somehow, it returns the value. Otherwise, it returns None. pub fn get_value(&self) -> Option { match self { @@ -142,9 +142,33 @@ impl Sharpness { } } -impl Debug - for Sharpness -{ +impl Sharpness { + /// Calculates the sum of two exact or inexact values in the type of [`ScalarValue`]. + /// If one of them is a [`Sharpness::Absent`], it returns [`Sharpness::Absent`]. + pub fn add(&self, other: &Sharpness) -> Sharpness { + match (self, other) { + (Sharpness::Exact(a), Sharpness::Exact(b)) => { + if let Ok(res) = a.add(b) { + Sharpness::Exact(res) + } else { + Sharpness::Absent + } + } + (Sharpness::Inexact(a), Sharpness::Exact(b)) + | (Sharpness::Exact(a), Sharpness::Inexact(b)) + | (Sharpness::Inexact(a), Sharpness::Inexact(b)) => { + if let Ok(res) = a.add(b) { + Sharpness::Inexact(res) + } else { + Sharpness::Absent + } + } + (_, _) => Sharpness::Absent, + } + } +} + +impl Debug for Sharpness { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { Sharpness::Exact(inner) => write!(f, "Exact({:?})", inner), @@ -154,13 +178,11 @@ impl Debug } } -impl Display - for Sharpness -{ +impl Display for Sharpness { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { - Sharpness::Exact(inner) => write!(f, "Exact({})", inner), - Sharpness::Inexact(inner) => write!(f, "Inexact({})", inner), + Sharpness::Exact(inner) => write!(f, "Exact({:?})", inner), + Sharpness::Inexact(inner) => write!(f, "Inexact({:?})", inner), Sharpness::Absent => write!(f, "Absent"), } } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 0eeafc58d33f..4c2113f0b199 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -31,6 +31,7 @@ use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use tokio::task::{JoinHandle, JoinSet}; use crate::datasource::file_format::file_compression_type::FileCompressionType; +use crate::datasource::statistics::create_max_min_accs; use arrow::datatypes::SchemaRef; use arrow::datatypes::{Fields, Schema}; use async_trait::async_trait; @@ -61,7 +62,7 @@ use crate::datasource::physical_plan::{ FileGroupDisplay, FileMeta, FileSinkConfig, ParquetExec, SchemaAdapter, }; -use crate::datasource::{create_max_min_accs, get_col_stats}; +use crate::datasource::get_col_stats; use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; @@ -555,7 +556,7 @@ async fn fetch_statistics( } } - let column_stats = if has_statistics { + let column_stats: Vec = if has_statistics { get_col_stats(&table_schema, null_counts, &mut max_values, &mut min_values) } else { Statistics::unbounded_column_statistics(&table_schema) diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 35f56536510c..455818056f2c 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -42,5 +42,5 @@ pub use self::memory::MemTable; pub use self::provider::TableProvider; pub use self::view::ViewTable; pub use crate::logical_expr::TableType; +pub(crate) use statistics::get_col_stats; pub use statistics::get_statistics_with_limit; -pub(crate) use statistics::{create_max_min_accs, get_col_stats}; diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index d1ff486df071..4853adf98af5 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -20,6 +20,7 @@ use crate::error::Result; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; use crate::physical_plan::{Accumulator, ColumnStatistics, Statistics}; use datafusion_common::stats::Sharpness; +use datafusion_common::ScalarValue; use futures::Stream; use futures::StreamExt; @@ -34,10 +35,8 @@ pub async fn get_statistics_with_limit( limit: Option, ) -> Result<(Vec, Statistics)> { let mut result_files = vec![]; - let mut null_counts = vec![Sharpness::Exact(0_usize); file_schema.fields().len()]; - let mut has_statistics = false; - let (mut max_values, mut min_values) = create_max_min_accs(&file_schema); + let (mut max_values, mut min_values) = create_max_min_vecs(&file_schema); // The number of rows and the total byte size can be calculated as long as // at least one file has them. If none of the files provide them, then they @@ -62,39 +61,60 @@ pub async fn get_statistics_with_limit( (lhs, rhs) => lhs.add(&rhs), }; - if !file_stats.column_statistics.is_empty() { - has_statistics = true; - for (i, cs) in file_stats.column_statistics.iter().enumerate() { - null_counts[i] = if cs.null_count == Sharpness::Absent { - // Downcast to inexact - null_counts[i].clone().to_inexact() - } else { - null_counts[i].add(&cs.null_count) - }; + for (i, cs) in file_stats.column_statistics.iter().enumerate() { + null_counts[i] = if cs.null_count == Sharpness::Absent { + // Downcast to inexact + null_counts[i].clone().to_inexact() + } else { + null_counts[i].add(&cs.null_count) + }; - if let Some(max_value) = &mut max_values[i] { - if let Some(file_max) = cs.max_value.get_value() { - match max_value.update_batch(&[file_max.to_array()]) { - Ok(_) => {} - Err(_) => { - max_values[i] = None; - } + if let Sharpness::Exact(max) = &cs.max_value { + match max_values[i].clone() { + Sharpness::Exact(val) => { + if val < *max { + max_values[i] = Sharpness::Exact(max.clone()) + } + } + Sharpness::Inexact(val) => { + max_values[i] = max_values[i].clone().to_inexact(); + if val < *max { + max_values[i] = Sharpness::Inexact(max.clone()) } - } else { - max_values[i] = None; + } + Sharpness::Absent => { + max_values[i] = max_values[i].clone().to_inexact() + } + } + } else if let Sharpness::Inexact(max) = &cs.max_value { + if let Some(val) = max_values[i].get_value() { + if val < *max { + max_values[i] = Sharpness::Inexact(max.clone()) } } + } - if let Some(min_value) = &mut min_values[i] { - if let Some(file_min) = cs.min_value.get_value() { - match min_value.update_batch(&[file_min.to_array()]) { - Ok(_) => {} - Err(_) => { - min_values[i] = None; - } + if let Sharpness::Exact(min) = &cs.min_value { + match min_values[i].clone() { + Sharpness::Exact(val) => { + if val > *min { + min_values[i] = Sharpness::Exact(min.clone()) + } + } + Sharpness::Inexact(val) => { + min_values[i] = min_values[i].clone().to_inexact(); + if val > *min { + min_values[i] = Sharpness::Inexact(min.clone()) } - } else { - min_values[i] = None; + } + Sharpness::Absent => { + min_values[i] = min_values[i].clone().to_inexact() + } + } + } else if let Sharpness::Inexact(min) = &cs.min_value { + if let Some(val) = min_values[i].get_value() { + if val > *min { + min_values[i] = Sharpness::Inexact(min.clone()) } } } @@ -108,11 +128,8 @@ pub async fn get_statistics_with_limit( } } - let column_stats = if has_statistics { - get_col_stats(&file_schema, null_counts, &mut max_values, &mut min_values) - } else { - Statistics::new_with_unbounded_columns(&file_schema).column_statistics - }; + let column_stats = + get_col_stats_vec(&file_schema, null_counts, &max_values, &min_values); let statistics = if all_files.next().await.is_some() { // if we still have files in the stream, it means that the limit kicked @@ -135,6 +152,37 @@ pub async fn get_statistics_with_limit( Ok((result_files, statistics)) } +pub(crate) fn create_max_min_vecs( + schema: &Schema, +) -> (Vec>, Vec>) { + ( + schema + .fields() + .iter() + .map(|field| { + let dt = ScalarValue::try_from(field.data_type()); + if let Ok(dt) = dt { + Sharpness::Exact(dt) + } else { + Sharpness::Absent + } + }) + .collect::>(), + schema + .fields() + .iter() + .map(|field| { + let dt = ScalarValue::try_from(field.data_type()); + if let Ok(dt) = dt { + Sharpness::Exact(dt) + } else { + Sharpness::Absent + } + }) + .collect::>(), + ) +} + pub(crate) fn create_max_min_accs( schema: &Schema, ) -> (Vec>, Vec>) { @@ -151,6 +199,22 @@ pub(crate) fn create_max_min_accs( (max_values, min_values) } +pub(crate) fn get_col_stats_vec( + schema: &Schema, + null_counts: Vec>, + max_values: &[Sharpness], + min_values: &[Sharpness], +) -> Vec { + (0..schema.fields().len()) + .map(|i| ColumnStatistics { + null_count: null_counts[i].clone(), + max_value: max_values[i].clone(), + min_value: min_values[i].clone(), + distinct_count: Sharpness::Absent, + }) + .collect() +} + pub(crate) fn get_col_stats( schema: &Schema, null_counts: Vec>, diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 6b5a2e4f1010..74843645c426 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -240,10 +240,11 @@ async fn test_prune( expected_row_pages_pruned: Option, expected_results: usize, ) { - let output = ContextWithParquet::new(case_data_type, Page) - .await - .query(sql) - .await; + let output: crate::parquet::TestOutput = + ContextWithParquet::new(case_data_type, Page) + .await + .query(sql) + .await; println!("{}", output.description()); assert_eq!(output.predicate_evaluation_errors(), expected_errors); From 0924618d9a6acccb659ba8000aaec04a8eec8a1d Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 2 Oct 2023 11:55:50 +0300 Subject: [PATCH 25/53] fix after merge --- datafusion/common/src/lib.rs | 2 +- datafusion/core/src/datasource/listing/table.rs | 1 + datafusion/core/src/physical_optimizer/enforce_distribution.rs | 1 + datafusion/core/src/physical_optimizer/output_requirements.rs | 2 +- 4 files changed, 4 insertions(+), 2 deletions(-) diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 71782f67046d..6e50e3b5fe53 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -58,7 +58,7 @@ pub use functional_dependencies::{ pub use join_type::{JoinConstraint, JoinType}; pub use scalar::{ScalarType, ScalarValue}; pub use schema_reference::{OwnedSchemaReference, SchemaReference}; -pub use stats::{ColumnStatistics, Statistics}; +pub use stats::{ColumnStatistics, Sharpness, Statistics}; pub use table_reference::{OwnedTableReference, ResolvedTableReference, TableReference}; pub use unnest::UnnestOptions; pub use utils::project_schema; diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 66ae75841aff..30197fc70596 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1018,6 +1018,7 @@ mod tests { use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; + use datafusion_common::stats::Sharpness; use datafusion_common::{assert_contains, GetExt, ScalarValue}; use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator}; use rstest::*; diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 8daf74c63600..c02b710518e7 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -47,6 +47,7 @@ use crate::physical_plan::windows::WindowAggExec; use crate::physical_plan::Partitioning; use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; +use datafusion_common::stats::Sharpness; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::equivalence::EquivalenceProperties; diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 4b687d7f3536..f5eacd5ee60c 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -179,7 +179,7 @@ impl ExecutionPlan for OutputRequirementExec { unreachable!(); } - fn statistics(&self) -> Statistics { + fn statistics(&self) -> Result { self.input.statistics() } } From 99fc41c1cdd33a4c7bb8fea3e4098a1fb9e6d928 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 3 Oct 2023 11:14:49 +0300 Subject: [PATCH 26/53] minor changes --- .../src/datasource/file_format/parquet.rs | 2 +- datafusion/core/src/datasource/statistics.rs | 158 +++++++++--------- .../src/physical_optimizer/join_selection.rs | 4 +- 3 files changed, 83 insertions(+), 81 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 4c2113f0b199..1eb9b2bc216f 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -556,7 +556,7 @@ async fn fetch_statistics( } } - let column_stats: Vec = if has_statistics { + let column_stats = if has_statistics { get_col_stats(&table_schema, null_counts, &mut max_values, &mut min_values) } else { Statistics::unbounded_column_statistics(&table_schema) diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 4853adf98af5..87e7908c0c6b 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -36,7 +36,10 @@ pub async fn get_statistics_with_limit( ) -> Result<(Vec, Statistics)> { let mut result_files = vec![]; let mut null_counts = vec![Sharpness::Exact(0_usize); file_schema.fields().len()]; - let (mut max_values, mut min_values) = create_max_min_vecs(&file_schema); + let (mut max_values, mut min_values) = ( + create_max_min_vec(&file_schema), + create_max_min_vec(&file_schema), + ); // The number of rows and the total byte size can be calculated as long as // at least one file has them. If none of the files provide them, then they @@ -69,55 +72,9 @@ pub async fn get_statistics_with_limit( null_counts[i].add(&cs.null_count) }; - if let Sharpness::Exact(max) = &cs.max_value { - match max_values[i].clone() { - Sharpness::Exact(val) => { - if val < *max { - max_values[i] = Sharpness::Exact(max.clone()) - } - } - Sharpness::Inexact(val) => { - max_values[i] = max_values[i].clone().to_inexact(); - if val < *max { - max_values[i] = Sharpness::Inexact(max.clone()) - } - } - Sharpness::Absent => { - max_values[i] = max_values[i].clone().to_inexact() - } - } - } else if let Sharpness::Inexact(max) = &cs.max_value { - if let Some(val) = max_values[i].get_value() { - if val < *max { - max_values[i] = Sharpness::Inexact(max.clone()) - } - } - } + set_max_if_greater(&mut max_values[i], &cs.max_value); - if let Sharpness::Exact(min) = &cs.min_value { - match min_values[i].clone() { - Sharpness::Exact(val) => { - if val > *min { - min_values[i] = Sharpness::Exact(min.clone()) - } - } - Sharpness::Inexact(val) => { - min_values[i] = min_values[i].clone().to_inexact(); - if val > *min { - min_values[i] = Sharpness::Inexact(min.clone()) - } - } - Sharpness::Absent => { - min_values[i] = min_values[i].clone().to_inexact() - } - } - } else if let Sharpness::Inexact(min) = &cs.min_value { - if let Some(val) = min_values[i].get_value() { - if val > *min { - min_values[i] = Sharpness::Inexact(min.clone()) - } - } - } + set_min_if_lesser(&mut min_values[i], &cs.min_value); } // If the number of rows exceeds the limit, we can stop processing // files. This only applies when we know the number of rows. It also @@ -152,35 +109,20 @@ pub async fn get_statistics_with_limit( Ok((result_files, statistics)) } -pub(crate) fn create_max_min_vecs( - schema: &Schema, -) -> (Vec>, Vec>) { - ( - schema - .fields() - .iter() - .map(|field| { - let dt = ScalarValue::try_from(field.data_type()); - if let Ok(dt) = dt { - Sharpness::Exact(dt) - } else { - Sharpness::Absent - } - }) - .collect::>(), - schema - .fields() - .iter() - .map(|field| { - let dt = ScalarValue::try_from(field.data_type()); - if let Ok(dt) = dt { - Sharpness::Exact(dt) - } else { - Sharpness::Absent - } - }) - .collect::>(), - ) +/// It is the [`Sharpness::Exact`] version of `ColumnStatistics::new_with_unbounded_column` function. +pub(crate) fn create_max_min_vec(schema: &Schema) -> Vec> { + schema + .fields() + .iter() + .map(|field| { + let dt = ScalarValue::try_from(field.data_type()); + if let Ok(dt) = dt { + Sharpness::Exact(dt) + } else { + Sharpness::Absent + } + }) + .collect::>() } pub(crate) fn create_max_min_accs( @@ -240,3 +182,63 @@ pub(crate) fn get_col_stats( }) .collect() } + +/// If the given value is numerically greater than the original value, +/// it set the new max value with the exactness information. +fn set_max_if_greater( + max_values: &mut Sharpness, + max_value: &Sharpness, +) { + if let Sharpness::Exact(max) = &max_value { + match max_values.clone() { + Sharpness::Exact(val) => { + if val < *max { + *max_values = Sharpness::Exact(max.clone()) + } + } + Sharpness::Inexact(val) => { + *max_values = max_values.clone().to_inexact(); + if val < *max { + *max_values = Sharpness::Inexact(max.clone()) + } + } + Sharpness::Absent => *max_values = max_values.clone().to_inexact(), + } + } else if let Sharpness::Inexact(max) = &max_value { + if let Some(val) = max_values.get_value() { + if val < *max { + *max_values = Sharpness::Inexact(max.clone()) + } + } + } +} + +/// If the given value is numerically lesser than the original value, +/// it set the new min value with the exactness information. +fn set_min_if_lesser( + max_values: &mut Sharpness, + max_value: &Sharpness, +) { + if let Sharpness::Exact(max) = &max_value { + match max_values.clone() { + Sharpness::Exact(val) => { + if val < *max { + *max_values = Sharpness::Exact(max.clone()) + } + } + Sharpness::Inexact(val) => { + *max_values = max_values.clone().to_inexact(); + if val < *max { + *max_values = Sharpness::Inexact(max.clone()) + } + } + Sharpness::Absent => *max_values = max_values.clone().to_inexact(), + } + } else if let Sharpness::Inexact(max) = &max_value { + if let Some(val) = max_values.get_value() { + if val < *max { + *max_values = Sharpness::Inexact(max.clone()) + } + } + } +} diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 2cc1f8c86e4f..555ca2bc4c4f 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -680,7 +680,7 @@ mod tests_statistical { let medium = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(0_000), + num_rows: Sharpness::Inexact(10_000), column_statistics: create_column_stats( Some(1000), Some(5000), @@ -927,7 +927,7 @@ mod tests_statistical { " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(small_col@0, big_col@0)]", " StatisticsExec: col_count=1, row_count=Inexact(1000)", " StatisticsExec: col_count=1, row_count=Inexact(100000)", - " StatisticsExec: col_count=1, row_count=Inexact(0)", + " StatisticsExec: col_count=1, row_count=Inexact(10000)", "", ]; assert_optimized!(expected, join); From 681cab38986b59651d09e5dca65e8d5b4bef54f8 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 5 Oct 2023 11:05:06 +0300 Subject: [PATCH 27/53] fix initialization of stats in limit --- datafusion/core/src/datasource/statistics.rs | 76 +++++++++++++++---- datafusion/core/tests/sql/explain_analyze.rs | 2 +- .../sqllogictest/test_files/explain.slt | 6 +- 3 files changed, 65 insertions(+), 19 deletions(-) diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 87e7908c0c6b..2919e8f48683 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -19,6 +19,7 @@ use crate::arrow::datatypes::{Schema, SchemaRef}; use crate::error::Result; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; use crate::physical_plan::{Accumulator, ColumnStatistics, Statistics}; +use arrow_array::Scalar; use datafusion_common::stats::Sharpness; use datafusion_common::ScalarValue; use futures::Stream; @@ -36,10 +37,10 @@ pub async fn get_statistics_with_limit( ) -> Result<(Vec, Statistics)> { let mut result_files = vec![]; let mut null_counts = vec![Sharpness::Exact(0_usize); file_schema.fields().len()]; - let (mut max_values, mut min_values) = ( - create_max_min_vec(&file_schema), - create_max_min_vec(&file_schema), - ); + let (mut max_values, mut min_values): ( + Option>>, + Option>>, + ) = (None, None); // The number of rows and the total byte size can be calculated as long as // at least one file has them. If none of the files provide them, then they @@ -71,11 +72,38 @@ pub async fn get_statistics_with_limit( } else { null_counts[i].add(&cs.null_count) }; + } - set_max_if_greater(&mut max_values[i], &cs.max_value); + if let Some(some_max_values) = &mut max_values { + for (i, cs) in file_stats.column_statistics.iter().enumerate() { + set_max_if_greater(&mut some_max_values[i], &cs.max_value); + } + } else { + let mut new_col_stats_max = vec![]; + for cs in file_stats.column_statistics.iter() { + new_col_stats_max.push(cs.max_value.clone()); + } + for _ in 0..file_schema.fields().len() - file_stats.column_statistics.len() { + new_col_stats_max.push(Sharpness::Absent) + } + max_values = Some(new_col_stats_max); + }; + + if let Some(some_min_values) = &mut min_values { + for (i, cs) in file_stats.column_statistics.iter().enumerate() { + set_min_if_lesser(&mut some_min_values[i], &cs.min_value); + } + } else { + let mut new_col_stats_min = vec![]; + for cs in file_stats.column_statistics.iter() { + new_col_stats_min.push(cs.min_value.clone()); + } + for _ in 0..file_schema.fields().len() - file_stats.column_statistics.len() { + new_col_stats_min.push(Sharpness::Absent) + } + min_values = Some(new_col_stats_min); + }; - set_min_if_lesser(&mut min_values[i], &cs.min_value); - } // If the number of rows exceeds the limit, we can stop processing // files. This only applies when we know the number of rows. It also // currently ignores tables that have no statistics regarding the @@ -86,7 +114,7 @@ pub async fn get_statistics_with_limit( } let column_stats = - get_col_stats_vec(&file_schema, null_counts, &max_values, &min_values); + get_col_stats_vec(&file_schema, null_counts, max_values, min_values); let statistics = if all_files.next().await.is_some() { // if we still have files in the stream, it means that the limit kicked @@ -144,15 +172,33 @@ pub(crate) fn create_max_min_accs( pub(crate) fn get_col_stats_vec( schema: &Schema, null_counts: Vec>, - max_values: &[Sharpness], - min_values: &[Sharpness], + max_values: Option>>, + min_values: Option>>, ) -> Vec { (0..schema.fields().len()) - .map(|i| ColumnStatistics { - null_count: null_counts[i].clone(), - max_value: max_values[i].clone(), - min_value: min_values[i].clone(), - distinct_count: Sharpness::Absent, + .map(|i| { + let max_value = if let Some(some_max_values) = &max_values { + some_max_values[i].clone() + } else { + match ScalarValue::try_from(schema.fields[i].data_type()) { + Ok(dt) => Sharpness::Inexact(dt), + Err(_) => Sharpness::Absent, + } + }; + let min_value = if let Some(some_min_values) = &min_values { + some_min_values[i].clone() + } else { + match ScalarValue::try_from(schema.fields[i].data_type()) { + Ok(dt) => Sharpness::Inexact(dt), + Err(_) => Sharpness::Absent, + } + }; + ColumnStatistics { + null_count: null_counts[i].clone(), + max_value, + min_value, + distinct_count: Sharpness::Absent, + } }) .collect() } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index e5a7f163221a..44e1e6d74ffb 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -826,5 +826,5 @@ async fn csv_explain_analyze_with_statistics() { .to_string(); // should contain scan statistics - assert_contains!(&formatted, ", statistics=[Rows=Inexact(0), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Exact(Utf8(NULL)), min_value: Exact(Utf8(NULL)), distinct_count: Absent }]]"); + assert_contains!(&formatted, ", statistics=[Rows=Inexact(0), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Utf8(NULL)), min_value: Inexact(Utf8(NULL)), distinct_count: Absent }]]"); } diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 3a173460532a..dc17034a618c 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -270,7 +270,7 @@ EXPLAIN SELECT a, b, c FROM simple_explain_test limit 10; ---- physical_plan GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(0), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }]] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Inexact(0), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }]] +--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Inexact(0), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }]] # Parquet scan with statistics collected statement ok @@ -283,8 +283,8 @@ query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Boolean(NULL)), min_value: Exact(Boolean(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int64(NULL)), min_value: Exact(Int64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Float32(NULL)), min_value: Exact(Float32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Float64(NULL)), min_value: Exact(Float64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Binary(NULL)), min_value: Exact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Binary(NULL)), min_value: Exact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(TimestampNanosecond(NULL, None)), min_value: Exact(TimestampNanosecond(NULL, None)), distinct_count: Absent }]] ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Boolean(NULL)), min_value: Exact(Boolean(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int32(NULL)), min_value: Exact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Int64(NULL)), min_value: Exact(Int64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Float32(NULL)), min_value: Exact(Float32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Float64(NULL)), min_value: Exact(Float64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Binary(NULL)), min_value: Exact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(Binary(NULL)), min_value: Exact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Exact(TimestampNanosecond(NULL, None)), min_value: Exact(TimestampNanosecond(NULL, None)), distinct_count: Absent }]] +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Boolean(NULL)), min_value: Inexact(Boolean(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int64(NULL)), min_value: Inexact(Int64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Float32(NULL)), min_value: Inexact(Float32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Float64(NULL)), min_value: Inexact(Float64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Binary(NULL)), min_value: Inexact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Binary(NULL)), min_value: Inexact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(TimestampNanosecond(NULL, None)), min_value: Inexact(TimestampNanosecond(NULL, None)), distinct_count: Absent }]] +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Boolean(NULL)), min_value: Inexact(Boolean(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int64(NULL)), min_value: Inexact(Int64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Float32(NULL)), min_value: Inexact(Float32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Float64(NULL)), min_value: Inexact(Float64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Binary(NULL)), min_value: Inexact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Binary(NULL)), min_value: Inexact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(TimestampNanosecond(NULL, None)), min_value: Inexact(TimestampNanosecond(NULL, None)), distinct_count: Absent }]] statement ok set datafusion.execution.collect_statistics = false; From d4899932a28a2fc27de534fcc55141ba06184edc Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 5 Oct 2023 13:45:12 +0300 Subject: [PATCH 28/53] minor changes --- datafusion/common/src/stats.rs | 4 +- .../core/src/datasource/physical_plan/avro.rs | 6 +- .../physical_plan/file_scan_config.rs | 2 +- .../datasource/physical_plan/file_stream.rs | 4 +- .../core/src/datasource/physical_plan/json.rs | 8 +- .../src/datasource/physical_plan/parquet.rs | 4 +- datafusion/core/src/datasource/statistics.rs | 124 ++++++++---------- .../core/tests/parquet/custom_reader.rs | 2 +- .../core/tests/parquet/schema_coercion.rs | 4 +- datafusion/physical-plan/src/joins/utils.rs | 17 ++- 10 files changed, 82 insertions(+), 93 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 9fba07818363..36962438818a 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -33,7 +33,7 @@ pub enum Sharpness { } impl Sharpness { - /// If the information is known somehow, it returns the value. Otherwise, it returns None. + /// If the information is known somehow, it returns the cloned value. Otherwise, it returns None. pub fn get_value(&self) -> Option { match self { Sharpness::Exact(val) | Sharpness::Inexact(val) => Some(val.clone()), @@ -271,7 +271,7 @@ pub struct ColumnStatistics { impl ColumnStatistics { /// Column contains a single non null value (e.g constant). pub fn is_singleton(&self) -> bool { - match (&self.min_value.get_value(), &self.max_value.get_value()) { + match (self.min_value.get_value(), self.max_value.get_value()) { // Min and max values are the same and not infinity. (Some(min), Some(max)) => !min.is_null() && !max.is_null() && (min == max), (_, _) => false, diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 75470d83ce54..4969b214ae39 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -272,8 +272,8 @@ mod tests { let avro_exec = AvroExec::new(FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![vec![meta.into()]], - file_schema: file_schema.clone(), statistics: Statistics::new_with_unbounded_columns(&file_schema), + file_schema, projection: Some(vec![0, 1, 2]), limit: None, table_partition_cols: vec![], @@ -344,8 +344,8 @@ mod tests { let avro_exec = AvroExec::new(FileScanConfig { object_store_url, file_groups: vec![vec![meta.into()]], - file_schema: file_schema.clone(), statistics: Statistics::new_with_unbounded_columns(&file_schema), + file_schema, projection, limit: None, table_partition_cols: vec![], @@ -417,8 +417,8 @@ mod tests { projection: Some(vec![0, 1, file_schema.fields().len(), 2]), object_store_url, file_groups: vec![vec![partitioned_file]], - file_schema: file_schema.clone(), statistics: Statistics::new_with_unbounded_columns(&file_schema), + file_schema, limit: None, table_partition_cols: vec![("date".to_owned(), DataType::Utf8)], output_ordering: vec![], 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 c7112d3d207f..e8f7823c0463 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -130,7 +130,7 @@ impl FileScanConfig { let mut table_cols_stats = vec![]; for idx in proj_iter { if idx < self.file_schema.fields().len() { - let field = self.file_schema.field(idx).clone(); + let field = self.file_schema.field(idx); table_fields.push(field.clone()); table_cols_stats.push(self.statistics.column_statistics[idx].clone()) } else { diff --git a/datafusion/core/src/datasource/physical_plan/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index e0101f793b75..12264f41291a 100644 --- a/datafusion/core/src/datasource/physical_plan/file_stream.rs +++ b/datafusion/core/src/datasource/physical_plan/file_stream.rs @@ -660,9 +660,9 @@ mod tests { let config = FileScanConfig { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), - file_schema: file_schema.clone(), - file_groups: vec![file_group], statistics: Statistics::new_with_unbounded_columns(&file_schema), + file_schema, + file_groups: vec![file_group], projection: None, limit: self.limit, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index fd25b70d132e..5f0a86e220c7 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -457,8 +457,8 @@ mod tests { FileScanConfig { object_store_url, file_groups, - file_schema: file_schema.clone(), statistics: Statistics::new_with_unbounded_columns(&file_schema), + file_schema, projection: None, limit: Some(3), table_partition_cols: vec![], @@ -536,8 +536,8 @@ mod tests { FileScanConfig { object_store_url, file_groups, - file_schema: file_schema.clone(), statistics: Statistics::new_with_unbounded_columns(&file_schema), + file_schema, projection: None, limit: Some(3), table_partition_cols: vec![], @@ -584,8 +584,8 @@ mod tests { FileScanConfig { object_store_url, file_groups, - file_schema: file_schema.clone(), statistics: Statistics::new_with_unbounded_columns(&file_schema), + file_schema, projection: Some(vec![0, 2]), limit: None, table_partition_cols: vec![], @@ -637,8 +637,8 @@ mod tests { FileScanConfig { object_store_url, file_groups, - file_schema: file_schema.clone(), statistics: Statistics::new_with_unbounded_columns(&file_schema), + file_schema, projection: Some(vec![3, 0, 2]), limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 22bf4ecd7ec7..b53c04c0df82 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -860,8 +860,8 @@ mod tests { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], - file_schema: file_schema.clone(), statistics: Statistics::new_with_unbounded_columns(&file_schema), + file_schema, projection, limit: None, table_partition_cols: vec![], @@ -1517,8 +1517,8 @@ mod tests { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups, - file_schema: file_schema.clone(), statistics: Statistics::new_with_unbounded_columns(&file_schema), + file_schema, projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 2919e8f48683..005b1c5ba233 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -19,7 +19,6 @@ use crate::arrow::datatypes::{Schema, SchemaRef}; use crate::error::Result; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; use crate::physical_plan::{Accumulator, ColumnStatistics, Statistics}; -use arrow_array::Scalar; use datafusion_common::stats::Sharpness; use datafusion_common::ScalarValue; use futures::Stream; @@ -37,10 +36,8 @@ pub async fn get_statistics_with_limit( ) -> Result<(Vec, Statistics)> { let mut result_files = vec![]; let mut null_counts = vec![Sharpness::Exact(0_usize); file_schema.fields().len()]; - let (mut max_values, mut min_values): ( - Option>>, - Option>>, - ) = (None, None); + let mut max_values: Option>> = None; + let mut min_values: Option>> = None; // The number of rows and the total byte size can be calculated as long as // at least one file has them. If none of the files provide them, then they @@ -54,15 +51,19 @@ pub async fn get_statistics_with_limit( while let Some(res) = all_files.next().await { let (file, file_stats) = res?; result_files.push(file); - num_rows = match (file_stats.num_rows, num_rows.clone()) { - (Sharpness::Absent, rhs) => rhs.to_inexact(), + + // Number of rows, total byte size and null counts are added for each file. + // In case of an absent information or inexact value coming from the file, + // it changes the statistic sharpness to inexact. + num_rows = match (file_stats.num_rows, &num_rows) { + (Sharpness::Absent, _) => num_rows.to_inexact(), (lhs, Sharpness::Absent) => lhs.to_inexact(), - (lhs, rhs) => lhs.add(&rhs), + (lhs, rhs) => lhs.add(rhs), }; - total_byte_size = match (file_stats.total_byte_size, total_byte_size.clone()) { - (Sharpness::Absent, rhs) => rhs.to_inexact(), + total_byte_size = match (file_stats.total_byte_size, &total_byte_size) { + (Sharpness::Absent, _) => total_byte_size.to_inexact(), (lhs, Sharpness::Absent) => lhs.to_inexact(), - (lhs, rhs) => lhs.add(&rhs), + (lhs, rhs) => lhs.add(rhs), }; for (i, cs) in file_stats.column_statistics.iter().enumerate() { @@ -76,9 +77,10 @@ pub async fn get_statistics_with_limit( if let Some(some_max_values) = &mut max_values { for (i, cs) in file_stats.column_statistics.iter().enumerate() { - set_max_if_greater(&mut some_max_values[i], &cs.max_value); + set_max_if_greater(some_max_values, cs.max_value.clone(), i); } } else { + // If it is the first file, we set it directly from the file statistics. let mut new_col_stats_max = vec![]; for cs in file_stats.column_statistics.iter() { new_col_stats_max.push(cs.max_value.clone()); @@ -91,7 +93,7 @@ pub async fn get_statistics_with_limit( if let Some(some_min_values) = &mut min_values { for (i, cs) in file_stats.column_statistics.iter().enumerate() { - set_min_if_lesser(&mut some_min_values[i], &cs.min_value); + set_min_if_lesser(some_min_values, cs.min_value.clone(), i); } } else { let mut new_col_stats_min = vec![]; @@ -137,22 +139,6 @@ pub async fn get_statistics_with_limit( Ok((result_files, statistics)) } -/// It is the [`Sharpness::Exact`] version of `ColumnStatistics::new_with_unbounded_column` function. -pub(crate) fn create_max_min_vec(schema: &Schema) -> Vec> { - schema - .fields() - .iter() - .map(|field| { - let dt = ScalarValue::try_from(field.data_type()); - if let Ok(dt) = dt { - Sharpness::Exact(dt) - } else { - Sharpness::Absent - } - }) - .collect::>() -} - pub(crate) fn create_max_min_accs( schema: &Schema, ) -> (Vec>, Vec>) { @@ -232,59 +218,63 @@ pub(crate) fn get_col_stats( /// If the given value is numerically greater than the original value, /// it set the new max value with the exactness information. fn set_max_if_greater( - max_values: &mut Sharpness, - max_value: &Sharpness, + max_values: &mut [Sharpness], + max_nominee: Sharpness, + index: usize, ) { - if let Sharpness::Exact(max) = &max_value { - match max_values.clone() { - Sharpness::Exact(val) => { - if val < *max { - *max_values = Sharpness::Exact(max.clone()) - } + match (&max_values[index], &max_nominee) { + (Sharpness::Exact(val1), Sharpness::Exact(val2)) => { + if val1 < val2 { + max_values[index] = max_nominee; } - Sharpness::Inexact(val) => { - *max_values = max_values.clone().to_inexact(); - if val < *max { - *max_values = Sharpness::Inexact(max.clone()) - } - } - Sharpness::Absent => *max_values = max_values.clone().to_inexact(), } - } else if let Sharpness::Inexact(max) = &max_value { - if let Some(val) = max_values.get_value() { - if val < *max { - *max_values = Sharpness::Inexact(max.clone()) + (Sharpness::Exact(val1), Sharpness::Inexact(val2)) + | (Sharpness::Inexact(val1), Sharpness::Inexact(val2)) + | (Sharpness::Inexact(val1), Sharpness::Exact(val2)) => { + if val1 < val2 { + max_values[index] = max_nominee.to_inexact() } } + (Sharpness::Inexact(_), Sharpness::Absent) + | (Sharpness::Exact(_), Sharpness::Absent) => { + max_values[index] = max_values[index].clone().to_inexact() + } + (Sharpness::Absent, Sharpness::Exact(_)) + | (Sharpness::Absent, Sharpness::Inexact(_)) => { + max_values[index] = max_nominee.to_inexact() + } + (Sharpness::Absent, Sharpness::Absent) => max_values[index] = Sharpness::Absent, } } /// If the given value is numerically lesser than the original value, /// it set the new min value with the exactness information. fn set_min_if_lesser( - max_values: &mut Sharpness, - max_value: &Sharpness, + min_values: &mut [Sharpness], + min_nominee: Sharpness, + index: usize, ) { - if let Sharpness::Exact(max) = &max_value { - match max_values.clone() { - Sharpness::Exact(val) => { - if val < *max { - *max_values = Sharpness::Exact(max.clone()) - } + match (&min_values[index], &min_nominee) { + (Sharpness::Exact(val1), Sharpness::Exact(val2)) => { + if val1 > val2 { + min_values[index] = min_nominee; } - Sharpness::Inexact(val) => { - *max_values = max_values.clone().to_inexact(); - if val < *max { - *max_values = Sharpness::Inexact(max.clone()) - } - } - Sharpness::Absent => *max_values = max_values.clone().to_inexact(), } - } else if let Sharpness::Inexact(max) = &max_value { - if let Some(val) = max_values.get_value() { - if val < *max { - *max_values = Sharpness::Inexact(max.clone()) + (Sharpness::Exact(val1), Sharpness::Inexact(val2)) + | (Sharpness::Inexact(val1), Sharpness::Inexact(val2)) + | (Sharpness::Inexact(val1), Sharpness::Exact(val2)) => { + if val1 > val2 { + min_values[index] = min_nominee.to_inexact() } } + (Sharpness::Inexact(_), Sharpness::Absent) + | (Sharpness::Exact(_), Sharpness::Absent) => { + min_values[index] = min_values[index].clone().to_inexact() + } + (Sharpness::Absent, Sharpness::Exact(_)) + | (Sharpness::Absent, Sharpness::Inexact(_)) => { + min_values[index] = min_nominee.to_inexact() + } + (Sharpness::Absent, Sharpness::Absent) => min_values[index] = Sharpness::Absent, } } diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index d755e5fd574d..e5586a053d61 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -77,8 +77,8 @@ async fn route_data_access_ops_to_parquet_file_reader_factory() { // just any url that doesn't point to in memory object store object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], - file_schema: file_schema.clone(), statistics: Statistics::new_with_unbounded_columns(&file_schema), + file_schema, projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/tests/parquet/schema_coercion.rs b/datafusion/core/tests/parquet/schema_coercion.rs index 614e2e3bb180..6ba74b817254 100644 --- a/datafusion/core/tests/parquet/schema_coercion.rs +++ b/datafusion/core/tests/parquet/schema_coercion.rs @@ -62,8 +62,8 @@ async fn multi_parquet_coercion() { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], - file_schema: file_schema.clone(), statistics: Statistics::new_with_unbounded_columns(&file_schema), + file_schema, projection: None, limit: None, table_partition_cols: vec![], @@ -126,8 +126,8 @@ async fn multi_parquet_coercion_projection() { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], - file_schema: file_schema.clone(), statistics: Statistics::new_with_unbounded_columns(&file_schema), + file_schema, projection: Some(vec![1, 0, 2]), limit: None, table_partition_cols: vec![], diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 1b2a0b4310c3..e14e0b85620a 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -852,10 +852,8 @@ fn estimate_inner_join_cardinality( }; } - let left_max_distinct = - max_distinct_count(left_stats.num_rows.clone(), left_stat.clone())?; - let right_max_distinct = - max_distinct_count(right_stats.num_rows.clone(), right_stat.clone())?; + let left_max_distinct = max_distinct_count(&left_stats.num_rows, left_stat)?; + let right_max_distinct = max_distinct_count(&right_stats.num_rows, right_stat)?; let max_distinct = left_max_distinct.max(&right_max_distinct); if max_distinct.get_value().is_some() { // Seems like there are a few implementations of this algorithm that implement @@ -892,8 +890,8 @@ fn estimate_inner_join_cardinality( /// has min/max values, then they might be used as a fallback option. Otherwise, /// returns None. fn max_distinct_count( - num_rows: Sharpness, - stats: ColumnStatistics, + num_rows: &Sharpness, + stats: &ColumnStatistics, ) -> Option> { match ( &stats.distinct_count, @@ -901,7 +899,7 @@ fn max_distinct_count( stats.min_value.get_value(), ) { (Sharpness::Exact(_), _, _) | (Sharpness::Inexact(_), _, _) => { - Some(stats.distinct_count) + Some(stats.distinct_count.clone()) } (_, Some(max), Some(min)) => { // Note that float support is intentionally omitted here, since the computation @@ -914,8 +912,9 @@ fn max_distinct_count( let ceiling = num_rows.get_value()? - stats.null_count.get_value().unwrap_or(0); Some( - if let (Sharpness::Exact(_), Sharpness::Exact(_), Sharpness::Exact(_)) = - (num_rows, stats.max_value, stats.min_value) + if num_rows.is_exact().unwrap_or(false) + && stats.max_value.is_exact().unwrap_or(false) + && stats.min_value.is_exact().unwrap_or(false) { Sharpness::Exact(numeric_range.min(ceiling)) } else { From 2d643cad0b8c50ea2180eb2046cf04525780363f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 5 Oct 2023 14:49:04 +0300 Subject: [PATCH 29/53] Simplifications --- datafusion/core/src/datasource/statistics.rs | 98 ++++++++++---------- 1 file changed, 47 insertions(+), 51 deletions(-) diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 005b1c5ba233..8adfdf5ce266 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -23,6 +23,7 @@ use datafusion_common::stats::Sharpness; use datafusion_common::ScalarValue; use futures::Stream; use futures::StreamExt; +use itertools::izip; use super::listing::PartitionedFile; @@ -81,10 +82,13 @@ pub async fn get_statistics_with_limit( } } else { // If it is the first file, we set it directly from the file statistics. - let mut new_col_stats_max = vec![]; - for cs in file_stats.column_statistics.iter() { - new_col_stats_max.push(cs.max_value.clone()); - } + let mut new_col_stats_max = file_stats + .column_statistics + .iter() + .map(|cs| cs.max_value.clone()) + .collect::>(); + // file schema may have additional fields other than each file (such as partition, guaranteed to be at the end) + // Hence, push rest of the fields with information Absent. for _ in 0..file_schema.fields().len() - file_stats.column_statistics.len() { new_col_stats_max.push(Sharpness::Absent) } @@ -96,10 +100,14 @@ pub async fn get_statistics_with_limit( set_min_if_lesser(some_min_values, cs.min_value.clone(), i); } } else { - let mut new_col_stats_min = vec![]; - for cs in file_stats.column_statistics.iter() { - new_col_stats_min.push(cs.min_value.clone()); - } + // If it is the first file, we set it directly from the file statistics. + let mut new_col_stats_min = file_stats + .column_statistics + .iter() + .map(|cs| cs.min_value.clone()) + .collect::>(); + // file schema may have additional fields other than each file (such as partition, guaranteed to be at the end) + // Hence, push rest of the fields with information Absent. for _ in 0..file_schema.fields().len() - file_stats.column_statistics.len() { new_col_stats_min.push(Sharpness::Absent) } @@ -114,31 +122,38 @@ pub async fn get_statistics_with_limit( break; } } + let max_values = max_values.unwrap_or(create_inf_stats(&file_schema)); + let min_values = min_values.unwrap_or(create_inf_stats(&file_schema)); - let column_stats = - get_col_stats_vec(&file_schema, null_counts, max_values, min_values); + let column_stats = get_col_stats_vec(null_counts, max_values, min_values); - let statistics = if all_files.next().await.is_some() { + let mut statistics = Statistics { + num_rows, + total_byte_size, + column_statistics: column_stats, + }; + if all_files.next().await.is_some() { // if we still have files in the stream, it means that the limit kicked // in and the statistic could have been different if we have // processed the files in a different order. - Statistics { - num_rows, - total_byte_size, - column_statistics: column_stats, - } - .make_inexact() - } else { - Statistics { - num_rows, - total_byte_size, - column_statistics: column_stats, - } - }; + statistics = statistics.make_inexact() + } Ok((result_files, statistics)) } +fn create_inf_stats(file_schema: &Schema) -> Vec> { + file_schema + .fields + .iter() + .map(|field| { + ScalarValue::try_from(field.data_type()) + .map(Sharpness::Inexact) + .unwrap_or(Sharpness::Absent) + }) + .collect() +} + pub(crate) fn create_max_min_accs( schema: &Schema, ) -> (Vec>, Vec>) { @@ -156,35 +171,16 @@ pub(crate) fn create_max_min_accs( } pub(crate) fn get_col_stats_vec( - schema: &Schema, null_counts: Vec>, - max_values: Option>>, - min_values: Option>>, + max_values: Vec>, + min_values: Vec>, ) -> Vec { - (0..schema.fields().len()) - .map(|i| { - let max_value = if let Some(some_max_values) = &max_values { - some_max_values[i].clone() - } else { - match ScalarValue::try_from(schema.fields[i].data_type()) { - Ok(dt) => Sharpness::Inexact(dt), - Err(_) => Sharpness::Absent, - } - }; - let min_value = if let Some(some_min_values) = &min_values { - some_min_values[i].clone() - } else { - match ScalarValue::try_from(schema.fields[i].data_type()) { - Ok(dt) => Sharpness::Inexact(dt), - Err(_) => Sharpness::Absent, - } - }; - ColumnStatistics { - null_count: null_counts[i].clone(), - max_value, - min_value, - distinct_count: Sharpness::Absent, - } + izip!(null_counts, max_values, min_values) + .map(|(null_count, max_value, min_value)| ColumnStatistics { + null_count, + max_value, + min_value, + distinct_count: Sharpness::Absent, }) .collect() } From cd3de94bc7ec4318d6f4e2aec78601a153f62a95 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 5 Oct 2023 18:03:28 +0300 Subject: [PATCH 30/53] more accurate row calculations --- datafusion/common/src/stats.rs | 6 +-- .../core/src/datasource/listing/table.rs | 4 +- datafusion/core/src/datasource/statistics.rs | 41 +++++++++----- .../src/physical_optimizer/join_selection.rs | 4 +- datafusion/core/tests/sql/explain_analyze.rs | 2 +- datafusion/physical-expr/src/analysis.rs | 12 ++++- .../physical-expr/src/expressions/negative.rs | 54 +++++-------------- .../src/intervals/interval_aritmetic.rs | 7 +++ datafusion/physical-plan/src/filter.rs | 16 +++--- datafusion/physical-plan/src/joins/utils.rs | 8 +-- .../sqllogictest/test_files/explain.slt | 4 +- 11 files changed, 80 insertions(+), 78 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 36962438818a..3a3a21c368ce 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -33,10 +33,10 @@ pub enum Sharpness { } impl Sharpness { - /// If the information is known somehow, it returns the cloned value. Otherwise, it returns None. - pub fn get_value(&self) -> Option { + /// If the information is known somehow, it returns the value. Otherwise, it returns None. + pub fn get_value(&self) -> Option<&T> { match self { - Sharpness::Exact(val) | Sharpness::Inexact(val) => Some(val.clone()), + Sharpness::Exact(val) | Sharpness::Inexact(val) => Some(val), Sharpness::Absent => None, } } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 30197fc70596..4358552d8860 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1114,8 +1114,8 @@ mod tests { let table = ListingTable::try_new(config)?; let exec = table.scan(&state, None, &[], None).await?; - assert_eq!(exec.statistics()?.num_rows, Sharpness::Inexact(0)); - assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Inexact(0)); + assert_eq!(exec.statistics()?.num_rows, Sharpness::Absent); + assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Absent); Ok(()) } diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 8adfdf5ce266..260b809a4e27 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -44,8 +44,8 @@ pub async fn get_statistics_with_limit( // at least one file has them. If none of the files provide them, then they // will be omitted from the statistics. The missing values will be counted // as zero. - let mut num_rows = Sharpness::Exact(0); - let mut total_byte_size = Sharpness::Exact(0); + let mut num_rows: Option> = None; + let mut total_byte_size: Option> = None; // fusing the stream allows us to call next safely even once it is finished let mut all_files = Box::pin(all_files.fuse()); @@ -56,15 +56,24 @@ pub async fn get_statistics_with_limit( // Number of rows, total byte size and null counts are added for each file. // In case of an absent information or inexact value coming from the file, // it changes the statistic sharpness to inexact. - num_rows = match (file_stats.num_rows, &num_rows) { - (Sharpness::Absent, _) => num_rows.to_inexact(), - (lhs, Sharpness::Absent) => lhs.to_inexact(), - (lhs, rhs) => lhs.add(rhs), + num_rows = if let Some(some_num_rows) = num_rows { + Some(match (file_stats.num_rows, &some_num_rows) { + (Sharpness::Absent, _) => some_num_rows.to_inexact(), + (lhs, Sharpness::Absent) => lhs.to_inexact(), + (lhs, rhs) => lhs.add(rhs), + }) + } else { + Some(file_stats.num_rows) }; - total_byte_size = match (file_stats.total_byte_size, &total_byte_size) { - (Sharpness::Absent, _) => total_byte_size.to_inexact(), - (lhs, Sharpness::Absent) => lhs.to_inexact(), - (lhs, rhs) => lhs.add(rhs), + + total_byte_size = if let Some(some_total_byte_size) = total_byte_size { + Some(match (file_stats.total_byte_size, &some_total_byte_size) { + (Sharpness::Absent, _) => some_total_byte_size.to_inexact(), + (lhs, Sharpness::Absent) => lhs.to_inexact(), + (lhs, rhs) => lhs.add(rhs), + }) + } else { + Some(file_stats.total_byte_size) }; for (i, cs) in file_stats.column_statistics.iter().enumerate() { @@ -118,7 +127,13 @@ pub async fn get_statistics_with_limit( // files. This only applies when we know the number of rows. It also // currently ignores tables that have no statistics regarding the // number of rows. - if num_rows.get_value().unwrap_or(usize::MIN) > limit.unwrap_or(usize::MAX) { + if num_rows + .as_ref() + .unwrap_or(&Sharpness::Absent) + .get_value() + .unwrap_or(&usize::MIN) + > &limit.unwrap_or(usize::MAX) + { break; } } @@ -128,8 +143,8 @@ pub async fn get_statistics_with_limit( let column_stats = get_col_stats_vec(null_counts, max_values, min_values); let mut statistics = Statistics { - num_rows, - total_byte_size, + num_rows: num_rows.unwrap_or(Sharpness::Absent), + total_byte_size: total_byte_size.unwrap_or(Sharpness::Absent), column_statistics: column_stats, }; if all_files.next().await.is_some() { diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 555ca2bc4c4f..4017f5f4f042 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -98,9 +98,9 @@ fn supports_collect_by_size( return false; }; if let Some(size) = stats.total_byte_size.get_value() { - size != 0 && size < collection_size_threshold + *size != 0 && *size < collection_size_threshold } else if let Some(row_count) = stats.num_rows.get_value() { - row_count != 0 && row_count < collection_size_threshold + *row_count != 0 && *row_count < collection_size_threshold } else { false } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 44e1e6d74ffb..00175a9f0406 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -826,5 +826,5 @@ async fn csv_explain_analyze_with_statistics() { .to_string(); // should contain scan statistics - assert_contains!(&formatted, ", statistics=[Rows=Inexact(0), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Utf8(NULL)), min_value: Inexact(Utf8(NULL)), distinct_count: Absent }]]"); + assert_contains!(&formatted, ", statistics=[Rows=Absent, Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Utf8(NULL)), min_value: Inexact(Utf8(NULL)), distinct_count: Absent }]]"); } diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index c571c5f69f04..fdc35a4d6f08 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -92,10 +92,18 @@ impl ExprBoundaries { column: Column::new(&col, index), interval: Interval::new( IntervalBound::new_closed( - stats.min_value.get_value().unwrap_or(ScalarValue::Null), + stats + .min_value + .get_value() + .cloned() + .unwrap_or(ScalarValue::Null), ), IntervalBound::new_closed( - stats.max_value.get_value().unwrap_or(ScalarValue::Null), + stats + .max_value + .get_value() + .cloned() + .unwrap_or(ScalarValue::Null), ), ), distinct_count: stats.distinct_count.clone(), diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index 5957f253155e..906f6709f248 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -21,7 +21,7 @@ use std::any::Any; use std::hash::{Hash, Hasher}; use std::sync::Arc; -use crate::intervals::{Interval, IntervalBound}; +use crate::intervals::Interval; use crate::physical_expr::down_cast_any_ref; use crate::sort_properties::SortProperties; use crate::PhysicalExpr; @@ -111,14 +111,8 @@ impl PhysicalExpr for NegativeExpr { /// Ex: (a, b] => [-b, a) fn evaluate_bounds(&self, children: &[&Interval]) -> Result { Ok(Interval::new( - IntervalBound::new( - children[0].upper.value.arithmetic_negate()?, - children[0].upper.open, - ), - IntervalBound::new( - children[0].lower.value.arithmetic_negate()?, - children[0].lower.open, - ), + children[0].upper.negate()?, + children[0].lower.negate()?, )) } @@ -130,16 +124,9 @@ impl PhysicalExpr for NegativeExpr { children: &[&Interval], ) -> Result>> { let child_interval = children[0]; - let negated_interval = Interval::new( - IntervalBound::new( - interval.upper.value.arithmetic_negate()?, - interval.upper.open, - ), - IntervalBound::new( - interval.lower.value.arithmetic_negate()?, - interval.lower.open, - ), - ); + let negated_interval = + Interval::new(interval.upper.negate()?, interval.lower.negate()?); + Ok(vec![child_interval.intersect(negated_interval)?]) } @@ -187,7 +174,6 @@ mod tests { use arrow::array::*; use arrow::datatypes::*; use arrow_schema::DataType::{Float32, Float64, Int16, Int32, Int64, Int8}; - use datafusion_common::ScalarValue; use datafusion_common::{cast::as_primitive_array, Result}; use paste::paste; @@ -232,14 +218,8 @@ mod tests { let negative_expr = NegativeExpr { arg: Arc::new(Column::new("a", 0)), }; - let child_interval = Interval::new( - IntervalBound::new(ScalarValue::Int64(Some(-2)), true), - IntervalBound::new(ScalarValue::Int64(Some(1)), false), - ); - let negative_expr_interval = Interval::new( - IntervalBound::new(ScalarValue::Int64(Some(-1)), false), - IntervalBound::new(ScalarValue::Int64(Some(2)), true), - ); + let child_interval = Interval::make(Some(-2), Some(1), (true, false)); + let negative_expr_interval = Interval::make(Some(-1), Some(2), (false, true)); assert_eq!( negative_expr.evaluate_bounds(&[&child_interval])?, negative_expr_interval @@ -248,22 +228,14 @@ mod tests { } #[test] - fn propagate_constraints() -> Result<()> { + fn test_propagate_constraints() -> Result<()> { let negative_expr = NegativeExpr { arg: Arc::new(Column::new("a", 0)), }; - let original_child_interval = Interval::new( - IntervalBound::new(ScalarValue::Int64(Some(-2)), false), - IntervalBound::new(ScalarValue::Int64(Some(3)), false), - ); - let negative_expr_interval = Interval::new( - IntervalBound::new(ScalarValue::Int64(Some(0)), true), - IntervalBound::new(ScalarValue::Int64(Some(4)), false), - ); - let after_propagation = vec![Some(Interval::new( - IntervalBound::new(ScalarValue::Int64(Some(-2)), false), - IntervalBound::new(ScalarValue::Int64(Some(0)), true), - ))]; + let original_child_interval = Interval::make(Some(-2), Some(3), (false, false)); + let negative_expr_interval = Interval::make(Some(0), Some(4), (true, false)); + let after_propagation = + vec![Some(Interval::make(Some(-2), Some(0), (false, true)))]; assert_eq!( negative_expr.propagate_constraints( &negative_expr_interval, diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index 3f8977c33c12..b40f6b496a02 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -88,6 +88,13 @@ impl IntervalBound { .map(|value| IntervalBound::new(value, self.open)) } + pub fn negate(&self) -> Result { + self.value.arithmetic_negate().map(|value| IntervalBound { + value, + open: self.open, + }) + } + /// This function adds the given `IntervalBound` to this `IntervalBound`. /// The result is unbounded if either is; otherwise, their values are /// added. The result is closed if both original bounds are closed, or open diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 30f758ae87f3..74ac765a8d1a 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -217,11 +217,11 @@ impl ExecutionPlan for FilterExec { let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); let num_rows = match num_rows.get_value() { - Some(nr) => Sharpness::Inexact((nr as f64 * selectivity).ceil() as usize), + Some(nr) => Sharpness::Inexact((*nr as f64 * selectivity).ceil() as usize), None => Sharpness::Absent, }; let total_byte_size = match total_byte_size.get_value() { - Some(tbs) => Sharpness::Inexact((tbs as f64 * selectivity).ceil() as usize), + Some(tbs) => Sharpness::Inexact((*tbs as f64 * selectivity).ceil() as usize), None => Sharpness::Absent, }; @@ -268,13 +268,13 @@ fn collect_new_statistics( let closed_interval = interval.close_bounds(); ColumnStatistics { null_count: match input_column_stats[idx].null_count.get_value() { - Some(nc) => Sharpness::Inexact(nc), + Some(nc) => Sharpness::Inexact(*nc), None => Sharpness::Absent, }, max_value: Sharpness::Inexact(closed_interval.upper.value), min_value: Sharpness::Inexact(closed_interval.lower.value), distinct_count: match distinct_count.get_value() { - Some(dc) => Sharpness::Inexact(dc), + Some(dc) => Sharpness::Inexact(*dc), None => Sharpness::Absent, }, } @@ -740,11 +740,11 @@ mod tests { let expected_max = expected.max_value.get_value().unwrap(); let eps = ScalarValue::Float32(Some(1e-6)); - assert!(actual_min.sub(&expected_min).unwrap() < eps); - assert!(actual_min.sub(&expected_min).unwrap() < eps); + assert!(actual_min.sub(expected_min).unwrap() < eps); + assert!(actual_min.sub(expected_min).unwrap() < eps); - assert!(actual_max.sub(&expected_max).unwrap() < eps); - assert!(actual_max.sub(&expected_max).unwrap() < eps); + assert!(actual_max.sub(expected_max).unwrap() < eps); + assert!(actual_max.sub(expected_max).unwrap() < eps); } else { assert_eq!(actual, expected); } diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index e14e0b85620a..4df458994614 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -794,7 +794,7 @@ fn estimate_join_cardinality( }; Some(PartialJoinStatistics { - num_rows: cardinality.get_value()?, + num_rows: *cardinality.get_value()?, // We don't do anything specific here, just combine the existing // statistics which might yield subpar results (although it is // true, esp regarding min/max). For a better estimation, we need @@ -910,7 +910,7 @@ fn max_distinct_count( // The number can never be greater than the number of rows we have (minus // the nulls, since they don't count as distinct values). let ceiling = - num_rows.get_value()? - stats.null_count.get_value().unwrap_or(0); + num_rows.get_value()? - stats.null_count.get_value().unwrap_or(&0); Some( if num_rows.is_exact().unwrap_or(false) && stats.max_value.is_exact().unwrap_or(false) @@ -927,8 +927,8 @@ fn max_distinct_count( } /// Return the numeric range between the given min and max values. -fn get_int_range(min: ScalarValue, max: ScalarValue) -> Option { - let delta = &max.sub(&min).ok()?; +fn get_int_range(min: &ScalarValue, max: &ScalarValue) -> Option { + let delta = &max.sub(min).ok()?; match delta { ScalarValue::Int8(Some(delta)) if *delta >= 0 => Some(*delta as usize), ScalarValue::Int16(Some(delta)) if *delta >= 0 => Some(*delta as usize), diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index dc17034a618c..698ec2b644f2 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -269,8 +269,8 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(0), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }]] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Inexact(0), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }]] +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(10), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }]] +--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Absent, Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }]] # Parquet scan with statistics collected statement ok From 872bc40ec7d65819a85be1b7158a6ad69a62a1c1 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 6 Oct 2023 14:53:18 +0300 Subject: [PATCH 31/53] Improve comments --- datafusion-examples/examples/csv_opener.rs | 1 + .../examples/custom_datasource.rs | 17 ++--- datafusion-examples/examples/json_opener.rs | 1 + datafusion/common/src/stats.rs | 65 +++++++++++-------- 4 files changed, 48 insertions(+), 36 deletions(-) diff --git a/datafusion-examples/examples/csv_opener.rs b/datafusion-examples/examples/csv_opener.rs index f745b867b1e2..77203a13230f 100644 --- a/datafusion-examples/examples/csv_opener.rs +++ b/datafusion-examples/examples/csv_opener.rs @@ -30,6 +30,7 @@ use datafusion::{ physical_plan::metrics::ExecutionPlanMetricsSet, test_util::aggr_test_schema, }; + use futures::StreamExt; use object_store::local::LocalFileSystem; diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index c97f48bd5987..e8afaaef2108 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -15,13 +15,17 @@ // specific language governing permissions and limitations // under the License. -use async_trait::async_trait; +use std::any::Any; +use std::collections::{BTreeMap, HashMap}; +use std::fmt::{self, Debug, Formatter}; +use std::sync::{Arc, Mutex}; +use std::time::Duration; + use datafusion::arrow::array::{UInt64Builder, UInt8Builder}; use datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion::arrow::record_batch::RecordBatch; use datafusion::dataframe::DataFrame; -use datafusion::datasource::provider_as_source; -use datafusion::datasource::{TableProvider, TableType}; +use datafusion::datasource::{provider_as_source, TableProvider, TableType}; use datafusion::error::Result; use datafusion::execution::context::{SessionState, TaskContext}; use datafusion::physical_plan::expressions::PhysicalSortExpr; @@ -32,11 +36,8 @@ use datafusion::physical_plan::{ }; use datafusion::prelude::*; use datafusion_expr::{Expr, LogicalPlanBuilder}; -use std::any::Any; -use std::collections::{BTreeMap, HashMap}; -use std::fmt::{self, Debug, Formatter}; -use std::sync::{Arc, Mutex}; -use std::time::Duration; + +use async_trait::async_trait; use tokio::time::timeout; /// This example demonstrates executing a simple query against a custom datasource diff --git a/datafusion-examples/examples/json_opener.rs b/datafusion-examples/examples/json_opener.rs index ee5bd6b50faf..7a43c7434192 100644 --- a/datafusion-examples/examples/json_opener.rs +++ b/datafusion-examples/examples/json_opener.rs @@ -30,6 +30,7 @@ use datafusion::{ physical_plan::metrics::ExecutionPlanMetricsSet, }; use datafusion_common::Statistics; + use futures::StreamExt; use object_store::ObjectStore; diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 3a3a21c368ce..80e5602fcd52 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -17,13 +17,14 @@ //! This module provides data structures to represent statistics +use std::fmt::{self, Debug, Display}; + use crate::ScalarValue; + use arrow::datatypes::{DataType, Schema}; -use core::fmt::Debug; -use std::fmt::{self, Display}; -/// To deal with information whose exactness is not guaranteed, it can be wrapped with [`Sharpness`] -/// to express its reliability, such as in Statistics. +/// To deal with information without exactness guarantees, we wrap it inside a +/// [`Sharpness`] object to express its reliability. See [`Statistics`] for a usage. #[derive(Clone, PartialEq, Eq, Default)] pub enum Sharpness { Exact(T), @@ -33,15 +34,17 @@ pub enum Sharpness { } impl Sharpness { - /// If the information is known somehow, it returns the value. Otherwise, it returns None. + /// If we have some value (exact or inexact), it returns that value. + /// Otherwise, it returns `None`. pub fn get_value(&self) -> Option<&T> { match self { - Sharpness::Exact(val) | Sharpness::Inexact(val) => Some(val), + Sharpness::Exact(value) | Sharpness::Inexact(value) => Some(value), Sharpness::Absent => None, } } - /// Value in the [`Sharpness`] is mapped to the function result wrapped with same exactness state. + /// Transform the value in this [`Sharpness`] object, if one exists, using + /// the given function. Preserves the exactness state. pub fn map(self, f: F) -> Sharpness where F: Fn(T) -> T, @@ -53,8 +56,8 @@ impl Sharpness { } } - /// Returns Some(true) if the information is exact, or Some(false) if not exact. - /// If the information does not even exist, it returns None. + /// Returns `Some(true)` if we have an exact value, `Some(false)` if we + /// have an inexact value, and `None` if there is no value. pub fn is_exact(&self) -> Option { match self { Sharpness::Exact(_) => Some(true), @@ -63,8 +66,9 @@ impl Sharpness { } } - /// Returns the greater one between two exact or inexact values. - /// If one of them is a [`Sharpness::Absent`], it returns [`Sharpness::Absent`]. + /// Returns the maximum of two (possibly inexact) values, conservatively + /// propagating exactness information. If one of the input values is + /// [`Sharpness::Absent`], the result is `Absent` too. pub fn max(&self, other: &Sharpness) -> Sharpness { match (self, other) { (Sharpness::Exact(a), Sharpness::Exact(b)) => { @@ -79,8 +83,9 @@ impl Sharpness { } } - /// Returns the smaller one between two exact or inexact values. - /// If one of them is a [`Sharpness::Absent`], it returns [`Sharpness::Absent`]. + /// Returns the minimum of two (possibly inexact) values, conservatively + /// propagating exactness information. If one of the input values is + /// [`Sharpness::Absent`], the result is `Absent` too. pub fn min(&self, other: &Sharpness) -> Sharpness { match (self, other) { (Sharpness::Exact(a), Sharpness::Exact(b)) => { @@ -95,18 +100,19 @@ impl Sharpness { } } - /// Converts Sharpness from exact to inexact. + /// Demotes the sharpness state to inexact (if present). pub fn to_inexact(self) -> Self { match self { - Sharpness::Exact(val) => Sharpness::Inexact(val), - other => other, + Sharpness::Exact(value) => Sharpness::Inexact(value), + _ => self, } } } impl Sharpness { - /// Calculates the sum of two exact or inexact values in the type of [`usize`]. - /// If one of them is a [`Sharpness::Absent`], it returns [`Sharpness::Absent`]. + /// Calculates the sum of two (possibly inexact) [`usize`] values, + /// conservatively propagating exactness information. If one of the input + /// values is [`Sharpness::Absent`], the result is `Absent` too. pub fn add(&self, other: &Sharpness) -> Sharpness { match (self, other) { (Sharpness::Exact(a), Sharpness::Exact(b)) => Sharpness::Exact(a + b), @@ -117,8 +123,9 @@ impl Sharpness { } } - /// Calculates the difference of two exact or inexact values in the type of [`usize`]. - /// If one of them is a [`Sharpness::Absent`], it returns [`Sharpness::Absent`]. + /// Calculates the difference of two (possibly inexact) [`usize`] values, + /// conservatively propagating exactness information. If one of the input + /// values is [`Sharpness::Absent`], the result is `Absent` too. pub fn sub(&self, other: &Sharpness) -> Sharpness { match (self, other) { (Sharpness::Exact(a), Sharpness::Exact(b)) => Sharpness::Exact(a - b), @@ -129,8 +136,9 @@ impl Sharpness { } } - /// Calculates the multiplication of two exact or inexact values in the type of [`usize`]. - /// If one of them is a [`Sharpness::Absent`], it returns [`Sharpness::Absent`]. + /// Calculates the multiplication of two (possibly inexact) [`usize`] values, + /// conservatively propagating exactness information. If one of the input + /// values is [`Sharpness::Absent`], the result is `Absent` too. pub fn multiply(&self, other: &Sharpness) -> Sharpness { match (self, other) { (Sharpness::Exact(a), Sharpness::Exact(b)) => Sharpness::Exact(a * b), @@ -143,13 +151,14 @@ impl Sharpness { } impl Sharpness { - /// Calculates the sum of two exact or inexact values in the type of [`ScalarValue`]. - /// If one of them is a [`Sharpness::Absent`], it returns [`Sharpness::Absent`]. + /// Calculates the sum of two (possibly inexact) [`ScalarValue`] values, + /// conservatively propagating exactness information. If one of the input + /// values is [`Sharpness::Absent`], the result is `Absent` too. pub fn add(&self, other: &Sharpness) -> Sharpness { match (self, other) { (Sharpness::Exact(a), Sharpness::Exact(b)) => { - if let Ok(res) = a.add(b) { - Sharpness::Exact(res) + if let Ok(result) = a.add(b) { + Sharpness::Exact(result) } else { Sharpness::Absent } @@ -157,8 +166,8 @@ impl Sharpness { (Sharpness::Inexact(a), Sharpness::Exact(b)) | (Sharpness::Exact(a), Sharpness::Inexact(b)) | (Sharpness::Inexact(a), Sharpness::Inexact(b)) => { - if let Ok(res) = a.add(b) { - Sharpness::Inexact(res) + if let Ok(result) = a.add(b) { + Sharpness::Inexact(result) } else { Sharpness::Absent } From f785f9a520b50f1ab5f1f8e981641fdecea44801 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 6 Oct 2023 15:49:52 +0300 Subject: [PATCH 32/53] min-max values are init as absent, not inf --- datafusion/common/src/stats.rs | 15 ++--- .../physical_plan/file_scan_config.rs | 2 +- datafusion/core/src/datasource/statistics.rs | 54 ++++++++++-------- .../src/physical_optimizer/join_selection.rs | 32 +++-------- datafusion/core/tests/path_partition.rs | 20 ++----- datafusion/core/tests/sql/explain_analyze.rs | 2 +- datafusion/physical-expr/src/analysis.rs | 57 +++++++++---------- datafusion/physical-plan/src/common.rs | 15 ++--- datafusion/physical-plan/src/projection.rs | 6 +- .../src/windows/bounded_window_agg_exec.rs | 6 +- .../src/windows/window_agg_exec.rs | 11 +--- .../sqllogictest/test_files/explain.slt | 8 +-- 12 files changed, 92 insertions(+), 136 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 80e5602fcd52..d1cec6c75374 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -21,7 +21,7 @@ use std::fmt::{self, Debug, Display}; use crate::ScalarValue; -use arrow::datatypes::{DataType, Schema}; +use arrow::datatypes::Schema; /// To deal with information without exactness guarantees, we wrap it inside a /// [`Sharpness`] object to express its reliability. See [`Statistics`] for a usage. @@ -228,7 +228,7 @@ impl Statistics { schema .fields() .iter() - .map(|field| ColumnStatistics::new_with_unbounded_column(field.data_type())) + .map(|_| ColumnStatistics::new_unknown()) .collect() } @@ -287,15 +287,12 @@ impl ColumnStatistics { } } - /// Returns the [`ColumnStatistics`] corresponding to the given datatype by assigning infinite bounds. - pub fn new_with_unbounded_column(dt: &DataType) -> ColumnStatistics { - let inf = ScalarValue::try_from(dt.clone()) - .map(Sharpness::Inexact) - .unwrap_or(Sharpness::Absent); + /// Returns a [`ColumnStatistics`] instance having all [`Sharpness::Absent`] parameters. + pub fn new_unknown() -> ColumnStatistics { ColumnStatistics { null_count: Sharpness::Absent, - max_value: inf.clone(), - min_value: inf, + max_value: Sharpness::Absent, + min_value: Sharpness::Absent, distinct_count: Sharpness::Absent, } } 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 e8f7823c0463..5231760d9ab7 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -139,7 +139,7 @@ impl FileScanConfig { let dtype = &self.table_partition_cols[partition_idx].1; table_fields.push(Field::new(name, dtype.to_owned(), false)); // TODO provide accurate stat for partition column (#1186) - table_cols_stats.push(ColumnStatistics::new_with_unbounded_column(dtype)) + table_cols_stats.push(ColumnStatistics::new_unknown()) } } diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 260b809a4e27..dce77094466f 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -36,7 +36,7 @@ pub async fn get_statistics_with_limit( limit: Option, ) -> Result<(Vec, Statistics)> { let mut result_files = vec![]; - let mut null_counts = vec![Sharpness::Exact(0_usize); file_schema.fields().len()]; + let mut null_counts: Option>> = None; let mut max_values: Option>> = None; let mut min_values: Option>> = None; @@ -76,14 +76,29 @@ pub async fn get_statistics_with_limit( Some(file_stats.total_byte_size) }; - for (i, cs) in file_stats.column_statistics.iter().enumerate() { - null_counts[i] = if cs.null_count == Sharpness::Absent { - // Downcast to inexact - null_counts[i].clone().to_inexact() - } else { - null_counts[i].add(&cs.null_count) - }; - } + if let Some(some_null_counts) = &mut null_counts { + for (i, cs) in file_stats.column_statistics.iter().enumerate() { + some_null_counts[i] = if cs.null_count == Sharpness::Absent { + // Downcast to inexact + some_null_counts[i].clone().to_inexact() + } else { + some_null_counts[i].add(&cs.null_count) + }; + } + } else { + // If it is the first file, we set it directly from the file statistics. + let mut new_col_stats_nulls = file_stats + .column_statistics + .iter() + .map(|cs| cs.null_count.clone()) + .collect::>(); + // file schema may have additional fields other than each file (such as partition, guaranteed to be at the end) + // Hence, push rest of the fields with information Absent. + for _ in 0..file_schema.fields().len() - file_stats.column_statistics.len() { + new_col_stats_nulls.push(Sharpness::Absent) + } + null_counts = Some(new_col_stats_nulls); + }; if let Some(some_max_values) = &mut max_values { for (i, cs) in file_stats.column_statistics.iter().enumerate() { @@ -137,8 +152,13 @@ pub async fn get_statistics_with_limit( break; } } - let max_values = max_values.unwrap_or(create_inf_stats(&file_schema)); - let min_values = min_values.unwrap_or(create_inf_stats(&file_schema)); + + let null_counts = + null_counts.unwrap_or(vec![Sharpness::Absent; file_schema.fields().len()]); + let max_values = + max_values.unwrap_or(vec![Sharpness::Absent; file_schema.fields().len()]); + let min_values = + min_values.unwrap_or(vec![Sharpness::Absent; file_schema.fields().len()]); let column_stats = get_col_stats_vec(null_counts, max_values, min_values); @@ -157,18 +177,6 @@ pub async fn get_statistics_with_limit( Ok((result_files, statistics)) } -fn create_inf_stats(file_schema: &Schema) -> Vec> { - file_schema - .fields - .iter() - .map(|field| { - ScalarValue::try_from(field.data_type()) - .map(Sharpness::Inexact) - .unwrap_or(Sharpness::Absent) - }) - .collect() -} - pub(crate) fn create_max_min_accs( schema: &Schema, ) -> (Vec>, Vec>) { diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 4017f5f4f042..5702bf6b4866 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -612,9 +612,7 @@ mod tests_statistical { Statistics { num_rows: Sharpness::Inexact(10), total_byte_size: Sharpness::Inexact(100000), - column_statistics: vec![ColumnStatistics::new_with_unbounded_column( - &DataType::Int32, - )], + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), )); @@ -623,9 +621,7 @@ mod tests_statistical { Statistics { num_rows: Sharpness::Inexact(100000), total_byte_size: Sharpness::Inexact(10), - column_statistics: vec![ColumnStatistics::new_with_unbounded_column( - &DataType::Int32, - )], + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), )); @@ -1010,9 +1006,7 @@ mod tests_statistical { Statistics { num_rows: Sharpness::Inexact(10000000), total_byte_size: Sharpness::Inexact(10000000), - column_statistics: vec![ColumnStatistics::new_with_unbounded_column( - &DataType::Int32, - )], + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), )); @@ -1021,9 +1015,7 @@ mod tests_statistical { Statistics { num_rows: Sharpness::Inexact(10), total_byte_size: Sharpness::Inexact(10), - column_statistics: vec![ColumnStatistics::new_with_unbounded_column( - &DataType::Int32, - )], + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), )); @@ -1032,9 +1024,7 @@ mod tests_statistical { Statistics { num_rows: Sharpness::Absent, total_byte_size: Sharpness::Absent, - column_statistics: vec![ColumnStatistics::new_with_unbounded_column( - &DataType::Int32, - )], + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), )); @@ -1094,9 +1084,7 @@ mod tests_statistical { Statistics { num_rows: Sharpness::Inexact(10000000), total_byte_size: Sharpness::Inexact(10000000), - column_statistics: vec![ColumnStatistics::new_with_unbounded_column( - &DataType::Int32, - )], + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("big_col1", DataType::Int32, false)]), )); @@ -1105,9 +1093,7 @@ mod tests_statistical { Statistics { num_rows: Sharpness::Inexact(20000000), total_byte_size: Sharpness::Inexact(20000000), - column_statistics: vec![ColumnStatistics::new_with_unbounded_column( - &DataType::Int32, - )], + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("big_col2", DataType::Int32, false)]), )); @@ -1116,9 +1102,7 @@ mod tests_statistical { Statistics { num_rows: Sharpness::Absent, total_byte_size: Sharpness::Absent, - column_statistics: vec![ColumnStatistics::new_with_unbounded_column( - &DataType::Int32, - )], + column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), )); diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index 34efc59216a7..62378cd9a509 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -467,18 +467,9 @@ async fn parquet_statistics() -> Result<()> { // stats for the first col are read from the parquet file assert_eq!(stat_cols[0].null_count, Sharpness::Exact(3)); // TODO assert partition column (1,2,3) stats once implemented (#1186) - assert_eq!( - stat_cols[1], - ColumnStatistics::new_with_unbounded_column(schema.field(1).data_type()), - ); - assert_eq!( - stat_cols[2], - ColumnStatistics::new_with_unbounded_column(schema.field(2).data_type()), - ); - assert_eq!( - stat_cols[3], - ColumnStatistics::new_with_unbounded_column(schema.field(3).data_type()), - ); + assert_eq!(stat_cols[1], ColumnStatistics::new_unknown(),); + assert_eq!(stat_cols[2], ColumnStatistics::new_unknown(),); + assert_eq!(stat_cols[3], ColumnStatistics::new_unknown(),); //// WITH PROJECTION //// let dataframe = ctx.sql("SELECT mycol, day FROM t WHERE day='28'").await?; @@ -491,10 +482,7 @@ async fn parquet_statistics() -> Result<()> { // stats for the first col are read from the parquet file assert_eq!(stat_cols[0].null_count, Sharpness::Exact(1)); // TODO assert partition column stats once implemented (#1186) - assert_eq!( - stat_cols[1], - ColumnStatistics::new_with_unbounded_column(schema.field(1).data_type()), - ); + assert_eq!(stat_cols[1], ColumnStatistics::new_unknown(),); Ok(()) } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 00175a9f0406..f6e305c7e3fd 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -826,5 +826,5 @@ async fn csv_explain_analyze_with_statistics() { .to_string(); // should contain scan statistics - assert_contains!(&formatted, ", statistics=[Rows=Absent, Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Utf8(NULL)), min_value: Inexact(Utf8(NULL)), distinct_count: Absent }]]"); + assert_contains!(&formatted, ", statistics=[Rows=Absent, Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }]]"); } diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index fdc35a4d6f08..15ce6e15baff 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -64,11 +64,32 @@ impl AnalysisContext { ) -> Self { let mut column_boundaries = vec![]; for (idx, stats) in statistics.iter().enumerate() { - column_boundaries.push(ExprBoundaries::from_column( - stats, - input_schema.fields()[idx].name().clone(), - idx, - )); + let field: &Arc = &input_schema.fields()[idx]; + if let Ok(inf_field) = ScalarValue::try_from(field.data_type()) { + let interval = Interval::new( + IntervalBound::new_closed( + stats + .min_value + .get_value() + .cloned() + .unwrap_or(inf_field.clone()), + ), + IntervalBound::new_closed( + stats.max_value.get_value().cloned().unwrap_or(inf_field), + ), + ); + let column = Column::new(input_schema.fields()[idx].name(), idx); + column_boundaries.push(ExprBoundaries { + column, + interval, + distinct_count: stats.distinct_count.clone(), + }); + } else { + return AnalysisContext { + boundaries: None, + selectivity: None, + }; + } } Self::new(column_boundaries) } @@ -85,32 +106,6 @@ pub struct ExprBoundaries { pub distinct_count: Sharpness, } -impl ExprBoundaries { - /// Create a new `ExprBoundaries` object from column level statistics. - pub fn from_column(stats: &ColumnStatistics, col: String, index: usize) -> Self { - Self { - column: Column::new(&col, index), - interval: Interval::new( - IntervalBound::new_closed( - stats - .min_value - .get_value() - .cloned() - .unwrap_or(ScalarValue::Null), - ), - IntervalBound::new_closed( - stats - .max_value - .get_value() - .cloned() - .unwrap_or(ScalarValue::Null), - ), - ), - distinct_count: stats.distinct_count.clone(), - } - } -} - /// Attempts to refine column boundaries and compute a selectivity value. /// /// The function accepts boundaries of the input columns in the `context` parameter. diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index 60b8b9346a52..aa07189e6ff3 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -147,11 +147,7 @@ pub fn compute_record_batch_statistics( None => (0..schema.fields().len()).collect(), }; - let fields = schema.fields(); - let mut column_statistics = projection - .iter() - .map(|&idx| ColumnStatistics::new_with_unbounded_column(fields[idx].data_type())) - .collect::>(); + let mut column_statistics = vec![ColumnStatistics::new_unknown(); projection.len()]; for partition in batches.iter() { for batch in partition { @@ -388,7 +384,6 @@ mod tests { datatypes::{DataType, Field, Schema}, record_batch::RecordBatch, }; - use datafusion_common::ScalarValue; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{col, Column}; @@ -703,14 +698,14 @@ mod tests { column_statistics: vec![ ColumnStatistics { distinct_count: Sharpness::Absent, - max_value: Sharpness::Inexact(ScalarValue::Float32(None)), - min_value: Sharpness::Inexact(ScalarValue::Float32(None)), + max_value: Sharpness::Absent, + min_value: Sharpness::Absent, null_count: Sharpness::Exact(0), }, ColumnStatistics { distinct_count: Sharpness::Absent, - max_value: Sharpness::Inexact(ScalarValue::Float64(None)), - min_value: Sharpness::Inexact(ScalarValue::Float64(None)), + max_value: Sharpness::Absent, + min_value: Sharpness::Absent, null_count: Sharpness::Exact(0), }, ], diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index f2f68262f4b1..59b988251c05 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -404,15 +404,13 @@ fn stats_projection( .clone() .into_iter() .enumerate() - .map(|(index, e)| { + .map(|(_index, e)| { if let Some(col) = e.as_any().downcast_ref::() { stats.column_statistics[col.index()].clone() } else { // TODO stats: estimate more statistics from expressions // (expressions should compute their statistics themselves) - ColumnStatistics::new_with_unbounded_column( - schema.field(index).data_type(), - ) + ColumnStatistics::new_unknown() } }) .collect(); 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 54b9e1605a76..33d6aea8f635 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -340,10 +340,8 @@ impl ExecutionPlan for BoundedWindowAggExec { let mut column_statistics = Vec::with_capacity(win_cols + input_cols); // copy stats of the input to the beginning of the schema. column_statistics.extend(input_stat.column_statistics); - for index in 0..win_cols { - column_statistics.push(ColumnStatistics::new_with_unbounded_column( - self.schema().field(index + input_cols).data_type(), - )) + for _ in 0..win_cols { + column_statistics.push(ColumnStatistics::new_unknown()) } Ok(Statistics { num_rows: input_stat.num_rows, diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index c78c6e67edbc..e33d0c9f609c 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -265,15 +265,8 @@ impl ExecutionPlan for WindowAggExec { let mut column_statistics = Vec::with_capacity(win_cols + input_cols); // copy stats of the input to the beginning of the schema. column_statistics.extend(input_stat.column_statistics); - for index in 0..win_cols { - column_statistics.push(ColumnStatistics::new_with_unbounded_column( - self.schema().field(index + input_cols).data_type(), - )) - } - for index in 0..win_cols { - column_statistics.push(ColumnStatistics::new_with_unbounded_column( - self.schema().field(index + input_cols).data_type(), - )) + for _ in 0..win_cols { + column_statistics.push(ColumnStatistics::new_unknown()) } Ok(Statistics { num_rows: input_stat.num_rows, diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 698ec2b644f2..c9b3d3936d09 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -269,8 +269,8 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(10), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }]] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Absent, Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }]] +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(10), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }]] +--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Absent, Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }]] # Parquet scan with statistics collected statement ok @@ -283,8 +283,8 @@ query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Boolean(NULL)), min_value: Inexact(Boolean(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int64(NULL)), min_value: Inexact(Int64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Float32(NULL)), min_value: Inexact(Float32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Float64(NULL)), min_value: Inexact(Float64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Binary(NULL)), min_value: Inexact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Binary(NULL)), min_value: Inexact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(TimestampNanosecond(NULL, None)), min_value: Inexact(TimestampNanosecond(NULL, None)), distinct_count: Absent }]] ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Boolean(NULL)), min_value: Inexact(Boolean(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int32(NULL)), min_value: Inexact(Int32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Int64(NULL)), min_value: Inexact(Int64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Float32(NULL)), min_value: Inexact(Float32(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Float64(NULL)), min_value: Inexact(Float64(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Binary(NULL)), min_value: Inexact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(Binary(NULL)), min_value: Inexact(Binary(NULL)), distinct_count: Absent }, ColumnStatistics { null_count: Inexact(0), max_value: Inexact(TimestampNanosecond(NULL, None)), min_value: Inexact(TimestampNanosecond(NULL, None)), distinct_count: Absent }]] +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }]] +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }]] statement ok set datafusion.execution.collect_statistics = false; From 429b5af5d8c331d8c5a6c0534a287b1a70c818a7 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 9 Oct 2023 16:56:03 +0300 Subject: [PATCH 33/53] fix after merge --- datafusion/core/tests/parquet/file_statistics.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index b1489b42e9dc..d88343824d2f 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -60,7 +60,7 @@ async fn load_table_stats_with_session_level_cache() { exec1.statistics().unwrap().total_byte_size, Sharpness::Exact(671) ); - assert_eq!(get_cache_size(&state1), 1); + assert_eq!(get_static_cache_size(&state1), 1); //Session 2 first time list files //check session 1 cache result not show in session 2 @@ -71,7 +71,7 @@ async fn load_table_stats_with_session_level_cache() { exec2.statistics().unwrap().total_byte_size, Sharpness::Exact(671) ); - assert_eq!(get_cache_size(&state2), 1); + assert_eq!(get_static_cache_size(&state2), 1); //Session 1 second time list files //check session 1 cache result not show in session 2 From 5e0df3330f48742e6ccf4148c091ee9d9491edd0 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 10 Oct 2023 15:06:12 +0300 Subject: [PATCH 34/53] Review Part 1 --- datafusion-examples/examples/csv_opener.rs | 2 +- .../examples/custom_datasource.rs | 2 +- datafusion-examples/examples/json_opener.rs | 2 +- datafusion/common/src/stats.rs | 13 ++- .../core/src/datasource/file_format/arrow.rs | 14 ++-- .../core/src/datasource/file_format/avro.rs | 2 +- .../core/src/datasource/file_format/csv.rs | 30 +++---- .../core/src/datasource/file_format/json.rs | 53 +++++------- .../src/datasource/file_format/parquet.rs | 2 +- .../core/src/datasource/listing/table.rs | 8 +- .../datasource/physical_plan/arrow_file.rs | 8 +- .../core/src/datasource/physical_plan/avro.rs | 18 ++-- .../core/src/datasource/physical_plan/csv.rs | 20 ++--- .../physical_plan/file_scan_config.rs | 23 +++-- .../datasource/physical_plan/file_stream.rs | 2 +- .../core/src/datasource/physical_plan/json.rs | 23 ++--- .../core/src/datasource/physical_plan/mod.rs | 62 +++++++------- .../src/datasource/physical_plan/parquet.rs | 28 +++---- datafusion/core/src/datasource/statistics.rs | 84 +++++++++---------- .../aggregate_statistics.rs | 29 ++++--- .../combine_partial_final_agg.rs | 16 ++-- .../enforce_distribution.rs | 20 +++-- .../src/physical_optimizer/join_selection.rs | 11 ++- .../physical_optimizer/pipeline_checker.rs | 6 +- .../replace_with_order_preserving_variants.rs | 2 +- .../core/src/physical_optimizer/test_utils.rs | 4 +- datafusion/core/src/physical_planner.rs | 53 ++++++------ datafusion/core/src/test/mod.rs | 31 +++---- datafusion/core/src/test_util/mod.rs | 9 +- datafusion/core/src/test_util/parquet.rs | 4 +- datafusion/core/tests/custom_sources.rs | 25 +++--- .../provider_filter_pushdown.rs | 2 +- .../tests/custom_sources_cases/statistics.rs | 4 +- .../core/tests/parquet/custom_reader.rs | 12 +-- .../core/tests/parquet/file_statistics.rs | 6 +- datafusion/core/tests/parquet/page_pruning.rs | 4 +- .../core/tests/parquet/schema_coercion.rs | 11 +-- datafusion/core/tests/path_partition.rs | 11 +-- datafusion/core/tests/sql/explain_analyze.rs | 1 + .../tests/user_defined/user_defined_plan.rs | 14 ++-- datafusion/execution/src/cache/cache_unit.rs | 9 +- datafusion/physical-expr/src/analysis.rs | 7 +- .../physical-expr/src/expressions/negative.rs | 3 +- .../physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/analyze.rs | 2 +- datafusion/physical-plan/src/explain.rs | 2 +- datafusion/physical-plan/src/filter.rs | 6 +- datafusion/physical-plan/src/insert.rs | 2 +- .../src/joins/symmetric_hash_join.rs | 2 +- datafusion/physical-plan/src/joins/utils.rs | 5 +- datafusion/physical-plan/src/limit.rs | 4 +- datafusion/physical-plan/src/streaming.rs | 2 +- datafusion/physical-plan/src/test/exec.rs | 2 +- datafusion/physical-plan/src/union.rs | 4 +- datafusion/physical-plan/src/unnest.rs | 2 +- .../tests/cases/roundtrip_physical_plan.rs | 6 +- .../substrait/src/physical_plan/consumer.rs | 4 +- .../tests/cases/roundtrip_physical_plan.rs | 2 +- 58 files changed, 369 insertions(+), 368 deletions(-) diff --git a/datafusion-examples/examples/csv_opener.rs b/datafusion-examples/examples/csv_opener.rs index 77203a13230f..15fb07ded481 100644 --- a/datafusion-examples/examples/csv_opener.rs +++ b/datafusion-examples/examples/csv_opener.rs @@ -62,7 +62,7 @@ async fn main() -> Result<()> { object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new(path.display().to_string(), 10)]], - statistics: Statistics::new_with_unbounded_columns(&schema), + statistics: Statistics::new_unknown(&schema), projection: Some(vec![12, 0]), limit: Some(5), table_partition_cols: vec![], diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index e8afaaef2108..dd36665a9344 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -272,6 +272,6 @@ impl ExecutionPlan for CustomExec { } fn statistics(&self) -> Result { - Ok(Statistics::new_with_unbounded_columns(&self.schema())) + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion-examples/examples/json_opener.rs b/datafusion-examples/examples/json_opener.rs index 7a43c7434192..1a3dbe57be75 100644 --- a/datafusion-examples/examples/json_opener.rs +++ b/datafusion-examples/examples/json_opener.rs @@ -65,7 +65,7 @@ async fn main() -> Result<()> { object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new(path.to_string(), 10)]], - statistics: Statistics::new_with_unbounded_columns(&schema), + statistics: Statistics::new_unknown(&schema), projection: Some(vec![1, 0]), limit: Some(5), table_partition_cols: vec![], diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index d1cec6c75374..47aa60bac6f1 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -212,19 +212,18 @@ pub struct Statistics { } impl Statistics { - /// Returns a [`Statistics`] instance corresponding to the given schema by assigning infinite - /// bounds to each column in the schema. This is useful when the input statistics are not - /// known to give an opportunity to the current executor to shrink the bounds of some columns. - pub fn new_with_unbounded_columns(schema: &Schema) -> Self { + /// Returns a [`Statistics`] instance for the given schema by assigning + /// unknown statistics to each column in the schema. + pub fn new_unknown(schema: &Schema) -> Self { Self { num_rows: Sharpness::Absent, total_byte_size: Sharpness::Absent, - column_statistics: Statistics::unbounded_column_statistics(schema), + column_statistics: Statistics::unknown_column(schema), } } - /// Returns an unbounded ColumnStatistics for each field in the schema. - pub fn unbounded_column_statistics(schema: &Schema) -> Vec { + /// Returns an unbounded `ColumnStatistics` for each field in the schema. + pub fn unknown_column(schema: &Schema) -> Vec { schema .fields() .iter() diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 04937660d65a..16ae4411d1bf 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -19,20 +19,24 @@ //! //! Works with files following the [Arrow IPC format](https://arrow.apache.org/docs/format/Columnar.html#ipc-file-format) +use std::any::Any; +use std::io::{Read, Seek}; +use std::sync::Arc; + use crate::datasource::file_format::FileFormat; use crate::datasource::physical_plan::{ArrowExec, FileScanConfig}; use crate::error::Result; use crate::execution::context::SessionState; use crate::physical_plan::ExecutionPlan; + use arrow::ipc::reader::FileReader; use arrow_schema::{Schema, SchemaRef}; -use async_trait::async_trait; + use datafusion_common::{FileType, Statistics}; use datafusion_physical_expr::PhysicalExpr; + +use async_trait::async_trait; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; -use std::any::Any; -use std::io::{Read, Seek}; -use std::sync::Arc; /// Arrow `FileFormat` implementation. #[derive(Default, Debug)] @@ -77,7 +81,7 @@ impl FileFormat for ArrowFormat { table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::new_with_unbounded_columns(&table_schema)) + Ok(Statistics::new_unknown(&table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index 0cee676f7c31..a24a28ad6fdd 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -77,7 +77,7 @@ impl FileFormat for AvroFormat { table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::new_with_unbounded_columns(&table_schema)) + Ok(Statistics::new_unknown(&table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index e7a88714f9f8..70caad43ad37 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -23,20 +23,6 @@ use std::fmt; use std::fmt::Debug; use std::sync::Arc; -use arrow::csv::WriterBuilder; -use arrow::datatypes::{DataType, Field, Fields, Schema}; -use arrow::{self, datatypes::SchemaRef}; -use arrow_array::RecordBatch; -use datafusion_common::{exec_err, not_impl_err, DataFusionError, FileType}; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; - -use async_trait::async_trait; -use bytes::{Buf, Bytes}; -use futures::stream::BoxStream; -use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; -use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; - use super::{FileFormat, DEFAULT_SCHEMA_INFER_MAX_RECORD}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::{ @@ -50,6 +36,20 @@ use crate::execution::context::SessionState; use crate::physical_plan::insert::{DataSink, FileSinkExec}; use crate::physical_plan::{DisplayAs, DisplayFormatType, Statistics}; use crate::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; + +use arrow::csv::WriterBuilder; +use arrow::datatypes::{DataType, Field, Fields, Schema}; +use arrow::{self, datatypes::SchemaRef}; +use arrow_array::RecordBatch; +use datafusion_common::{exec_err, not_impl_err, DataFusionError, FileType}; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; + +use async_trait::async_trait; +use bytes::{Buf, Bytes}; +use futures::stream::BoxStream; +use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; +use object_store::{delimited::newline_delimited_stream, ObjectMeta, ObjectStore}; use rand::distributions::{Alphanumeric, DistString}; /// Character Separated Value `FileFormat` implementation. @@ -238,7 +238,7 @@ impl FileFormat for CsvFormat { table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::new_with_unbounded_columns(&table_schema)) + Ok(Statistics::new_unknown(&table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index c66111532f0b..4ec6fd8716b0 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -18,50 +18,37 @@ //! Line delimited JSON format abstractions use std::any::Any; - -use bytes::Bytes; -use datafusion_common::not_impl_err; -use datafusion_common::DataFusionError; -use datafusion_common::FileType; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalSortRequirement; -use rand::distributions::Alphanumeric; -use rand::distributions::DistString; use std::fmt; use std::fmt::Debug; use std::io::BufReader; use std::sync::Arc; -use arrow::datatypes::Schema; -use arrow::datatypes::SchemaRef; -use arrow::json; -use arrow::json::reader::infer_json_schema_from_iterator; -use arrow::json::reader::ValueIter; -use arrow_array::RecordBatch; -use async_trait::async_trait; -use bytes::Buf; - -use datafusion_physical_expr::PhysicalExpr; -use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; - -use crate::datasource::physical_plan::FileGroupDisplay; -use crate::physical_plan::insert::DataSink; -use crate::physical_plan::insert::FileSinkExec; -use crate::physical_plan::SendableRecordBatchStream; -use crate::physical_plan::{DisplayAs, DisplayFormatType, Statistics}; - -use super::FileFormat; -use super::FileScanConfig; +use super::{FileFormat, FileScanConfig}; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::file_format::write::{ create_writer, stateless_serialize_and_write_files, BatchSerializer, FileWriterMode, }; use crate::datasource::file_format::DEFAULT_SCHEMA_INFER_MAX_RECORD; -use crate::datasource::physical_plan::FileSinkConfig; -use crate::datasource::physical_plan::NdJsonExec; +use crate::datasource::physical_plan::{FileGroupDisplay, FileSinkConfig, NdJsonExec}; use crate::error::Result; use crate::execution::context::SessionState; -use crate::physical_plan::ExecutionPlan; +use crate::physical_plan::insert::{DataSink, FileSinkExec}; +use crate::physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, Statistics, +}; + +use arrow::datatypes::{Schema, SchemaRef}; +use arrow::json; +use arrow::json::reader::{infer_json_schema_from_iterator, ValueIter}; +use arrow_array::RecordBatch; +use datafusion_common::{not_impl_err, DataFusionError, FileType}; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; + +use async_trait::async_trait; +use bytes::{Buf, Bytes}; +use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; +use rand::distributions::{Alphanumeric, DistString}; /// New line delimited JSON `FileFormat` implementation. #[derive(Debug)] @@ -156,7 +143,7 @@ impl FileFormat for JsonFormat { table_schema: SchemaRef, _object: &ObjectMeta, ) -> Result { - Ok(Statistics::new_with_unbounded_columns(&table_schema)) + Ok(Statistics::new_unknown(&table_schema)) } async fn create_physical_plan( diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 3655b6dc6012..ab08075cade3 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -565,7 +565,7 @@ async fn fetch_statistics( let column_stats = if has_statistics { get_col_stats(&table_schema, null_counts, &mut max_values, &mut min_values) } else { - Statistics::unbounded_column_statistics(&table_schema) + Statistics::unknown_column(&table_schema) }; let statistics = Statistics { diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 9357094c0c0a..83e62b685f32 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -962,10 +962,7 @@ impl ListingTable { let store = if let Some(url) = self.table_paths.get(0) { ctx.runtime_env().object_store(url)? } else { - return Ok(( - vec![], - Statistics::new_with_unbounded_columns(&self.file_schema), - )); + return Ok((vec![], Statistics::new_unknown(&self.file_schema))); }; // list files (with partitions) let file_list = future::try_join_all(self.table_paths.iter().map(|table_path| { @@ -984,8 +981,7 @@ impl ListingTable { let files = file_list .map(|part_file| async { let part_file = part_file?; - let mut statistics_result = - Statistics::new_with_unbounded_columns(&self.file_schema); + let mut statistics_result = Statistics::new_unknown(&self.file_schema); if self.options.collect_stat { let statistics_cache = self.collected_statistics.clone(); match statistics_cache.get_with_extra( diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index ef4e3d418ef7..e00e8aea0a04 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -16,6 +16,10 @@ // under the License. //! Execution plan for reading Arrow files + +use std::any::Any; +use std::sync::Arc; + use crate::datasource::physical_plan::{ FileMeta, FileOpenFuture, FileOpener, FileScanConfig, }; @@ -24,6 +28,7 @@ use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, }; + use arrow_schema::SchemaRef; use datafusion_common::Statistics; use datafusion_execution::TaskContext; @@ -31,10 +36,9 @@ use datafusion_physical_expr::{ ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, PhysicalSortExpr, }; + use futures::StreamExt; use object_store::{GetResultPayload, ObjectStore}; -use std::any::Any; -use std::sync::Arc; /// Execution plan for scanning Arrow data source #[derive(Debug, Clone)] diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 4969b214ae39..f08bc9b8df20 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -16,6 +16,11 @@ // under the License. //! Execution plan for reading line-delimited Avro files + +use std::any::Any; +use std::sync::Arc; + +use super::FileScanConfig; use crate::error::Result; use crate::physical_plan::expressions::PhysicalSortExpr; use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; @@ -23,18 +28,13 @@ use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; -use datafusion_execution::TaskContext; use arrow::datatypes::SchemaRef; +use datafusion_execution::TaskContext; use datafusion_physical_expr::{ ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, }; -use std::any::Any; -use std::sync::Arc; - -use super::FileScanConfig; - /// Execution plan for scanning Avro data source #[derive(Debug, Clone)] #[allow(dead_code)] @@ -272,7 +272,7 @@ mod tests { let avro_exec = AvroExec::new(FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![vec![meta.into()]], - statistics: Statistics::new_with_unbounded_columns(&file_schema), + statistics: Statistics::new_unknown(&file_schema), file_schema, projection: Some(vec![0, 1, 2]), limit: None, @@ -344,7 +344,7 @@ mod tests { let avro_exec = AvroExec::new(FileScanConfig { object_store_url, file_groups: vec![vec![meta.into()]], - statistics: Statistics::new_with_unbounded_columns(&file_schema), + statistics: Statistics::new_unknown(&file_schema), file_schema, projection, limit: None, @@ -417,7 +417,7 @@ mod tests { projection: Some(vec![0, 1, file_schema.fields().len(), 2]), object_store_url, file_groups: vec![vec![partitioned_file]], - statistics: Statistics::new_with_unbounded_columns(&file_schema), + statistics: Statistics::new_unknown(&file_schema), file_schema, limit: None, table_partition_cols: vec![("date".to_owned(), DataType::Utf8)], diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index e62f145e9720..f3b2fa9de7a9 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -17,6 +17,13 @@ //! Execution plan for reading CSV files +use std::any::Any; +use std::io::{Read, Seek, SeekFrom}; +use std::ops::Range; +use std::sync::Arc; +use std::task::Poll; + +use super::FileScanConfig; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::{FileRange, ListingTableUrl}; use crate::datasource::physical_plan::file_stream::{ @@ -30,25 +37,18 @@ use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; + use arrow::csv; use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; use datafusion_physical_expr::{ ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, }; -use tokio::io::AsyncWriteExt; - -use super::FileScanConfig; use bytes::{Buf, Bytes}; -use futures::ready; -use futures::{StreamExt, TryStreamExt}; +use futures::{ready, StreamExt, TryStreamExt}; use object_store::{GetOptions, GetResultPayload, ObjectStore}; -use std::any::Any; -use std::io::{Read, Seek, SeekFrom}; -use std::ops::Range; -use std::sync::Arc; -use std::task::Poll; +use tokio::io::AsyncWriteExt; use tokio::task::JoinSet; /// Execution plan for scanning a CSV file 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 5231760d9ab7..8e4782a59e94 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -18,6 +18,12 @@ //! [`FileScanConfig`] to configure scanning of possibly partitioned //! file sources. +use std::{ + borrow::Cow, cmp::min, collections::HashMap, fmt::Debug, marker::PhantomData, + sync::Arc, vec, +}; + +use super::get_projected_output_ordering; use crate::datasource::{ listing::{FileRange, PartitionedFile}, object_store::ObjectStoreUrl, @@ -32,18 +38,12 @@ use arrow::buffer::Buffer; use arrow::datatypes::{ArrowNativeType, UInt16Type}; use arrow_array::{ArrayRef, DictionaryArray, RecordBatch}; use arrow_schema::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::{exec_err, stats::Sharpness}; -use datafusion_common::{ColumnStatistics, Statistics}; +use datafusion_common::stats::Sharpness; +use datafusion_common::{exec_err, ColumnStatistics, Statistics}; use datafusion_physical_expr::LexOrdering; use itertools::Itertools; use log::warn; -use std::{ - borrow::Cow, cmp::min, collections::HashMap, fmt::Debug, marker::PhantomData, - sync::Arc, vec, -}; - -use super::get_projected_output_ordering; /// Convert type to a type suitable for use as a [`ListingTable`] /// partition column. Returns `Dictionary(UInt16, val_type)`, which is @@ -135,8 +135,7 @@ impl FileScanConfig { table_cols_stats.push(self.statistics.column_statistics[idx].clone()) } else { let partition_idx = idx - self.file_schema.fields().len(); - let name = &self.table_partition_cols[partition_idx].0; - let dtype = &self.table_partition_cols[partition_idx].1; + let (name, dtype) = &self.table_partition_cols[partition_idx]; table_fields.push(Field::new(name, dtype.to_owned(), false)); // TODO provide accurate stat for partition column (#1186) table_cols_stats.push(ColumnStatistics::new_unknown()) @@ -501,7 +500,7 @@ mod tests { let conf = config_for_projection( Arc::clone(&file_schema), None, - Statistics::new_with_unbounded_columns(&file_schema), + Statistics::new_unknown(&file_schema), vec![( "date".to_owned(), wrap_partition_type_in_dict(DataType::Utf8), @@ -602,7 +601,7 @@ mod tests { file_batch.schema().fields().len(), file_batch.schema().fields().len() + 2, ]), - Statistics::new_with_unbounded_columns(&file_batch.schema()), + Statistics::new_unknown(&file_batch.schema()), partition_cols.clone(), ); let (proj_schema, ..) = conf.project(); diff --git a/datafusion/core/src/datasource/physical_plan/file_stream.rs b/datafusion/core/src/datasource/physical_plan/file_stream.rs index 12264f41291a..487d17d9c1c3 100644 --- a/datafusion/core/src/datasource/physical_plan/file_stream.rs +++ b/datafusion/core/src/datasource/physical_plan/file_stream.rs @@ -660,7 +660,7 @@ mod tests { let config = FileScanConfig { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), - statistics: Statistics::new_with_unbounded_columns(&file_schema), + statistics: Statistics::new_unknown(&file_schema), file_schema, file_groups: vec![file_group], projection: None, diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 97cd6b71729c..1ba8e47a523c 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -16,6 +16,13 @@ // under the License. //! Execution plan for reading line-delimited JSON files + +use std::any::Any; +use std::io::BufReader; +use std::sync::Arc; +use std::task::Poll; + +use super::FileScanConfig; use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::ListingTableUrl; use crate::datasource::physical_plan::file_stream::{ @@ -29,10 +36,10 @@ use crate::physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; -use datafusion_execution::TaskContext; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; +use datafusion_execution::TaskContext; use datafusion_physical_expr::{ ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, }; @@ -41,15 +48,9 @@ use bytes::{Buf, Bytes}; use futures::{ready, stream, StreamExt, TryStreamExt}; use object_store; use object_store::{GetResultPayload, ObjectStore}; -use std::any::Any; -use std::io::BufReader; -use std::sync::Arc; -use std::task::Poll; use tokio::io::AsyncWriteExt; use tokio::task::JoinSet; -use super::FileScanConfig; - /// Execution plan for scanning NdJson data source #[derive(Debug, Clone)] pub struct NdJsonExec { @@ -457,7 +458,7 @@ mod tests { FileScanConfig { object_store_url, file_groups, - statistics: Statistics::new_with_unbounded_columns(&file_schema), + statistics: Statistics::new_unknown(&file_schema), file_schema, projection: None, limit: Some(3), @@ -536,7 +537,7 @@ mod tests { FileScanConfig { object_store_url, file_groups, - statistics: Statistics::new_with_unbounded_columns(&file_schema), + statistics: Statistics::new_unknown(&file_schema), file_schema, projection: None, limit: Some(3), @@ -584,7 +585,7 @@ mod tests { FileScanConfig { object_store_url, file_groups, - statistics: Statistics::new_with_unbounded_columns(&file_schema), + statistics: Statistics::new_unknown(&file_schema), file_schema, projection: Some(vec![0, 2]), limit: None, @@ -637,7 +638,7 @@ mod tests { FileScanConfig { object_store_url, file_groups, - statistics: Statistics::new_with_unbounded_columns(&file_schema), + statistics: Statistics::new_unknown(&file_schema), file_schema, projection: Some(vec![3, 0, 2]), limit: None, diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 2d855cfd2811..57844aac5181 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -20,32 +20,33 @@ mod arrow_file; mod avro; mod csv; +mod file_scan_config; mod file_stream; mod json; pub mod parquet; pub(crate) use self::csv::plan_to_csv; pub use self::csv::{CsvConfig, CsvExec, CsvOpener}; +pub(crate) use self::file_scan_config::PartitionColumnProjector; +pub(crate) use self::json::plan_to_json; pub(crate) use self::parquet::plan_to_parquet; pub use self::parquet::{ParquetExec, ParquetFileMetrics, ParquetFileReaderFactory}; -use arrow::{ - array::new_null_array, - compute::can_cast_types, - datatypes::{DataType, Schema, SchemaRef}, - record_batch::{RecordBatch, RecordBatchOptions}, -}; + pub use arrow_file::ArrowExec; pub use avro::AvroExec; -use datafusion_physical_expr::PhysicalSortExpr; -pub use file_stream::{FileOpenFuture, FileOpener, FileStream, OnError}; -pub(crate) use json::plan_to_json; -pub use json::{JsonOpener, NdJsonExec}; -mod file_scan_config; -pub(crate) use file_scan_config::PartitionColumnProjector; pub use file_scan_config::{ wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileScanConfig, }; +pub use file_stream::{FileOpenFuture, FileOpener, FileStream, OnError}; +pub use json::{JsonOpener, NdJsonExec}; +use std::{ + fmt::{Debug, Formatter, Result as FmtResult}, + sync::Arc, + vec, +}; + +use super::listing::ListingTableUrl; use crate::error::{DataFusionError, Result}; use crate::{ datasource::file_format::write::FileWriterMode, @@ -59,21 +60,20 @@ use crate::{ physical_plan::display::{OutputOrderingDisplay, ProjectSchemaDisplay}, }; +use arrow::{ + array::new_null_array, + compute::{can_cast_types, cast}, + datatypes::{DataType, Schema, SchemaRef}, + record_batch::{RecordBatch, RecordBatchOptions}, +}; use datafusion_common::{file_options::FileTypeWriterOptions, plan_err}; use datafusion_physical_expr::expressions::Column; - -use arrow::compute::cast; +use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_plan::ExecutionPlan; + use log::debug; use object_store::path::Path; use object_store::ObjectMeta; -use std::{ - fmt::{Debug, Formatter, Result as FmtResult}, - sync::Arc, - vec, -}; - -use super::listing::ListingTableUrl; /// The base configurations to provide when creating a physical plan for /// writing to any given file format. @@ -815,7 +815,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: file_group, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -878,9 +878,9 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: fg.clone(), file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns( - &Arc::new(Schema::empty()), - ), + statistics: Statistics::new_unknown(&Arc::new( + Schema::empty(), + )), projection: None, limit: None, table_partition_cols: vec![], @@ -914,7 +914,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: single_partition, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -951,7 +951,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: single_partition, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -994,7 +994,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: source_partitions, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -1033,7 +1033,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: source_partitions, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -1073,7 +1073,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: source_partitions, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], @@ -1102,7 +1102,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: single_partition, file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 79ae8a45e6d3..c781ad81c172 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -17,6 +17,11 @@ //! Execution plan for reading Parquet files +use std::any::Any; +use std::fmt::Debug; +use std::ops::Range; +use std::sync::Arc; + use crate::datasource::physical_plan::file_stream::{ FileOpenFuture, FileOpener, FileStream, }; @@ -36,27 +41,19 @@ use crate::{ Statistics, }, }; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, PhysicalSortExpr, -}; -use fmt::Debug; -use object_store::path::Path; -use std::any::Any; -use std::fmt; -use std::ops::Range; -use std::sync::Arc; -use tokio::task::JoinSet; use arrow::datatypes::{DataType, SchemaRef}; use arrow::error::ArrowError; use datafusion_physical_expr::{ - LexOrdering, OrderingEquivalenceProperties, PhysicalExpr, + ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, + PhysicalExpr, PhysicalSortExpr, }; use bytes::Bytes; use futures::future::BoxFuture; use futures::{StreamExt, TryStreamExt}; use log::debug; +use object_store::path::Path; use object_store::ObjectStore; use parquet::arrow::arrow_reader::ArrowReaderOptions; use parquet::arrow::async_reader::{AsyncFileReader, ParquetObjectReader}; @@ -64,6 +61,7 @@ use parquet::arrow::{AsyncArrowWriter, ParquetRecordBatchStreamBuilder, Projecti use parquet::basic::{ConvertedType, LogicalType}; use parquet::file::{metadata::ParquetMetaData, properties::WriterProperties}; use parquet::schema::types::ColumnDescriptor; +use tokio::task::JoinSet; mod metrics; pub mod page_filter; @@ -860,7 +858,7 @@ mod tests { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], - statistics: Statistics::new_with_unbounded_columns(&file_schema), + statistics: Statistics::new_unknown(&file_schema), file_schema, projection, limit: None, @@ -1517,7 +1515,7 @@ mod tests { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups, - statistics: Statistics::new_with_unbounded_columns(&file_schema), + statistics: Statistics::new_unknown(&file_schema), file_schema, projection: None, limit: None, @@ -1621,7 +1619,7 @@ mod tests { object_store_url, file_groups: vec![vec![partitioned_file]], file_schema: schema.clone(), - statistics: Statistics::new_with_unbounded_columns(&schema), + statistics: Statistics::new_unknown(&schema), // file has 10 cols so index 12 should be month and 13 should be day projection: Some(vec![0, 1, 2, 12, 13]), limit: None, @@ -1695,7 +1693,7 @@ mod tests { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![vec![partitioned_file]], file_schema: Arc::new(Schema::empty()), - statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index dce77094466f..9905e5bf774a 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -15,17 +15,17 @@ // specific language governing permissions and limitations // under the License. +use super::listing::PartitionedFile; use crate::arrow::datatypes::{Schema, SchemaRef}; use crate::error::Result; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; use crate::physical_plan::{Accumulator, ColumnStatistics, Statistics}; + use datafusion_common::stats::Sharpness; use datafusion_common::ScalarValue; -use futures::Stream; -use futures::StreamExt; -use itertools::izip; -use super::listing::PartitionedFile; +use futures::{Stream, StreamExt}; +use itertools::izip; /// Get all files as well as the file level summary statistics (no statistic for partition columns). /// If the optional `limit` is provided, includes only sufficient files. @@ -47,42 +47,45 @@ pub async fn get_statistics_with_limit( let mut num_rows: Option> = None; let mut total_byte_size: Option> = None; - // fusing the stream allows us to call next safely even once it is finished + // Fusing the stream allows us to call next safely even once it is finished. let mut all_files = Box::pin(all_files.fuse()); - while let Some(res) = all_files.next().await { - let (file, file_stats) = res?; + while let Some(current) = all_files.next().await { + let (file, file_stats) = current?; result_files.push(file); // Number of rows, total byte size and null counts are added for each file. // In case of an absent information or inexact value coming from the file, // it changes the statistic sharpness to inexact. - num_rows = if let Some(some_num_rows) = num_rows { - Some(match (file_stats.num_rows, &some_num_rows) { + num_rows = Some(if let Some(some_num_rows) = num_rows { + match (file_stats.num_rows, &some_num_rows) { (Sharpness::Absent, _) => some_num_rows.to_inexact(), (lhs, Sharpness::Absent) => lhs.to_inexact(), (lhs, rhs) => lhs.add(rhs), - }) + } } else { - Some(file_stats.num_rows) - }; + file_stats.num_rows + }); - total_byte_size = if let Some(some_total_byte_size) = total_byte_size { - Some(match (file_stats.total_byte_size, &some_total_byte_size) { + total_byte_size = Some(if let Some(some_total_byte_size) = total_byte_size { + match (file_stats.total_byte_size, &some_total_byte_size) { (Sharpness::Absent, _) => some_total_byte_size.to_inexact(), (lhs, Sharpness::Absent) => lhs.to_inexact(), (lhs, rhs) => lhs.add(rhs), - }) + } } else { - Some(file_stats.total_byte_size) - }; + file_stats.total_byte_size + }); if let Some(some_null_counts) = &mut null_counts { - for (i, cs) in file_stats.column_statistics.iter().enumerate() { - some_null_counts[i] = if cs.null_count == Sharpness::Absent { - // Downcast to inexact - some_null_counts[i].clone().to_inexact() + for (target, cs) in some_null_counts + .iter_mut() + .zip(file_stats.column_statistics.iter()) + { + *target = if cs.null_count == Sharpness::Absent { + // Downcast to inexact: + target.clone().to_inexact() } else { - some_null_counts[i].add(&cs.null_count) + target.add(&cs.null_count) }; } } else { @@ -92,8 +95,9 @@ pub async fn get_statistics_with_limit( .iter() .map(|cs| cs.null_count.clone()) .collect::>(); - // file schema may have additional fields other than each file (such as partition, guaranteed to be at the end) - // Hence, push rest of the fields with information Absent. + // File schema may have additional fields other than each file (such + // as partitions, guaranteed to be at the end). Hence, rest of the + // fields are initialized with `Absent`. for _ in 0..file_schema.fields().len() - file_stats.column_statistics.len() { new_col_stats_nulls.push(Sharpness::Absent) } @@ -153,24 +157,20 @@ pub async fn get_statistics_with_limit( } } - let null_counts = - null_counts.unwrap_or(vec![Sharpness::Absent; file_schema.fields().len()]); - let max_values = - max_values.unwrap_or(vec![Sharpness::Absent; file_schema.fields().len()]); - let min_values = - min_values.unwrap_or(vec![Sharpness::Absent; file_schema.fields().len()]); - - let column_stats = get_col_stats_vec(null_counts, max_values, min_values); + let size = file_schema.fields().len(); + let null_counts = null_counts.unwrap_or(vec![Sharpness::Absent; size]); + let max_values = max_values.unwrap_or(vec![Sharpness::Absent; size]); + let min_values = min_values.unwrap_or(vec![Sharpness::Absent; size]); let mut statistics = Statistics { num_rows: num_rows.unwrap_or(Sharpness::Absent), total_byte_size: total_byte_size.unwrap_or(Sharpness::Absent), - column_statistics: column_stats, + column_statistics: get_col_stats_vec(null_counts, max_values, min_values), }; if all_files.next().await.is_some() { - // if we still have files in the stream, it means that the limit kicked - // in and the statistic could have been different if we have - // processed the files in a different order. + // If we still have files in the stream, it means that the limit kicked + // in, and the statistic could have been different had we processed the + // files in a different order. statistics = statistics.make_inexact() } @@ -184,12 +184,12 @@ pub(crate) fn create_max_min_accs( .fields() .iter() .map(|field| MaxAccumulator::try_new(field.data_type()).ok()) - .collect::>(); + .collect(); let min_values: Vec> = schema .fields() .iter() .map(|field| MinAccumulator::try_new(field.data_type()).ok()) - .collect::>(); + .collect(); (max_values, min_values) } @@ -234,8 +234,8 @@ pub(crate) fn get_col_stats( .collect() } -/// If the given value is numerically greater than the original value, -/// it set the new max value with the exactness information. +/// If the given value is numerically greater than the original maximum value, +/// set the new maximum value with appropriate exactness information. fn set_max_if_greater( max_values: &mut [Sharpness], max_nominee: Sharpness, @@ -266,8 +266,8 @@ fn set_max_if_greater( } } -/// If the given value is numerically lesser than the original value, -/// it set the new min value with the exactness information. +/// If the given value is numerically lesser than the original minimum value, +/// set the new minimum value with appropriate exactness information. fn set_min_if_lesser( min_values: &mut [Sharpness], min_nominee: Sharpness, diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index c50ad8f648e3..7a11c24095eb 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -16,21 +16,20 @@ // under the License. //! Utilizing exact statistics from sources to avoid scanning data -use datafusion_common::stats::Sharpness; use std::sync::Arc; +use super::optimizer::PhysicalOptimizerRule; use crate::config::ConfigOptions; -use datafusion_common::tree_node::TreeNode; -use datafusion_expr::utils::COUNT_STAR_EXPANSION; - +use crate::error::Result; use crate::physical_plan::aggregates::AggregateExec; use crate::physical_plan::empty::EmptyExec; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::{expressions, AggregateExpr, ExecutionPlan, Statistics}; use crate::scalar::ScalarValue; -use super::optimizer::PhysicalOptimizerRule; -use crate::error::Result; +use datafusion_common::stats::Sharpness; +use datafusion_common::tree_node::TreeNode; +use datafusion_expr::utils::COUNT_STAR_EXPANSION; /// Optimizer that uses available statistics for aggregate functions #[derive(Default)] @@ -243,17 +242,9 @@ fn take_optimizable_max( #[cfg(test)] mod tests { - use super::*; use std::sync::Arc; - use arrow::array::Int32Array; - use arrow::datatypes::{DataType, Field, Schema}; - use arrow::record_batch::RecordBatch; - use datafusion_common::cast::as_int64_array; - use datafusion_physical_expr::expressions::cast; - use datafusion_physical_expr::PhysicalExpr; - use datafusion_physical_plan::aggregates::AggregateMode; - + use super::*; use crate::error::Result; use crate::logical_expr::Operator; use crate::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; @@ -264,6 +255,14 @@ mod tests { use crate::physical_plan::memory::MemoryExec; use crate::prelude::SessionContext; + use arrow::array::Int32Array; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow::record_batch::RecordBatch; + use datafusion_common::cast::as_int64_array; + use datafusion_physical_expr::expressions::cast; + use datafusion_physical_expr::PhysicalExpr; + use datafusion_physical_plan::aggregates::AggregateMode; + /// Mock data using a MemoryExec which has an exact count statistic fn mock_data() -> Result> { let schema = Arc::new(Schema::new(vec![ 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 2a54df3e8f35..838ae613683e 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -17,13 +17,15 @@ //! 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_optimizer::PhysicalOptimizerRule; use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use crate::physical_plan::ExecutionPlan; -use datafusion_common::config::ConfigOptions; -use std::sync::Arc; +use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{AggregateExpr, PhysicalExpr}; @@ -191,10 +193,6 @@ fn discard_column_index(group_expr: Arc) -> Arc { @@ -248,7 +250,7 @@ mod tests { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_with_unbounded_columns(schema), + statistics: Statistics::new_unknown(schema), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index ed8b96174303..f42fc5a2d57c 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -44,8 +44,9 @@ use crate::physical_plan::sorts::sort::SortOptions; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::union::{can_interleave, InterleaveExec, UnionExec}; use crate::physical_plan::windows::WindowAggExec; -use crate::physical_plan::Partitioning; -use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; +use crate::physical_plan::{ + with_new_children_if_necessary, Distribution, ExecutionPlan, Partitioning, +}; use datafusion_common::stats::Sharpness; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; @@ -60,6 +61,7 @@ use datafusion_physical_expr::{ }; use datafusion_physical_plan::unbounded_output; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; + use itertools::izip; /// The `EnforceDistribution` rule ensures that distribution requirements are @@ -1237,8 +1239,8 @@ fn ensure_distribution( let stats = dist_context.plan.statistics()?; let mut repartition_beneficial_stat = true; if let Sharpness::Exact(num_rows) = stats.num_rows { - // when we are sure that number of rows is <=1 - // repartitioning is not beneficial + // If the number of rows is surely less than one, then repartitioning + // is not beneficial. repartition_beneficial_stat = num_rows > 1; } // Remove unnecessary repartition from the physical plan if any @@ -1823,7 +1825,7 @@ mod tests { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_with_unbounded_columns(&schema()), + statistics: Statistics::new_unknown(&schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -1851,7 +1853,7 @@ mod tests { vec![PartitionedFile::new("x".to_string(), 100)], vec![PartitionedFile::new("y".to_string(), 100)], ], - statistics: Statistics::new_with_unbounded_columns(&schema()), + statistics: Statistics::new_unknown(&schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -1873,7 +1875,7 @@ mod tests { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_with_unbounded_columns(&schema()), + statistics: Statistics::new_unknown(&schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -1904,7 +1906,7 @@ mod tests { vec![PartitionedFile::new("x".to_string(), 100)], vec![PartitionedFile::new("y".to_string(), 100)], ], - statistics: Statistics::new_with_unbounded_columns(&schema()), + statistics: Statistics::new_unknown(&schema()), projection: None, limit: None, table_partition_cols: vec![], @@ -3941,7 +3943,7 @@ mod tests { "x".to_string(), 100, )]], - statistics: Statistics::new_with_unbounded_columns(&schema()), + statistics: Statistics::new_unknown(&schema()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 5702bf6b4866..4869e2c55055 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -92,9 +92,7 @@ fn supports_collect_by_size( ) -> bool { // Currently we do not trust the 0 value from stats, due to stats collection might have bug // TODO check the logic in datasource::get_statistics_with_limit() - let stats = if let Ok(stats) = plan.statistics() { - stats - } else { + let Ok(stats) = plan.statistics() else { return false; }; if let Some(size) = stats.total_byte_size.get_value() { @@ -592,6 +590,8 @@ fn apply_subrules( #[cfg(test)] mod tests_statistical { + use std::sync::Arc; + use super::*; use crate::{ physical_plan::{ @@ -600,8 +600,6 @@ mod tests_statistical { test::StatisticsExec, }; - use std::sync::Arc; - use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{stats::Sharpness, JoinType, ScalarValue}; use datafusion_physical_expr::expressions::Column; @@ -1204,6 +1202,8 @@ mod tests_statistical { #[cfg(test)] mod util_tests { + use std::sync::Arc; + use arrow_schema::{DataType, Field, Schema}; use datafusion_common::ScalarValue; use datafusion_expr::Operator; @@ -1212,7 +1212,6 @@ mod util_tests { }; use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::PhysicalExpr; - use std::sync::Arc; #[test] fn check_expr_supported() { diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index be2341f1f74d..2541661269b2 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -18,17 +18,19 @@ //! The [PipelineChecker] rule ensures that a given plan can accommodate its //! infinite sources, if there are any. It will reject non-runnable query plans //! that use pipeline-breaking operators on infinite input(s). -//! + +use std::sync::Arc; + use crate::config::ConfigOptions; use crate::error::Result; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::joins::SymmetricHashJoinExec; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; + use datafusion_common::config::OptimizerOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{plan_err, DataFusionError}; use datafusion_physical_expr::intervals::utils::{check_support, is_datatype_supported}; -use std::sync::Arc; /// The PipelineChecker rule rejects non-runnable query plans that use /// pipeline-breaking operators on infinite input(s). diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index beabe29855f4..5b2ecbdf8795 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -896,7 +896,7 @@ mod tests { "file_path".to_string(), 100, )]], - statistics: Statistics::new_with_unbounded_columns(schema), + statistics: Statistics::new_unknown(schema), projection: Some(projection), limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index bd80b5e6f158..71004b28fef8 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -268,7 +268,7 @@ pub fn parquet_exec(schema: &SchemaRef) -> Arc { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_with_unbounded_columns(schema), + statistics: Statistics::new_unknown(schema), projection: None, limit: None, table_partition_cols: vec![], @@ -292,7 +292,7 @@ pub fn parquet_exec_sorted( object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_with_unbounded_columns(schema), + statistics: Statistics::new_unknown(schema), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 1ab10dc9de26..9110a6f3f05c 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -17,6 +17,10 @@ //! Planner for [`LogicalPlan`] to [`ExecutionPlan`] +use std::collections::HashMap; +use std::fmt::Write; +use std::sync::Arc; + use crate::datasource::file_format::arrow::ArrowFormat; use crate::datasource::file_format::avro::AvroFormat; use crate::datasource::file_format::csv::CsvFormat; @@ -27,6 +31,7 @@ use crate::datasource::file_format::FileFormat; use crate::datasource::listing::ListingTableUrl; use crate::datasource::physical_plan::FileSinkConfig; use crate::datasource::source_as_provider; +use crate::error::{DataFusionError, Result}; use crate::execution::context::{ExecutionProps, SessionState}; use crate::logical_expr::utils::generate_sort_key; use crate::logical_expr::{ @@ -37,49 +42,45 @@ use crate::logical_expr::{ CrossJoin, Expr, LogicalPlan, Partitioning as LogicalPartitioning, PlanType, Repartition, Union, UserDefinedLogicalNode, }; -use crate::physical_plan::memory::MemoryExec; -use arrow_array::builder::StringBuilder; -use arrow_array::RecordBatch; -use datafusion_common::display::ToStringifiedPlan; -use datafusion_common::file_options::FileTypeWriterOptions; -use datafusion_common::FileType; -use datafusion_expr::dml::{CopyOptions, CopyTo}; - use crate::logical_expr::{Limit, Values}; use crate::physical_expr::create_physical_expr; use crate::physical_optimizer::optimizer::PhysicalOptimizerRule; use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use crate::physical_plan::analyze::AnalyzeExec; +use crate::physical_plan::empty::EmptyExec; use crate::physical_plan::explain::ExplainExec; use crate::physical_plan::expressions::{Column, PhysicalSortExpr}; use crate::physical_plan::filter::FilterExec; -use crate::physical_plan::joins::HashJoinExec; -use crate::physical_plan::joins::SortMergeJoinExec; -use crate::physical_plan::joins::{CrossJoinExec, NestedLoopJoinExec}; +use crate::physical_plan::joins::utils as join_utils; +use crate::physical_plan::joins::{ + CrossJoinExec, HashJoinExec, NestedLoopJoinExec, PartitionMode, SortMergeJoinExec, +}; use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use crate::physical_plan::memory::MemoryExec; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; +use crate::physical_plan::union::UnionExec; use crate::physical_plan::unnest::UnnestExec; +use crate::physical_plan::values::ValuesExec; use crate::physical_plan::windows::{ BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, }; use crate::physical_plan::{ - aggregates, empty::EmptyExec, joins::PartitionMode, udaf, union::UnionExec, - values::ValuesExec, windows, -}; -use crate::physical_plan::{joins::utils as join_utils, Partitioning}; -use crate::physical_plan::{AggregateExpr, ExecutionPlan, PhysicalExpr, WindowExpr}; -use crate::{ - error::{DataFusionError, Result}, - physical_plan::displayable, + aggregates, displayable, udaf, windows, AggregateExpr, ExecutionPlan, Partitioning, + PhysicalExpr, WindowExpr, }; + use arrow::compute::SortOptions; use arrow::datatypes::{Schema, SchemaRef}; -use async_trait::async_trait; +use arrow_array::builder::StringBuilder; +use arrow_array::RecordBatch; +use datafusion_common::display::ToStringifiedPlan; +use datafusion_common::file_options::FileTypeWriterOptions; use datafusion_common::{ - exec_err, internal_err, not_impl_err, plan_err, DFSchema, ScalarValue, + exec_err, internal_err, not_impl_err, plan_err, DFSchema, FileType, ScalarValue, }; +use datafusion_expr::dml::{CopyOptions, CopyTo}; use datafusion_expr::expr::{ self, AggregateFunction, AggregateUDF, Alias, Between, BinaryExpr, Cast, GetFieldAccess, GetIndexedField, GroupingSet, InList, Like, ScalarUDF, TryCast, @@ -87,17 +88,17 @@ use datafusion_expr::expr::{ }; use datafusion_expr::expr_rewriter::{unalias, unnormalize_cols}; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; -use datafusion_expr::{DescribeTable, DmlStatement, StringifiedPlan, WriteOp}; -use datafusion_expr::{WindowFrame, WindowFrameBound}; +use datafusion_expr::{ + DescribeTable, DmlStatement, StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, +}; use datafusion_physical_expr::expressions::Literal; use datafusion_sql::utils::window_expr_common_partition_keys; + +use async_trait::async_trait; use futures::future::BoxFuture; use futures::{FutureExt, StreamExt, TryStreamExt}; use itertools::{multiunzip, Itertools}; use log::{debug, trace}; -use std::collections::HashMap; -use std::fmt::Write; -use std::sync::Arc; fn create_function_physical_name( fun: &str, diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 69f06f6db7a7..aad5c19044ea 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -17,6 +17,13 @@ //! Common unit test utility methods +use std::any::Any; +use std::fs::File; +use std::io::prelude::*; +use std::io::{BufReader, BufWriter}; +use std::path::Path; +use std::sync::Arc; + use crate::datasource::file_format::file_compression_type::{ FileCompressionType, FileTypeExt, }; @@ -29,29 +36,23 @@ use crate::logical_expr::LogicalPlan; use crate::physical_plan::ExecutionPlan; use crate::test::object_store::local_unpartitioned_file; use crate::test_util::{aggr_test_schema, arrow_test_data}; -use array::ArrayRef; -use arrow::array::{self, Array, Decimal128Builder, Int32Array}; + +use arrow::array::{self, Array, ArrayRef, Decimal128Builder, Int32Array}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; +use datafusion_common::{DataFusionError, FileType, Statistics}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{Partitioning, PhysicalSortExpr}; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; + #[cfg(feature = "compression")] use bzip2::write::BzEncoder; #[cfg(feature = "compression")] use bzip2::Compression as BzCompression; -use datafusion_common::FileType; -use datafusion_common::{DataFusionError, Statistics}; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::{Partitioning, PhysicalSortExpr}; -use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; #[cfg(feature = "compression")] use flate2::write::GzEncoder; #[cfg(feature = "compression")] use flate2::Compression as GzCompression; -use std::any::Any; -use std::fs::File; -use std::io::prelude::*; -use std::io::{BufReader, BufWriter}; -use std::path::Path; -use std::sync::Arc; #[cfg(feature = "compression")] use xz2::write::XzEncoder; #[cfg(feature = "compression")] @@ -197,7 +198,7 @@ pub fn partitioned_csv_config( object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: schema.clone(), file_groups, - statistics: Statistics::new_with_unbounded_columns(&schema), + statistics: Statistics::new_unknown(&schema), projection: None, limit: None, table_partition_cols: vec![], @@ -285,7 +286,7 @@ pub fn csv_exec_sorted( object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), file_schema: schema.clone(), file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_with_unbounded_columns(schema), + statistics: Statistics::new_unknown(schema), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index 54d9332b5bbc..d826ec8bfbb6 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -37,12 +37,14 @@ use crate::physical_plan::{ SendableRecordBatchStream, }; use crate::prelude::{CsvReadOptions, SessionContext}; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use async_trait::async_trait; use datafusion_common::{Statistics, TableReference}; use datafusion_expr::{CreateExternalTable, Expr, TableType}; use datafusion_physical_expr::PhysicalSortExpr; + +use async_trait::async_trait; use futures::Stream; // backwards compatibility @@ -50,8 +52,7 @@ pub use datafusion_common::test_util::{ arrow_test_data, get_data_dir, parquet_test_data, }; -pub use datafusion_common::assert_batches_eq; -pub use datafusion_common::assert_batches_sorted_eq; +pub use datafusion_common::{assert_batches_eq, assert_batches_sorted_eq}; /// Scan an empty data source, mainly used in tests pub fn scan_empty( @@ -239,7 +240,7 @@ impl ExecutionPlan for UnboundedExec { } fn statistics(&self) -> Result { - Ok(Statistics::new_with_unbounded_columns(&self.schema())) + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index f06c8506ea52..0d11526703b4 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -35,7 +35,9 @@ use crate::physical_plan::filter::FilterExec; use crate::physical_plan::metrics::MetricsSet; use crate::physical_plan::ExecutionPlan; use crate::prelude::{Expr, SessionConfig}; + use datafusion_common::Statistics; + use object_store::path::Path; use object_store::ObjectMeta; use parquet::arrow::ArrowWriter; @@ -148,7 +150,7 @@ impl TestParquetFile { range: None, extensions: None, }]], - statistics: Statistics::new_with_unbounded_columns(&self.schema), + statistics: Statistics::new_unknown(&self.schema), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index 8f57f2bd377d..0f7036e51df7 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -15,10 +15,17 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + use arrow::array::{Int32Array, Int64Array}; use arrow::compute::kernels::aggregate; use arrow::datatypes::{DataType, Field, Int32Type, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; +use datafusion::datasource::{TableProvider, TableType}; +use datafusion::error::Result; use datafusion::execution::context::{SessionContext, SessionState, TaskContext}; use datafusion::logical_expr::{ col, Expr, LogicalPlan, LogicalPlanBuilder, TableScan, UNNAMED_TABLE, @@ -26,30 +33,20 @@ use datafusion::logical_expr::{ use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::{ - ColumnStatistics, DisplayAs, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, Statistics, + collect, ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, + RecordBatchStream, SendableRecordBatchStream, Statistics, }; use datafusion::scalar::ScalarValue; -use datafusion::{ - datasource::{TableProvider, TableType}, - physical_plan::collect, -}; -use datafusion::{error::Result, physical_plan::DisplayFormatType}; - use datafusion_common::cast::as_primitive_array; use datafusion_common::project_schema; use datafusion_common::stats::Sharpness; + +use async_trait::async_trait; use futures::stream::Stream; -use std::any::Any; -use std::pin::Pin; -use std::sync::Arc; -use std::task::{Context, Poll}; /// Also run all tests that are found in the `custom_sources_cases` directory mod custom_sources_cases; -use async_trait::async_trait; - //--- Custom source dataframe tests ---// struct CustomTableProvider; diff --git a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs index 351b26b6554b..4679ca6d07df 100644 --- a/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs +++ b/datafusion/core/tests/custom_sources_cases/provider_filter_pushdown.rs @@ -122,7 +122,7 @@ impl ExecutionPlan for CustomPlan { fn statistics(&self) -> Result { // here we could provide more accurate statistics // but we want to test the filter pushdown not the CBOs - Ok(Statistics::new_with_unbounded_columns(&self.schema())) + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index 3bbe880a7949..bad6a22bafd2 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -236,13 +236,13 @@ async fn sql_filter() -> Result<()> { #[tokio::test] async fn sql_limit() -> Result<()> { let (stats, schema) = fully_defined(); - let ctx = init_ctx(stats.clone(), schema.clone())?; + let col_stats = Statistics::unknown_column(&schema); + let ctx = init_ctx(stats.clone(), schema)?; let df = ctx.sql("SELECT * FROM stats_table LIMIT 5").await.unwrap(); let physical_plan = df.create_physical_plan().await.unwrap(); // when the limit is smaller than the original number of lines // we loose all statistics except the for number of rows which becomes the limit - let col_stats = Statistics::unbounded_column_statistics(&schema); assert_eq!( Statistics { num_rows: Sharpness::Exact(5), diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index e5586a053d61..37481b936d24 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -15,6 +15,11 @@ // specific language governing permissions and limitations // under the License. +use std::io::Cursor; +use std::ops::Range; +use std::sync::Arc; +use std::time::SystemTime; + use arrow::array::{ArrayRef, Int64Array, Int8Array, StringArray}; use arrow::datatypes::{Field, Schema, SchemaBuilder}; use arrow::record_batch::RecordBatch; @@ -30,6 +35,7 @@ use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::physical_plan::{collect, Statistics}; use datafusion::prelude::SessionContext; use datafusion_common::Result; + use futures::future::BoxFuture; use futures::{FutureExt, TryFutureExt}; use object_store::memory::InMemory; @@ -39,10 +45,6 @@ use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::ArrowWriter; use parquet::errors::ParquetError; use parquet::file::metadata::ParquetMetaData; -use std::io::Cursor; -use std::ops::Range; -use std::sync::Arc; -use std::time::SystemTime; const EXPECTED_USER_DEFINED_METADATA: &str = "some-user-defined-metadata"; @@ -77,7 +79,7 @@ async fn route_data_access_ops_to_parquet_file_reader_factory() { // just any url that doesn't point to in memory object store object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], - statistics: Statistics::new_with_unbounded_columns(&file_schema), + statistics: Statistics::new_unknown(&file_schema), file_schema, projection: None, limit: None, diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index d88343824d2f..6bb53b0c8322 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +use std::fs; +use std::sync::Arc; + use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::{ ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, @@ -31,8 +34,7 @@ use datafusion_execution::cache::cache_unit::{ }; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; -use std::fs; -use std::sync::Arc; + use tempfile::tempdir; #[tokio::test] diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 74843645c426..b77643c35e84 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -17,6 +17,7 @@ use crate::parquet::Unit::Page; use crate::parquet::{ContextWithParquet, Scenario}; + use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::file_format::FileFormat; use datafusion::datasource::listing::PartitionedFile; @@ -30,6 +31,7 @@ use datafusion_common::{ScalarValue, Statistics, ToDFSchema}; use datafusion_expr::{col, lit, Expr}; use datafusion_physical_expr::create_physical_expr; use datafusion_physical_expr::execution_props::ExecutionProps; + use futures::StreamExt; use object_store::path::Path; use object_store::ObjectMeta; @@ -72,7 +74,7 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { object_store_url, file_groups: vec![vec![partitioned_file]], file_schema: schema.clone(), - statistics: Statistics::new_with_unbounded_columns(&schema), + statistics: Statistics::new_unknown(&schema), // file has 10 cols so index 12 should be month projection: None, limit: None, diff --git a/datafusion/core/tests/parquet/schema_coercion.rs b/datafusion/core/tests/parquet/schema_coercion.rs index 6ba74b817254..b3134d470b56 100644 --- a/datafusion/core/tests/parquet/schema_coercion.rs +++ b/datafusion/core/tests/parquet/schema_coercion.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + use arrow::datatypes::{Field, Schema}; use arrow::record_batch::RecordBatch; use arrow_array::types::Int32Type; @@ -24,14 +26,13 @@ use datafusion::assert_batches_sorted_eq; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion::physical_plan::collect; use datafusion::prelude::SessionContext; -use datafusion_common::Result; -use datafusion_common::Statistics; +use datafusion_common::{Result, Statistics}; use datafusion_execution::object_store::ObjectStoreUrl; + use object_store::path::Path; use object_store::ObjectMeta; use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; -use std::sync::Arc; use tempfile::NamedTempFile; /// Test for reading data from multiple parquet files with different schemas and coercing them into a single schema. @@ -62,7 +63,7 @@ async fn multi_parquet_coercion() { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], - statistics: Statistics::new_with_unbounded_columns(&file_schema), + statistics: Statistics::new_unknown(&file_schema), file_schema, projection: None, limit: None, @@ -126,7 +127,7 @@ async fn multi_parquet_coercion_projection() { FileScanConfig { object_store_url: ObjectStoreUrl::local_filesystem(), file_groups: vec![file_groups], - statistics: Statistics::new_with_unbounded_columns(&file_schema), + statistics: Statistics::new_unknown(&file_schema), file_schema, projection: Some(vec![1, 0, 2]), limit: None, diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index 62378cd9a509..9e13f83fd473 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -17,17 +17,13 @@ //! Test queries on partitioned datasets -use arrow::datatypes::DataType; -use datafusion_common::stats::Sharpness; use std::collections::BTreeSet; use std::fs::File; use std::io::{Read, Seek, SeekFrom}; use std::ops::Range; use std::sync::Arc; -use async_trait::async_trait; -use bytes::Bytes; -use chrono::{TimeZone, Utc}; +use arrow::datatypes::DataType; use datafusion::datasource::listing::ListingTableUrl; use datafusion::{ assert_batches_sorted_eq, @@ -40,7 +36,12 @@ use datafusion::{ prelude::SessionContext, test_util::{self, arrow_test_data, parquet_test_data}, }; +use datafusion_common::stats::Sharpness; use datafusion_common::ScalarValue; + +use async_trait::async_trait; +use bytes::Bytes; +use chrono::{TimeZone, Utc}; use futures::stream; use futures::stream::BoxStream; use object_store::{ diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 0ec1a1564d1b..d20070183ad1 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -16,6 +16,7 @@ // under the License. use super::*; + use datafusion::config::ConfigOptions; use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::physical_plan::metrics::Timestamp; diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index c6be211c4bb9..d4a8842c0a7a 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -58,7 +58,9 @@ //! N elements, reducing the total amount of required buffer memory. //! -use futures::{Stream, StreamExt}; +use std::fmt::Debug; +use std::task::{Context, Poll}; +use std::{any::Any, collections::BTreeMap, fmt, sync::Arc}; use arrow::{ array::{Int64Array, StringArray}, @@ -68,8 +70,7 @@ use arrow::{ }; use datafusion::{ common::cast::{as_int64_array, as_string_array}, - common::internal_err, - common::DFSchemaRef, + common::{internal_err, DFSchemaRef}, error::{DataFusionError, Result}, execution::{ context::{QueryPlanner, SessionState, TaskContext}, @@ -89,11 +90,8 @@ use datafusion::{ prelude::{SessionConfig, SessionContext}, }; -use fmt::Debug; -use std::task::{Context, Poll}; -use std::{any::Any, collections::BTreeMap, fmt, sync::Arc}; - use async_trait::async_trait; +use futures::{Stream, StreamExt}; /// Execute the specified sql and return the resulting record batches /// pretty printed as a String. @@ -493,7 +491,7 @@ impl ExecutionPlan for TopKExec { fn statistics(&self) -> Result { // to improve the optimizability of this plan // better statistics inference could be provided - Ok(Statistics::new_with_unbounded_columns(&self.schema())) + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/execution/src/cache/cache_unit.rs b/datafusion/execution/src/cache/cache_unit.rs index e7333b9b72ee..4a21dc02bd13 100644 --- a/datafusion/execution/src/cache/cache_unit.rs +++ b/datafusion/execution/src/cache/cache_unit.rs @@ -15,12 +15,15 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + use crate::cache::CacheAccessor; -use dashmap::DashMap; + use datafusion_common::Statistics; + +use dashmap::DashMap; use object_store::path::Path; use object_store::ObjectMeta; -use std::sync::Arc; /// Collected statistics for files /// Cache is invalided when file size or last modification has changed @@ -179,7 +182,7 @@ mod tests { cache.put_with_extra( &meta.location, - Statistics::new_with_unbounded_columns(&Schema::new(vec![Field::new( + Statistics::new_unknown(&Schema::new(vec![Field::new( "test_column", DataType::Timestamp(TimeUnit::Second, None), false, diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index 15ce6e15baff..0c089ce396ea 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -21,6 +21,7 @@ use std::fmt::Debug; use std::sync::Arc; use arrow::datatypes::Schema; +use arrow_schema::Field; use datafusion_common::stats::Sharpness; use datafusion_common::{ColumnStatistics, DataFusionError, Result, ScalarValue}; @@ -64,7 +65,7 @@ impl AnalysisContext { ) -> Self { let mut column_boundaries = vec![]; for (idx, stats) in statistics.iter().enumerate() { - let field: &Arc = &input_schema.fields()[idx]; + let field: &Arc = &input_schema.fields()[idx]; if let Ok(inf_field) = ScalarValue::try_from(field.data_type()) { let interval = Interval::new( IntervalBound::new_closed( @@ -78,7 +79,7 @@ impl AnalysisContext { stats.max_value.get_value().cloned().unwrap_or(inf_field), ), ); - let column = Column::new(input_schema.fields()[idx].name(), idx); + let column = Column::new(field.name(), idx); column_boundaries.push(ExprBoundaries { column, interval, @@ -191,7 +192,7 @@ fn shrink_boundaries( } }); let graph_nodes = graph.gather_node_indices(&[expr.clone()]); - // Since the propagation result is success, the graph has at least one element. + // Since propagation result was successful, the graph has at least one element. // An empty check is also done at the outer scope, do not repeat it here. let (_, root_index) = graph_nodes[0]; let final_result = graph.get_interval(root_index); diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index 906f6709f248..09650e39df8b 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -170,11 +170,12 @@ pub fn negative( mod tests { use super::*; use crate::expressions::{col, Column}; - #[allow(unused_imports)] + use arrow::array::*; use arrow::datatypes::*; use arrow_schema::DataType::{Float32, Float64, Int16, Int32, Int64, Int8}; use datafusion_common::{cast::as_primitive_array, Result}; + use paste::paste; macro_rules! test_array_negative_op { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 86a8df3c99ea..6852d3490717 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1010,7 +1010,7 @@ impl ExecutionPlan for AggregateExec { // - case where we group by on a column for which with have the `distinct` stat // TODO stats: aggr expression: // - aggregations somtimes also preserve invariants such as min, max... - let column_statistics = Statistics::unbounded_column_statistics(&self.schema()); + let column_statistics = Statistics::unknown_column(&self.schema()); match self.mode { AggregateMode::Final | AggregateMode::FinalPartitioned if self.group_by.expr.is_empty() => diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index 0830315b3600..9652464b3d8c 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -198,7 +198,7 @@ impl ExecutionPlan for AnalyzeExec { fn statistics(&self) -> Result { // Statistics an an ANALYZE plan are not relevant - Ok(Statistics::new_with_unbounded_columns(&self.schema())) + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index 7dffc8ec668e..6aa4b8f4327a 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -171,7 +171,7 @@ impl ExecutionPlan for ExplainExec { fn statistics(&self) -> Result { // Statistics an EXPLAIN plan are not relevant - Ok(Statistics::new_with_unbounded_columns(&self.schema())) + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 74ac765a8d1a..27b97757de49 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -203,7 +203,7 @@ impl ExecutionPlan for FilterExec { let predicate = self.predicate(); if !check_support(predicate, self.schema()) { - return Ok(Statistics::new_with_unbounded_columns(&self.schema())); + return Ok(Statistics::new_unknown(&self.schema())); } let input_stats = self.input.statistics()?; @@ -618,7 +618,7 @@ mod tests { // a: min=???, max=??? (missing) let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let input = Arc::new(StatisticsExec::new( - Statistics::new_with_unbounded_columns(&schema), + Statistics::new_unknown(&schema), schema.clone(), )); @@ -938,7 +938,7 @@ mod tests { async fn test_empty_input_statistics() -> Result<()> { let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let input = Arc::new(StatisticsExec::new( - Statistics::new_with_unbounded_columns(&schema), + Statistics::new_unknown(&schema), schema, )); // WHERE a <= 10 AND 0 <= a - 5 diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index ae2c938ca814..4c0f7ada790c 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -264,7 +264,7 @@ impl ExecutionPlan for FileSinkExec { } fn statistics(&self) -> Result { - Ok(Statistics::new_with_unbounded_columns(&self.schema())) + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 7341f2f68dec..5758cbc4c08e 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -469,7 +469,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { fn statistics(&self) -> Result { // TODO stats: it is not possible in general to know the output size of joins - Ok(Statistics::new_with_unbounded_columns(&self.schema())) + Ok(Statistics::new_unknown(&self.schema())) } fn execute( diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 4df458994614..912835ca67cd 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -734,10 +734,7 @@ pub(crate) fn estimate_join_statistics( let join_stats = estimate_join_cardinality(join_type, left_stats, right_stats, &on); let (num_rows, column_statistics) = match join_stats { Some(stats) => (Sharpness::Inexact(stats.num_rows), stats.column_statistics), - None => ( - Sharpness::Absent, - Statistics::unbounded_column_statistics(schema), - ), + None => (Sharpness::Absent, Statistics::unknown_column(schema)), }; Ok(Statistics { num_rows, diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 50ecc8e9ae42..f1fb8b14a456 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -206,7 +206,7 @@ impl ExecutionPlan for GlobalLimitExec { } }) .unwrap_or(usize::MAX); - let col_stats = Statistics::unbounded_column_statistics(&self.schema()); + let col_stats = Statistics::unknown_column(&self.schema()); let fetched_row_number_stats = Statistics { num_rows: Sharpness::Exact(max_row_num), @@ -372,7 +372,7 @@ impl ExecutionPlan for LocalLimitExec { fn statistics(&self) -> Result { let input_stats = self.input.statistics()?; - let col_stats = Statistics::unbounded_column_statistics(&self.schema()); + let col_stats = Statistics::unknown_column(&self.schema()); let stats = match input_stats { // if the input does not reach the limit globally, return input stats Statistics { diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index eb331442aa6e..5ab5a6159ad3 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -190,6 +190,6 @@ impl ExecutionPlan for StreamingTableExec { } fn statistics(&self) -> Result { - Ok(Statistics::new_with_unbounded_columns(&self.schema())) + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index 94741c6ff89d..0431e45b4023 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -457,7 +457,7 @@ impl ExecutionPlan for ErrorExec { } fn statistics(&self) -> Result { - Ok(Statistics::new_with_unbounded_columns(&self.schema())) + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index c4ae4a82460c..0a4e49d6c73b 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -275,7 +275,7 @@ impl ExecutionPlan for UnionExec { Ok(stats .into_iter() .reduce(stats_union) - .unwrap_or_else(|| Statistics::new_with_unbounded_columns(&self.schema()))) + .unwrap_or_else(|| Statistics::new_unknown(&self.schema()))) } fn benefits_from_input_partitioning(&self) -> Vec { @@ -454,7 +454,7 @@ impl ExecutionPlan for InterleaveExec { Ok(stats .into_iter() .reduce(stats_union) - .unwrap_or_else(|| Statistics::new_with_unbounded_columns(&self.schema()))) + .unwrap_or_else(|| Statistics::new_unknown(&self.schema()))) } fn benefits_from_input_partitioning(&self) -> Vec { diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 5c6f1b5b1463..be5fcb40d243 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -161,7 +161,7 @@ impl ExecutionPlan for UnnestExec { } fn statistics(&self) -> Result { - Ok(Statistics::new_with_unbounded_columns(&self.schema())) + Ok(Statistics::new_unknown(&self.schema())) } } diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 62ecaab58df9..9547053db13a 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -476,9 +476,9 @@ fn roundtrip_parquet_exec_with_pruning_predicate() -> Result<()> { statistics: Statistics { num_rows: Sharpness::Inexact(100), total_byte_size: Sharpness::Inexact(1024), - column_statistics: Statistics::unbounded_column_statistics(&Arc::new( - Schema::new(vec![Field::new("col", DataType::Utf8, false)]), - )), + column_statistics: Statistics::unknown_column(&Arc::new(Schema::new(vec![ + Field::new("col", DataType::Utf8, false), + ]))), }, projection: None, limit: None, diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index 4e25d4952a9a..d005a318b0b2 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -104,9 +104,7 @@ pub async fn from_substrait_rel( object_store_url: ObjectStoreUrl::local_filesystem(), file_schema: Arc::new(Schema::empty()), file_groups, - statistics: Statistics::new_with_unbounded_columns( - &Schema::empty(), - ), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], diff --git a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs index 49b20cc17877..1a9bb21a93a1 100644 --- a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs @@ -42,7 +42,7 @@ async fn parquet_exec() -> Result<()> { 123, )], ], - statistics: Statistics::new_with_unbounded_columns(&Schema::empty()), + statistics: Statistics::new_unknown(&Schema::empty()), projection: None, limit: None, table_partition_cols: vec![], From a104fabcd93e3438e6883cf2a4c39b60d5be99b4 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 10 Oct 2023 15:51:30 +0300 Subject: [PATCH 35/53] Cardinality calculation is fixed --- .../src/intervals/interval_aritmetic.rs | 71 ++++++++++++------- 1 file changed, 47 insertions(+), 24 deletions(-) diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index b40f6b496a02..7908e8e98668 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -518,6 +518,7 @@ impl Interval { /// - If any of the bounds is an infinite bound /// - If the type is not implemented yet /// - If there is an overflow during a computation + /// In case of a malformed interval, the function returns an error. pub fn cardinality(&self) -> Result> { let data_type = self.get_datatype()?; if data_type.is_integer() { @@ -534,34 +535,53 @@ impl Interval { // binary representations as "indices" and subtract them. For details, see: // https://stackoverflow.com/questions/8875064/how-many-distinct-floating-point-numbers-in-a-specific-range else if data_type.is_floating() { - // If the minimum value is a negative number, we need to - // switch sides to ensure an unsigned result. - let new_zero = ScalarValue::new_zero(&self.lower.value.data_type())?; - let (min, max) = if self.lower.value < new_zero { - (self.upper.value.clone(), self.lower.value.clone()) - } else { - (self.lower.value.clone(), self.upper.value.clone()) - }; - - Ok(match (min, max) { + match (&self.lower.value, &self.upper.value) { ( ScalarValue::Float32(Some(lower)), ScalarValue::Float32(Some(upper)), - ) => Some(calculate_cardinality_based_on_bounds( - self.lower.open, - self.upper.open, - (upper.to_bits().sub_checked(lower.to_bits())?) as u64, - )), + ) => { + // Negative numbers are sorted in the reverse order. To always have a positive difference after the subtraction, + // we perform following transformation: + let transformed_lower = (lower.to_bits() as i32) + ^ (((lower.to_bits() as i32) >> 31) & 0x7fffffff); + let transformed_upper = (upper.to_bits() as i32) + ^ (((upper.to_bits() as i32) >> 31) & 0x7fffffff); + let diff = if let Ok(result) = + transformed_upper.sub_checked(transformed_lower) + { + result + } else { + return Ok(None); + }; + Ok(Some(calculate_cardinality_based_on_bounds( + self.lower.open, + self.upper.open, + diff as u64, + ))) + } ( ScalarValue::Float64(Some(lower)), ScalarValue::Float64(Some(upper)), - ) => Some(calculate_cardinality_based_on_bounds( - self.lower.open, - self.upper.open, - upper.to_bits().sub_checked(lower.to_bits())?, - )), - _ => None, - }) + ) => { + let transformed_lower = (lower.to_bits() as i64) + ^ (((lower.to_bits() as i64) >> 63) & 0x7fffffffffffffff); + let transformed_upper = (upper.to_bits() as i64) + ^ (((upper.to_bits() as i64) >> 63) & 0x7fffffffffffffff); + let diff = if let Ok(result) = + transformed_upper.sub_checked(transformed_lower) + { + result + } else { + return Ok(None); + }; + Ok(Some(calculate_cardinality_based_on_bounds( + self.lower.open, + self.upper.open, + diff as u64, + ))) + } + _ => Ok(None), + } } else { Ok(None) } @@ -1718,17 +1738,20 @@ mod tests { assert_eq!(interval.cardinality()?.unwrap(), distinct_f32); } + // If the floating numbers has showned a homogeneous distribution pattern, the result would to be + // (distinct_f64 * 2_048 = 9223372036854775808); however, due to subnormal numbers around 0, + // the result will be a specific value, close to the expected one. let interval = Interval::new( IntervalBound::new(ScalarValue::from(-0.0625), false), IntervalBound::new(ScalarValue::from(0.0625), true), ); - assert_eq!(interval.cardinality()?.unwrap(), distinct_f64 * 2_048); + assert_eq!(interval.cardinality()?.unwrap(), 9178336040581070849); let interval = Interval::new( IntervalBound::new(ScalarValue::from(-0.0625_f32), false), IntervalBound::new(ScalarValue::from(0.0625_f32), true), ); - assert_eq!(interval.cardinality()?.unwrap(), distinct_f32 * 256); + assert_eq!(interval.cardinality()?.unwrap(), 2063597569); Ok(()) } From 9bb034e65ea4b172e0da083b252698afc0e9c162 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 11 Oct 2023 00:53:02 +0300 Subject: [PATCH 36/53] Review Part 2 --- .../enforce_distribution.rs | 2 +- .../src/physical_optimizer/join_selection.rs | 12 +- .../physical_optimizer/pipeline_checker.rs | 2 +- .../src/intervals/interval_aritmetic.rs | 53 ++++---- .../physical-expr/src/intervals/utils.rs | 6 +- .../physical-plan/src/aggregates/mod.rs | 25 ++-- datafusion/physical-plan/src/analyze.rs | 16 +-- .../physical-plan/src/coalesce_batches.rs | 8 +- datafusion/physical-plan/src/common.rs | 14 +- datafusion/physical-plan/src/display.rs | 7 +- datafusion/physical-plan/src/empty.rs | 10 +- datafusion/physical-plan/src/explain.rs | 15 +- datafusion/physical-plan/src/filter.rs | 14 +- datafusion/physical-plan/src/insert.rs | 21 +-- .../physical-plan/src/joins/cross_join.rs | 26 ++-- .../src/joins/nested_loop_join.rs | 34 ++--- .../src/joins/sort_merge_join.rs | 20 ++- .../src/joins/symmetric_hash_join.rs | 55 ++++---- datafusion/physical-plan/src/joins/utils.rs | 87 ++++++------ datafusion/physical-plan/src/lib.rs | 128 +++++++++--------- datafusion/physical-plan/src/limit.rs | 20 ++- datafusion/physical-plan/src/memory.rs | 21 +-- datafusion/physical-plan/src/projection.rs | 69 ++++------ datafusion/physical-plan/src/sorts/cursor.rs | 5 +- datafusion/physical-plan/src/sorts/sort.rs | 23 ++-- .../src/sorts/sort_preserving_merge.rs | 19 ++- datafusion/physical-plan/src/streaming.rs | 17 ++- datafusion/physical-plan/src/test/exec.rs | 18 ++- datafusion/physical-plan/src/union.rs | 27 ++-- datafusion/physical-plan/src/unnest.rs | 30 ++-- datafusion/physical-plan/src/values.rs | 10 +- .../src/windows/bounded_window_agg_exec.rs | 8 +- .../proto/src/physical_plan/from_proto.rs | 49 +++---- .../proto/src/physical_plan/to_proto.rs | 59 +++----- .../tests/cases/roundtrip_physical_plan.rs | 22 +-- .../substrait/src/physical_plan/consumer.rs | 10 +- .../tests/cases/roundtrip_physical_plan.rs | 6 +- 37 files changed, 458 insertions(+), 510 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index f42fc5a2d57c..15bfa97e5176 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -40,7 +40,6 @@ use crate::physical_plan::joins::{ }; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::repartition::RepartitionExec; -use crate::physical_plan::sorts::sort::SortOptions; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::union::{can_interleave, InterleaveExec, UnionExec}; use crate::physical_plan::windows::WindowAggExec; @@ -48,6 +47,7 @@ use crate::physical_plan::{ with_new_children_if_necessary, Distribution, ExecutionPlan, Partitioning, }; +use arrow::compute::SortOptions; use datafusion_common::stats::Sharpness; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_expr::logical_plan::JoinType; diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 4869e2c55055..331083dafe68 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -1224,33 +1224,33 @@ mod util_tests { Operator::Plus, Arc::new(Column::new("a", 0)), )) as Arc; - assert!(check_support(&supported_expr, schema.clone())); + assert!(check_support(&supported_expr, &schema)); let supported_expr_2 = Arc::new(Column::new("a", 0)) as Arc; - assert!(check_support(&supported_expr_2, schema.clone())); + assert!(check_support(&supported_expr_2, &schema)); let unsupported_expr = Arc::new(BinaryExpr::new( Arc::new(Column::new("a", 0)), Operator::Or, Arc::new(Column::new("a", 0)), )) as Arc; - assert!(!check_support(&unsupported_expr, schema.clone())); + assert!(!check_support(&unsupported_expr, &schema)); let unsupported_expr_2 = Arc::new(BinaryExpr::new( Arc::new(Column::new("a", 0)), Operator::Or, Arc::new(NegativeExpr::new(Arc::new(Column::new("a", 0)))), )) as Arc; - assert!(!check_support(&unsupported_expr_2, schema.clone())); + assert!(!check_support(&unsupported_expr_2, &schema)); let unsupported_expr_3 = Arc::new(BinaryExpr::new( Arc::new(Column::new("b", 1)), Operator::Eq, Arc::new(Literal::new(ScalarValue::Int16(Some(-1)))), )) as Arc; - assert!(!check_support(&unsupported_expr_3, schema.clone())); + assert!(!check_support(&unsupported_expr_3, &schema)); let unsupported_expr_4 = Arc::new(BinaryExpr::new( Arc::new(Column::new("c", 2)), Operator::Eq, Arc::new(Literal::new(ScalarValue::Int16(Some(-1)))), )) as Arc; - assert!(!check_support(&unsupported_expr_4, schema.clone())); + assert!(!check_support(&unsupported_expr_4, &schema)); } } diff --git a/datafusion/core/src/physical_optimizer/pipeline_checker.rs b/datafusion/core/src/physical_optimizer/pipeline_checker.rs index 2541661269b2..43ae7dbfe7b6 100644 --- a/datafusion/core/src/physical_optimizer/pipeline_checker.rs +++ b/datafusion/core/src/physical_optimizer/pipeline_checker.rs @@ -165,7 +165,7 @@ pub fn check_finiteness_requirements( /// [`Operator`]: datafusion_expr::Operator fn is_prunable(join: &SymmetricHashJoinExec) -> bool { join.filter().map_or(false, |filter| { - check_support(filter.expression(), join.schema()) + check_support(filter.expression(), &join.schema()) && filter .schema() .fields() diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index 7908e8e98668..44bf5481c44b 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -513,12 +513,14 @@ impl Interval { upper: IntervalBound::new_closed(ScalarValue::Boolean(Some(true))), }; - /// Returns the cardinality of this interval which is the number of all - /// distinct points inside it. Under these conditions the function return None: - /// - If any of the bounds is an infinite bound - /// - If the type is not implemented yet - /// - If there is an overflow during a computation - /// In case of a malformed interval, the function returns an error. + /// Returns the cardinality of this interval, which is the number of all + /// distinct points inside it. This function returns `None` if: + /// - The interval is unbounded from either side, or + /// - Cardinality calculations for the datatype in question is not + /// implemented yet, or + /// - An overflow occurs during the calculation. + /// + /// This function returns an error if the given interval is malformed. pub fn cardinality(&self) -> Result> { let data_type = self.get_datatype()?; if data_type.is_integer() { @@ -542,47 +544,46 @@ impl Interval { ) => { // Negative numbers are sorted in the reverse order. To always have a positive difference after the subtraction, // we perform following transformation: - let transformed_lower = (lower.to_bits() as i32) - ^ (((lower.to_bits() as i32) >> 31) & 0x7fffffff); - let transformed_upper = (upper.to_bits() as i32) - ^ (((upper.to_bits() as i32) >> 31) & 0x7fffffff); - let diff = if let Ok(result) = - transformed_upper.sub_checked(transformed_lower) - { - result - } else { + let lower_bits = lower.to_bits() as i32; + let upper_bits = upper.to_bits() as i32; + let transformed_lower = + lower_bits ^ ((lower_bits >> 31) & 0x7fffffff); + let transformed_upper = + upper_bits ^ ((upper_bits >> 31) & 0x7fffffff); + let Ok(count) = transformed_upper.sub_checked(transformed_lower) + else { return Ok(None); }; Ok(Some(calculate_cardinality_based_on_bounds( self.lower.open, self.upper.open, - diff as u64, + count as u64, ))) } ( ScalarValue::Float64(Some(lower)), ScalarValue::Float64(Some(upper)), ) => { - let transformed_lower = (lower.to_bits() as i64) - ^ (((lower.to_bits() as i64) >> 63) & 0x7fffffffffffffff); - let transformed_upper = (upper.to_bits() as i64) - ^ (((upper.to_bits() as i64) >> 63) & 0x7fffffffffffffff); - let diff = if let Ok(result) = - transformed_upper.sub_checked(transformed_lower) - { - result - } else { + let lower_bits = lower.to_bits() as i64; + let upper_bits = upper.to_bits() as i64; + let transformed_lower = + lower_bits ^ ((lower_bits >> 63) & 0x7fffffffffffffff); + let transformed_upper = + upper_bits ^ ((upper_bits >> 63) & 0x7fffffffffffffff); + let Ok(count) = transformed_upper.sub_checked(transformed_lower) + else { return Ok(None); }; Ok(Some(calculate_cardinality_based_on_bounds( self.lower.open, self.upper.open, - diff as u64, + count as u64, ))) } _ => Ok(None), } } else { + // Cardinality calculations are not implemented for this data type yet: Ok(None) } } diff --git a/datafusion/physical-expr/src/intervals/utils.rs b/datafusion/physical-expr/src/intervals/utils.rs index 41d9c1672b0e..7a4ccff950e6 100644 --- a/datafusion/physical-expr/src/intervals/utils.rs +++ b/datafusion/physical-expr/src/intervals/utils.rs @@ -38,11 +38,11 @@ const DT_MS_MASK: i64 = 0xFFFF_FFFF; /// We do not support every type of [`Operator`]s either. Over time, this check /// will relax as more types of `PhysicalExpr`s and `Operator`s are supported. /// Currently, [`CastExpr`], [`NegativeExpr`], [`BinaryExpr`], [`Column`] and [`Literal`] are supported. -pub fn check_support(expr: &Arc, schema: SchemaRef) -> bool { +pub fn check_support(expr: &Arc, schema: &SchemaRef) -> bool { let expr_any = expr.as_any(); if let Some(binary_expr) = expr_any.downcast_ref::() { is_operator_supported(binary_expr.op()) - && check_support(binary_expr.left(), schema.clone()) + && check_support(binary_expr.left(), schema) && check_support(binary_expr.right(), schema) } else if let Some(column) = expr_any.downcast_ref::() { if let Ok(field) = schema.field_with_name(column.name()) { @@ -51,7 +51,7 @@ pub fn check_support(expr: &Arc, schema: SchemaRef) -> bool { return false; } } else if let Some(literal) = expr_any.downcast_ref::() { - if let Ok(dt) = literal.data_type(&schema) { + if let Ok(dt) = literal.data_type(schema) { is_datatype_supported(&dt) } else { return false; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 6852d3490717..6d0391de51c1 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -17,8 +17,14 @@ //! Aggregates functionalities +use std::any::Any; +use std::collections::HashMap; +use std::sync::Arc; + +use super::DisplayAs; use crate::aggregates::{ no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, + topk_stream::GroupedTopKAggregateStream, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ @@ -34,19 +40,20 @@ use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; +use datafusion_physical_expr::utils::{ + convert_to_expr, get_finer_ordering, get_indices_of_matching_exprs, + ordering_satisfy_requirement_concrete, +}; use datafusion_physical_expr::{ + aggregate::is_order_sensitive, equivalence::project_equivalence_properties, - expressions::Column, + expressions::{Column, Max, Min}, normalize_out_expr_with_columns_map, physical_exprs_contains, reverse_order_bys, - utils::{convert_to_expr, get_indices_of_matching_exprs}, AggregateExpr, LexOrdering, LexOrderingReq, OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; use itertools::Itertools; -use std::any::Any; -use std::collections::HashMap; -use std::sync::Arc; mod group_values; mod no_grouping; @@ -55,16 +62,8 @@ mod row_hash; mod topk; mod topk_stream; -use crate::aggregates::topk_stream::GroupedTopKAggregateStream; pub use datafusion_expr::AggregateFunction; -use datafusion_physical_expr::aggregate::is_order_sensitive; pub use datafusion_physical_expr::expressions::create_aggregate_expr; -use datafusion_physical_expr::expressions::{Max, Min}; -use datafusion_physical_expr::utils::{ - get_finer_ordering, ordering_satisfy_requirement_concrete, -}; - -use super::DisplayAs; /// Hash aggregate modes #[derive(Debug, Copy, Clone, PartialEq, Eq)] diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index 9652464b3d8c..bce242513559 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -20,19 +20,19 @@ use std::sync::Arc; use std::{any::Any, time::Instant}; -use crate::{ - display::DisplayableExecutionPlan, DisplayFormatType, ExecutionPlan, Partitioning, - Statistics, -}; -use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; -use datafusion_common::{internal_err, DataFusionError, Result}; -use futures::StreamExt; - use super::expressions::PhysicalSortExpr; use super::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use super::{DisplayAs, Distribution, SendableRecordBatchStream}; + +use crate::display::DisplayableExecutionPlan; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; + +use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; +use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; +use futures::StreamExt; + /// `EXPLAIN ANALYZE` execution plan operator. This operator runs its input, /// discards the results, and then prints out an annotated plan with metrics #[derive(Debug, Clone)] diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index c80b743cf917..df9e8a8a2b8c 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -23,6 +23,9 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; +use super::expressions::PhysicalSortExpr; +use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use super::{DisplayAs, Statistics}; use crate::{ DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, @@ -35,11 +38,6 @@ use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::OrderingEquivalenceProperties; -use super::expressions::PhysicalSortExpr; -use super::metrics::{BaselineMetrics, MetricsSet}; -use super::DisplayAs; -use super::{metrics::ExecutionPlanMetricsSet, Statistics}; - use futures::stream::{Stream, StreamExt}; use log::trace; diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index aa07189e6ff3..37cfcd28ce5c 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -17,9 +17,16 @@ //! Defines common code used in execution plans +use std::fs; +use std::fs::{metadata, File}; +use std::path::{Path, PathBuf}; +use std::sync::Arc; +use std::task::{Context, Poll}; + use super::SendableRecordBatchStream; use crate::stream::RecordBatchReceiverStream; use crate::{ColumnStatistics, ExecutionPlan, Statistics}; + use arrow::datatypes::Schema; use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; use arrow::record_batch::RecordBatch; @@ -28,14 +35,10 @@ use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; use datafusion_physical_expr::expressions::{BinaryExpr, Column}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; + use futures::{Future, StreamExt, TryStreamExt}; use parking_lot::Mutex; use pin_project_lite::pin_project; -use std::fs; -use std::fs::{metadata, File}; -use std::path::{Path, PathBuf}; -use std::sync::Arc; -use std::task::{Context, Poll}; use tokio::task::JoinHandle; /// [`MemoryReservation`] used across query execution streams @@ -378,6 +381,7 @@ mod tests { use crate::memory::MemoryExec; use crate::sorts::sort::SortExec; use crate::union::UnionExec; + use arrow::compute::SortOptions; use arrow::{ array::{Float32Array, Float64Array}, diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index 010e19f59990..aa368251ebf3 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -20,13 +20,12 @@ use std::fmt; +use super::{accept, ExecutionPlan, ExecutionPlanVisitor}; + use arrow_schema::SchemaRef; -use datafusion_common::display::StringifiedPlan; +use datafusion_common::display::{GraphvizBuilder, PlanType, StringifiedPlan}; use datafusion_physical_expr::PhysicalSortExpr; -use super::{accept, ExecutionPlan, ExecutionPlanVisitor}; -use datafusion_common::display::{GraphvizBuilder, PlanType}; - /// Options for controlling how each [`ExecutionPlan`] should format itself #[derive(Debug, Clone, Copy)] pub enum DisplayFormatType { diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index 82314905b588..a3e1fb79edb5 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -20,18 +20,18 @@ use std::any::Any; use std::sync::Arc; +use super::expressions::PhysicalSortExpr; +use super::{common, DisplayAs, SendableRecordBatchStream, Statistics}; use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning}; + use arrow::array::{ArrayRef, NullArray}; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, DataFusionError, Result}; -use log::trace; - -use super::expressions::PhysicalSortExpr; -use super::{common, DisplayAs, SendableRecordBatchStream, Statistics}; - use datafusion_execution::TaskContext; +use log::trace; + /// Execution plan for empty relation (produces no rows) #[derive(Debug)] pub struct EmptyExec { diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index 6aa4b8f4327a..81b8f9944110 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -20,19 +20,18 @@ use std::any::Any; use std::sync::Arc; -use datafusion_common::display::StringifiedPlan; +use super::expressions::PhysicalSortExpr; +use super::{DisplayAs, SendableRecordBatchStream}; +use crate::stream::RecordBatchStreamAdapter; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; +use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; +use datafusion_common::display::StringifiedPlan; use datafusion_common::{internal_err, DataFusionError, Result}; +use datafusion_execution::TaskContext; -use crate::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; -use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; use log::trace; -use super::DisplayAs; -use super::{expressions::PhysicalSortExpr, SendableRecordBatchStream}; -use crate::stream::RecordBatchStreamAdapter; -use datafusion_execution::TaskContext; - /// Explain execution plan operator. This operator contains the string /// values of the various plans it has when it is created, and passes /// them to its output. diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 27b97757de49..b63c9751a171 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -49,6 +49,7 @@ use datafusion_physical_expr::{ use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::utils::collect_columns; + use futures::stream::{Stream, StreamExt}; use log::trace; @@ -202,8 +203,9 @@ impl ExecutionPlan for FilterExec { fn statistics(&self) -> Result { let predicate = self.predicate(); - if !check_support(predicate, self.schema()) { - return Ok(Statistics::new_unknown(&self.schema())); + let schema = self.schema(); + if !check_support(predicate, &schema) { + return Ok(Statistics::new_unknown(&schema)); } let input_stats = self.input.statistics()?; @@ -392,18 +394,18 @@ pub type EqualAndNonEqual<'a> = #[cfg(test)] mod tests { + use std::iter::Iterator; + use std::sync::Arc; use super::*; use crate::expressions::*; use crate::test; use crate::test::exec::StatisticsExec; use crate::ExecutionPlan; + use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::ColumnStatistics; - use datafusion_common::ScalarValue; + use datafusion_common::{ColumnStatistics, ScalarValue}; use datafusion_expr::Operator; - use std::iter::Iterator; - use std::sync::Arc; #[tokio::test] async fn collect_columns_predicates() -> Result<()> { diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index 4c0f7ada790c..b1db3661afe4 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -17,27 +17,28 @@ //! Execution plan for writing data to [`DataSink`]s +use std::any::Any; +use std::fmt; +use std::fmt::Debug; +use std::sync::Arc; + use super::expressions::PhysicalSortExpr; use super::{ DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; +use crate::stream::RecordBatchStreamAdapter; + use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use arrow_array::{ArrayRef, UInt64Array}; use arrow_schema::{DataType, Field, Schema}; -use async_trait::async_trait; -use core::fmt; -use datafusion_common::Result; +use datafusion_common::{exec_err, internal_err, DataFusionError, Result}; +use datafusion_execution::TaskContext; use datafusion_physical_expr::PhysicalSortRequirement; -use futures::StreamExt; -use std::any::Any; -use std::fmt::Debug; -use std::sync::Arc; -use crate::stream::RecordBatchStreamAdapter; -use datafusion_common::{exec_err, internal_err, DataFusionError}; -use datafusion_execution::TaskContext; +use async_trait::async_trait; +use futures::StreamExt; /// `DataSink` implements writing streams of [`RecordBatch`]es to /// user defined destinations. diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index e05b762b13e7..8637da252cab 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -18,14 +18,12 @@ //! Defines the cross join plan for loading the left side of the cross join //! and producing batches in parallel for the right partitions -use datafusion_common::stats::Sharpness; -use futures::{ready, StreamExt}; -use futures::{Stream, TryStreamExt}; use std::{any::Any, sync::Arc, task::Poll}; -use arrow::datatypes::{Fields, Schema, SchemaRef}; -use arrow::record_batch::RecordBatch; - +use super::utils::{ + adjust_right_output_partitioning, cross_join_equivalence_properties, + BuildProbeJoinMetrics, OnceAsync, OnceFut, +}; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::DisplayAs; use crate::{ @@ -34,16 +32,17 @@ use crate::{ ExecutionPlan, Partitioning, PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use async_trait::async_trait; -use datafusion_common::{plan_err, DataFusionError}; -use datafusion_common::{Result, ScalarValue}; + +use arrow::datatypes::{Fields, Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; +use datafusion_common::stats::Sharpness; +use datafusion_common::{plan_err, DataFusionError, Result, ScalarValue}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; -use super::utils::{ - adjust_right_output_partitioning, cross_join_equivalence_properties, - BuildProbeJoinMetrics, OnceAsync, OnceFut, -}; +use async_trait::async_trait; +use futures::{ready, StreamExt}; +use futures::{Stream, TryStreamExt}; /// Data of the left side type JoinLeftData = (RecordBatch, MemoryReservation); @@ -438,6 +437,7 @@ mod tests { use super::*; use crate::common; use crate::test::build_table_scan_i32; + use datafusion_common::{assert_batches_sorted_eq, assert_contains}; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 368ae825caab..944efb47f429 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -19,6 +19,12 @@ //! The nested loop join can execute in parallel by partitions and it is //! determined by the [`JoinType`]. +use std::any::Any; +use std::fmt::Formatter; +use std::sync::Arc; +use std::task::Poll; + +use crate::coalesce_batches::concat_batches; use crate::joins::utils::{ append_right_indices, apply_join_filter_to_indices, build_batch_from_indices, build_join_schema, check_join_is_valid, combine_join_equivalence_properties, @@ -32,26 +38,20 @@ use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, }; + use arrow::array::{ BooleanBufferBuilder, UInt32Array, UInt32Builder, UInt64Array, UInt64Builder, }; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow::util::bit_util; -use datafusion_common::{exec_err, DataFusionError, Statistics}; -use datafusion_execution::memory_pool::MemoryReservation; +use datafusion_common::{exec_err, DataFusionError, Result, Statistics}; +use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use datafusion_execution::TaskContext; use datafusion_expr::JoinType; use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortExpr}; -use futures::{ready, Stream, StreamExt, TryStreamExt}; -use std::any::Any; -use std::fmt::Formatter; -use std::sync::Arc; -use std::task::Poll; -use crate::coalesce_batches::concat_batches; -use datafusion_common::Result; -use datafusion_execution::memory_pool::MemoryConsumer; -use datafusion_execution::TaskContext; +use futures::{ready, Stream, StreamExt, TryStreamExt}; /// Data of the inner table side type JoinLeftData = (RecordBatch, MemoryReservation); @@ -740,21 +740,21 @@ impl RecordBatchStream for NestedLoopJoinStream { #[cfg(test)] mod tests { + use std::sync::Arc; + use super::*; + use crate::joins::utils::JoinSide; use crate::{ common, expressions::Column, memory::MemoryExec, repartition::RepartitionExec, test::build_table_i32, }; + use arrow::datatypes::{DataType, Field}; + use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::BinaryExpr; - - use crate::joins::utils::JoinSide; - use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; - use datafusion_physical_expr::expressions::Literal; + use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; use datafusion_physical_expr::PhysicalExpr; - use std::sync::Arc; fn build_table( a: (&str, &Vec), diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index dd74abe7713c..61ac864bf20d 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -30,8 +30,7 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::expressions::Column; -use crate::expressions::PhysicalSortExpr; +use crate::expressions::{Column, PhysicalSortExpr}; use crate::joins::utils::{ build_join_schema, calculate_join_output_ordering, check_join_is_valid, combine_join_equivalence_properties, combine_join_ordering_equivalence_properties, @@ -1398,24 +1397,23 @@ fn is_join_arrays_equal( mod tests { use std::sync::Arc; - use arrow::array::{Date32Array, Date64Array, Int32Array}; - use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, Schema}; - use arrow::record_batch::RecordBatch; - use datafusion_execution::config::SessionConfig; - use datafusion_execution::TaskContext; - use crate::expressions::Column; use crate::joins::utils::JoinOn; use crate::joins::SortMergeJoinExec; use crate::memory::MemoryExec; use crate::test::build_table_i32; use crate::{common, ExecutionPlan}; - use datafusion_common::Result; + + use arrow::array::{Date32Array, Date64Array, Int32Array}; + use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow::record_batch::RecordBatch; use datafusion_common::{ - assert_batches_eq, assert_batches_sorted_eq, assert_contains, JoinType, + assert_batches_eq, assert_batches_sorted_eq, assert_contains, JoinType, Result, }; + use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_execution::TaskContext; fn build_table( a: (&str, &Vec), diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 5758cbc4c08e..64128b6e3a12 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -18,11 +18,11 @@ //! This file implements the symmetric hash join algorithm with range-based //! data pruning to join two (potentially infinite) streams. //! -//! A [SymmetricHashJoinExec] plan takes two children plan (with appropriate +//! A [`SymmetricHashJoinExec`] plan takes two children plan (with appropriate //! output ordering) and produces the join output according to the given join //! type and other options. //! -//! This plan uses the [OneSideHashJoiner] object to facilitate join calculations +//! This plan uses the [`OneSideHashJoiner`] object to facilitate join calculations //! for both its children. use std::fmt; @@ -38,23 +38,19 @@ use crate::joins::hash_join_utils::{ calculate_filter_expr_intervals, combine_two_batches, convert_sort_expr_with_filter_schema, get_pruning_anti_indices, get_pruning_semi_indices, record_visited_indices, PruningJoinHashMap, + SortedFilterExpr, +}; +use crate::joins::utils::{ + build_batch_from_indices, build_join_schema, check_join_is_valid, + combine_join_equivalence_properties, partitioned_join_output_partitioning, + prepare_sorted_exprs, ColumnIndex, JoinFilter, JoinOn, JoinSide, }; -use crate::joins::StreamJoinPartitionMode; -use crate::DisplayAs; use crate::{ - expressions::Column, - expressions::PhysicalSortExpr, - joins::{ - hash_join_utils::SortedFilterExpr, - utils::{ - build_batch_from_indices, build_join_schema, check_join_is_valid, - combine_join_equivalence_properties, partitioned_join_output_partitioning, - ColumnIndex, JoinFilter, JoinOn, JoinSide, - }, - }, + expressions::{Column, PhysicalSortExpr}, + joins::StreamJoinPartitionMode, metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, - DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, - RecordBatchStream, SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, + Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray, PrimitiveBuilder}; @@ -62,13 +58,11 @@ use arrow::compute::concat_batches; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::utils::bisect; -use datafusion_common::{internal_err, plan_err, JoinType}; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{internal_err, plan_err, DataFusionError, JoinType, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::intervals::ExprIntervalGraph; -use crate::joins::utils::prepare_sorted_exprs; use ahash::RandomState; use futures::stream::{select, BoxStream}; use futures::{Stream, StreamExt}; @@ -1204,17 +1198,11 @@ impl SymmetricHashJoinStream { #[cfg(test)] mod tests { - use super::*; - use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, IntervalUnit, Schema, TimeUnit}; - use datafusion_execution::config::SessionConfig; - use rstest::*; - - use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::{binary, col, Column}; + use std::collections::HashMap; + use std::sync::Mutex; + use super::*; use crate::joins::hash_join_utils::tests::complicated_filter; - use crate::joins::test_utils::{ build_sides_record_batches, compare_batches, create_memory_table, join_expr_tests_fixture_f64, join_expr_tests_fixture_i32, @@ -1222,11 +1210,16 @@ mod tests { partitioned_sym_join_with_filter, split_record_batches, }; - const TABLE_SIZE: i32 = 30; + use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field, IntervalUnit, Schema, TimeUnit}; + use datafusion_execution::config::SessionConfig; + use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::{binary, col, Column}; use once_cell::sync::Lazy; - use std::collections::HashMap; - use std::sync::Mutex; + use rstest::*; + + const TABLE_SIZE: i32 = 30; type TableKey = (i32, i32, usize); // (cardinality.0, cardinality.1, batch_size) type TableValue = (Vec, Vec); // (left, right) diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 912835ca67cd..1903eb3f560a 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -24,10 +24,11 @@ use std::sync::Arc; use std::task::{Context, Poll}; use std::usize; +use crate::joins::hash_join_utils::{build_filter_input_order, SortedFilterExpr}; use crate::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder}; -use crate::SchemaRef; use crate::{ - ColumnStatistics, EquivalenceProperties, ExecutionPlan, Partitioning, Statistics, + ColumnStatistics, EquivalenceProperties, ExecutionPlan, Partitioning, SchemaRef, + Statistics, }; use arrow::array::{ @@ -44,15 +45,14 @@ use datafusion_common::{ exec_err, plan_err, DataFusionError, JoinType, Result, ScalarValue, SharedResult, }; use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::intervals::ExprIntervalGraph; +use datafusion_physical_expr::utils::merge_vectors; use datafusion_physical_expr::{ add_offset_to_lex_ordering, EquivalentClass, LexOrdering, LexOrderingRef, OrderingEquivalenceProperties, OrderingEquivalentClass, PhysicalExpr, PhysicalSortExpr, }; -use crate::joins::hash_join_utils::{build_filter_input_order, SortedFilterExpr}; -use datafusion_physical_expr::intervals::ExprIntervalGraph; -use datafusion_physical_expr::utils::merge_vectors; use futures::future::{BoxFuture, Shared}; use futures::{ready, FutureExt}; use parking_lot::Mutex; @@ -827,26 +827,30 @@ fn estimate_inner_join_cardinality( .iter() .zip(right_stats.column_statistics.iter()) { - // If there is no overlap in any of the join columns, that means the join + // If there is no overlap in any of the join columns, this means the join // itself is disjoint and the cardinality is 0. Though we can only assume // this when the statistics are exact (since it is a very strong assumption). if left_stat.min_value.get_value()? > right_stat.max_value.get_value()? { - return match ( - left_stat.min_value.is_exact().unwrap_or(false), - right_stat.max_value.is_exact().unwrap_or(false), - ) { - (true, true) => Some(Sharpness::Exact(0)), - _ => Some(Sharpness::Inexact(0)), - }; + return Some( + if left_stat.min_value.is_exact().unwrap_or(false) + && right_stat.max_value.is_exact().unwrap_or(false) + { + Sharpness::Exact(0) + } else { + Sharpness::Inexact(0) + }, + ); } if left_stat.max_value.get_value()? < right_stat.min_value.get_value()? { - return match ( - left_stat.max_value.is_exact().unwrap_or(false), - right_stat.min_value.is_exact().unwrap_or(false), - ) { - (true, true) => Some(Sharpness::Exact(0)), - _ => Some(Sharpness::Inexact(0)), - }; + return Some( + if left_stat.max_value.is_exact().unwrap_or(false) + && right_stat.min_value.is_exact().unwrap_or(false) + { + Sharpness::Exact(0) + } else { + Sharpness::Inexact(0) + }, + ); } let left_max_distinct = max_distinct_count(&left_stats.num_rows, left_stat)?; @@ -866,11 +870,11 @@ fn estimate_inner_join_cardinality( let left_num_rows = left_stats.num_rows.get_value()?; let right_num_rows = right_stats.num_rows.get_value()?; match join_selectivity { - Sharpness::Exact(val) if val > 0 => { - Some(Sharpness::Exact((left_num_rows * right_num_rows) / val)) + Sharpness::Exact(value) if value > 0 => { + Some(Sharpness::Exact((left_num_rows * right_num_rows) / value)) } - Sharpness::Inexact(val) if val > 0 => { - Some(Sharpness::Inexact((left_num_rows * right_num_rows) / val)) + Sharpness::Inexact(value) if value > 0 => { + Some(Sharpness::Inexact((left_num_rows * right_num_rows) / value)) } // Since we don't have any information about the selectivity (which is derived // from the number of distinct rows information) we can give up here for now. @@ -1419,13 +1423,15 @@ pub fn prepare_sorted_exprs( #[cfg(test)] mod tests { + use std::pin::Pin; + use super::*; - use arrow::datatypes::Fields; - use arrow::error::Result as ArrowResult; - use arrow::{datatypes::DataType, error::ArrowError}; + + use arrow::datatypes::{DataType, Fields}; + use arrow::error::{ArrowError, Result as ArrowResult}; use arrow_schema::SortOptions; + use datafusion_common::ScalarValue; - use std::pin::Pin; fn check(left: &[Column], right: &[Column], on: &[(Column, Column)]) -> Result<()> { let left = left @@ -1581,20 +1587,15 @@ mod tests { column_stats: Vec, is_exact: bool, ) -> Statistics { - if is_exact { - Statistics { - num_rows: num_rows.map(Sharpness::Exact).unwrap_or(Sharpness::Absent), - column_statistics: column_stats, - total_byte_size: Sharpness::Absent, - } - } else { - Statistics { - num_rows: num_rows - .map(Sharpness::Inexact) - .unwrap_or(Sharpness::Absent), - column_statistics: column_stats, - total_byte_size: Sharpness::Absent, + Statistics { + num_rows: if is_exact { + num_rows.map(Sharpness::Exact) + } else { + num_rows.map(Sharpness::Inexact) } + .unwrap_or(Sharpness::Absent), + column_statistics: column_stats, + total_byte_size: Sharpness::Absent, } } @@ -1608,10 +1609,10 @@ mod tests { .map(Sharpness::Inexact) .unwrap_or(Sharpness::Absent), min_value: min - .map(|size| Sharpness::Inexact(ScalarValue::Int64(Some(size)))) + .map(|size| Sharpness::Inexact(ScalarValue::from(size))) .unwrap_or(Sharpness::Absent), max_value: max - .map(|size| Sharpness::Inexact(ScalarValue::Int64(Some(size)))) + .map(|size| Sharpness::Inexact(ScalarValue::from(size))) .unwrap_or(Sharpness::Absent), ..Default::default() } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 9620576a07d8..2252d8dc94c6 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -17,39 +17,80 @@ //! Traits for physical query plan, supporting parallel execution for partitioned relations. -mod topk; -mod visitor; -pub use self::metrics::Metric; -use self::metrics::MetricsSet; -use self::{ - coalesce_partitions::CoalescePartitionsExec, display::DisplayableExecutionPlan, -}; -pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; -use datafusion_common::{plan_err, Result}; -use datafusion_physical_expr::PhysicalSortExpr; -pub use topk::TopK; -pub use visitor::{accept, visit_execution_plan, ExecutionPlanVisitor}; +use std::any::Any; +use std::fmt::Debug; +use std::sync::Arc; + +use crate::coalesce_partitions::CoalescePartitionsExec; +use crate::display::DisplayableExecutionPlan; +use crate::metrics::MetricsSet; +use crate::repartition::RepartitionExec; +use crate::sorts::sort_preserving_merge::SortPreservingMergeExec; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; - +use datafusion_common::tree_node::Transformed; use datafusion_common::utils::DataPtr; -pub use datafusion_expr::Accumulator; -pub use datafusion_expr::ColumnarValue; +use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::OrderingEquivalenceProperties; -pub use display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::{ + EquivalenceProperties, PhysicalSortExpr, PhysicalSortRequirement, +}; + use futures::stream::TryStreamExt; -use std::fmt::Debug; use tokio::task::JoinSet; -use datafusion_common::tree_node::Transformed; -use datafusion_common::DataFusionError; -use std::any::Any; -use std::sync::Arc; +// Interfaces: + +mod topk; +mod visitor; + +pub mod aggregates; +pub mod analyze; +pub mod coalesce_batches; +pub mod coalesce_partitions; +pub mod common; +pub mod display; +pub mod empty; +pub mod explain; +pub mod filter; +pub mod insert; +pub mod joins; +pub mod limit; +pub mod memory; +pub mod metrics; +pub mod projection; +pub mod repartition; +pub mod sorts; +pub mod stream; +pub mod streaming; +pub mod tree_node; +pub mod udaf; +pub mod union; +pub mod unnest; +pub mod values; +pub mod windows; + +pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; +pub use crate::metrics::Metric; +pub use crate::topk::TopK; +pub use crate::visitor::{accept, visit_execution_plan, ExecutionPlanVisitor}; + +pub use datafusion_common::hash_utils; +pub use datafusion_common::utils::project_schema; +pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; +pub use datafusion_expr::{Accumulator, ColumnarValue}; +pub use datafusion_physical_expr::window::WindowExpr; +pub use datafusion_physical_expr::{ + expressions, functions, ordering_equivalence_properties_helper, udf, AggregateExpr, + Distribution, Partitioning, PhysicalExpr, +}; -// backwards compatibility +// Backwards compatibility +pub use crate::stream::EmptyRecordBatchStream; pub use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; -pub use stream::EmptyRecordBatchStream; /// `ExecutionPlan` represent nodes in the DataFusion Physical Plan. /// @@ -343,46 +384,5 @@ pub fn unbounded_output(plan: &Arc) -> bool { .unwrap_or(true) } -use datafusion_physical_expr::expressions::Column; -pub use datafusion_physical_expr::window::WindowExpr; -pub use datafusion_physical_expr::{AggregateExpr, PhysicalExpr}; -pub use datafusion_physical_expr::{Distribution, Partitioning}; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; - -pub mod aggregates; -pub mod analyze; -pub mod coalesce_batches; -pub mod coalesce_partitions; -pub mod common; -pub mod display; -pub mod empty; -pub mod explain; -pub mod filter; -pub mod insert; -pub mod joins; -pub mod limit; -pub mod memory; -pub mod metrics; -pub mod projection; -pub mod repartition; -pub mod sorts; -pub mod stream; -pub mod streaming; -pub mod tree_node; -pub mod udaf; -pub mod union; -pub mod unnest; -pub mod values; -pub mod windows; - -use crate::repartition::RepartitionExec; -use crate::sorts::sort_preserving_merge::SortPreservingMergeExec; -pub use datafusion_common::hash_utils; -pub use datafusion_common::utils::project_schema; -use datafusion_execution::TaskContext; -pub use datafusion_physical_expr::{ - expressions, functions, ordering_equivalence_properties_helper, udf, -}; - #[cfg(test)] pub mod test; diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index f1fb8b14a456..f2e2cc93afd7 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -22,13 +22,12 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::{ - DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, -}; - use super::expressions::PhysicalSortExpr; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics}; +use crate::{ + DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, +}; use arrow::array::ArrayRef; use arrow::datatypes::SchemaRef; @@ -38,8 +37,7 @@ use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::OrderingEquivalenceProperties; -use futures::stream::Stream; -use futures::stream::StreamExt; +use futures::stream::{Stream, StreamExt}; use log::trace; /// Limit execution plan @@ -558,14 +556,12 @@ impl RecordBatchStream for LimitStream { #[cfg(test)] mod tests { - - use arrow_schema::Schema; - use common::collect; - use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; - use crate::common; - use crate::test; + use crate::common::collect; + use crate::{common, test}; + + use arrow_schema::Schema; #[tokio::test] async fn limit() -> Result<()> { diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 133ec74f5d96..dc6343c28a91 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -17,23 +17,24 @@ //! Execution plan for reading in-memory batches of data +use std::any::Any; +use std::fmt; +use std::sync::Arc; +use std::task::{Context, Poll}; + use super::expressions::PhysicalSortExpr; use super::{ common, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; +use crate::ordering_equivalence_properties_helper; + use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use core::fmt; -use datafusion_common::{internal_err, project_schema, Result}; -use std::any::Any; -use std::sync::Arc; -use std::task::{Context, Poll}; - -use crate::ordering_equivalence_properties_helper; -use datafusion_common::DataFusionError; +use datafusion_common::{internal_err, project_schema, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{LexOrdering, OrderingEquivalenceProperties}; + use futures::Stream; /// Execution plan for reading in-memory batches of data @@ -265,12 +266,14 @@ impl RecordBatchStream for MemoryStream { #[cfg(test)] mod tests { + use std::sync::Arc; + use crate::memory::MemoryExec; use crate::ExecutionPlan; + use arrow_schema::{DataType, Field, Schema, SortOptions}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; - use std::sync::Arc; #[test] fn test_memory_order_eq() -> datafusion_common::Result<()> { diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index a142972191ab..e27193aa2378 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -40,12 +40,12 @@ use datafusion_common::stats::Sharpness; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; +use datafusion_physical_expr::utils::find_orderings_of_exprs; use datafusion_physical_expr::{ normalize_out_expr_with_columns_map, project_equivalence_properties, project_ordering_equivalence_properties, OrderingEquivalenceProperties, }; -use datafusion_physical_expr::utils::find_orderings_of_exprs; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -393,54 +393,37 @@ fn get_field_metadata( } fn stats_projection( - stats: Statistics, + mut stats: Statistics, exprs: impl Iterator>, schema: SchemaRef, ) -> Statistics { - let inner_exprs = exprs.collect::>(); - let column_statistics = inner_exprs - .clone() - .into_iter() - .enumerate() - .map(|(_index, e)| { - if let Some(col) = e.as_any().downcast_ref::() { - stats.column_statistics[col.index()].clone() - } else { - // TODO stats: estimate more statistics from expressions - // (expressions should compute their statistics themselves) - ColumnStatistics::new_unknown() - } - }) - .collect(); - - let primitive_row_size = inner_exprs - .into_iter() - .map(|e| match e.data_type(schema.as_ref()) { - Ok(data_type) => data_type.primitive_width(), - Err(_) => None, - }) - .try_fold(0usize, |init, v| v.map(|value| init + value)); - - match (primitive_row_size, stats.num_rows.get_value()) { - (Some(row_size), Some(_)) => { - Statistics { - num_rows: stats.num_rows.clone(), - column_statistics, - // Use the row_size * row_count as the total byte size - total_byte_size: Sharpness::Exact(row_size).multiply(&stats.num_rows), - } - } - _ => { - Statistics { - num_rows: stats.num_rows, - column_statistics, - // TODO stats: knowing the type of the new columns we can guess the output size - // If we can't get the exact statistics for the project - // Before we get the exact result, we just use the child status - total_byte_size: stats.total_byte_size, + let mut primitive_row_size = 0; + let mut primitive_row_size_possible = true; + let mut column_statistics = vec![]; + for expr in exprs { + let col_stats = if let Some(col) = expr.as_any().downcast_ref::() { + stats.column_statistics[col.index()].clone() + } else { + // TODO stats: estimate more statistics from expressions + // (expressions should compute their statistics themselves) + ColumnStatistics::new_unknown() + }; + column_statistics.push(col_stats); + if let Ok(data_type) = expr.data_type(&schema) { + if let Some(value) = data_type.primitive_width() { + primitive_row_size += value; + continue; } } + primitive_row_size_possible = false; + } + + if primitive_row_size_possible { + stats.total_byte_size = + Sharpness::Exact(primitive_row_size).multiply(&stats.num_rows); } + stats.column_statistics = column_statistics; + stats } impl ProjectionStream { diff --git a/datafusion/physical-plan/src/sorts/cursor.rs b/datafusion/physical-plan/src/sorts/cursor.rs index baa417649fb0..ebca9aad630e 100644 --- a/datafusion/physical-plan/src/sorts/cursor.rs +++ b/datafusion/physical-plan/src/sorts/cursor.rs @@ -15,14 +15,15 @@ // specific language governing permissions and limitations // under the License. -use crate::sorts::sort::SortOptions; +use std::cmp::Ordering; + use arrow::buffer::ScalarBuffer; +use arrow::compute::SortOptions; use arrow::datatypes::ArrowNativeTypeOp; use arrow::row::{Row, Rows}; use arrow_array::types::ByteArrayType; use arrow_array::{Array, ArrowPrimitiveType, GenericByteArray, PrimitiveArray}; use datafusion_execution::memory_pool::MemoryReservation; -use std::cmp::Ordering; /// A [`Cursor`] for [`Rows`] pub struct RowCursor { diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 7a40e1d52354..4b845025b29f 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -19,6 +19,14 @@ //! It will do in-memory sorting if it has enough memory budget //! but spills to disk if needed. +use std::any::Any; +use std::fmt; +use std::fmt::{Debug, Formatter}; +use std::fs::File; +use std::io::BufReader; +use std::path::{Path, PathBuf}; +use std::sync::Arc; + use crate::common::{spawn_buffered, IPCWriter}; use crate::expressions::PhysicalSortExpr; use crate::metrics::{ @@ -31,7 +39,7 @@ use crate::{ DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; -pub use arrow::compute::SortOptions; + use arrow::compute::{concat_batches, lexsort_to_indices, take}; use arrow::datatypes::SchemaRef; use arrow::ipc::reader::FileReader; @@ -44,15 +52,9 @@ use datafusion_execution::memory_pool::{ use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; + use futures::{StreamExt, TryStreamExt}; use log::{debug, error, trace}; -use std::any::Any; -use std::fmt; -use std::fmt::{Debug, Formatter}; -use std::fs::File; -use std::io::BufReader; -use std::path::{Path, PathBuf}; -use std::sync::Arc; use tokio::sync::mpsc::Sender; use tokio::task; @@ -920,6 +922,8 @@ impl ExecutionPlan for SortExec { #[cfg(test)] mod tests { + use std::collections::HashMap; + use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::collect; @@ -928,14 +932,15 @@ mod tests { use crate::test; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; + use arrow::array::*; use arrow::compute::SortOptions; use arrow::datatypes::*; use datafusion_common::cast::as_primitive_array; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeConfig; + use futures::FutureExt; - use std::collections::HashMap; #[tokio::test] async fn test_in_mem_sort() -> Result<()> { diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index a72277367e3d..973476148b64 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -28,10 +28,10 @@ use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; -use datafusion_execution::memory_pool::MemoryConsumer; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; +use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::{ EquivalenceProperties, OrderingEquivalenceProperties, PhysicalSortRequirement, @@ -270,13 +270,7 @@ impl ExecutionPlan for SortPreservingMergeExec { mod tests { use std::iter::FromIterator; - use arrow::array::ArrayRef; - use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, Schema}; - use arrow::record_batch::RecordBatch; - use datafusion_execution::config::SessionConfig; - use futures::{FutureExt, StreamExt}; - + use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::expressions::col; use crate::memory::MemoryExec; @@ -286,10 +280,15 @@ mod tests { use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::test::{self, assert_is_pending, make_partition}; use crate::{collect, common}; - use arrow::array::{Int32Array, StringArray, TimestampNanosecondArray}; + + use arrow::array::{ArrayRef, Int32Array, StringArray, TimestampNanosecondArray}; + use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field, Schema}; + use arrow::record_batch::RecordBatch; use datafusion_common::assert_batches_eq; + use datafusion_execution::config::SessionConfig; - use super::*; + use futures::{FutureExt, StreamExt}; #[tokio::test] async fn test_merge_interleave() { diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 5ab5a6159ad3..7bfa7e2ceefb 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -20,20 +20,19 @@ use std::any::Any; use std::sync::Arc; -use arrow::datatypes::SchemaRef; -use async_trait::async_trait; -use futures::stream::StreamExt; - -use datafusion_common::{internal_err, plan_err, DataFusionError, Result, Statistics}; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; -use log::debug; - +use super::{DisplayAs, DisplayFormatType}; use crate::display::{OutputOrderingDisplay, ProjectSchemaDisplay}; use crate::stream::RecordBatchStreamAdapter; use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; + +use arrow::datatypes::SchemaRef; +use datafusion_common::{internal_err, plan_err, DataFusionError, Result, Statistics}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; -use super::{DisplayAs, DisplayFormatType}; +use async_trait::async_trait; +use futures::stream::StreamExt; +use log::debug; /// A partition that can be converted into a [`SendableRecordBatchStream`] pub trait PartitionStream: Send + Sync { diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index 0431e45b4023..f90f4231c620 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -23,23 +23,21 @@ use std::{ sync::{Arc, Weak}, task::{Context, Poll}, }; -use tokio::sync::Barrier; - -use arrow::{ - datatypes::{DataType, Field, Schema, SchemaRef}, - record_batch::RecordBatch, -}; -use futures::Stream; +use crate::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use crate::{ - common, stream::RecordBatchReceiverStream, stream::RecordBatchStreamAdapter, - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, + common, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use datafusion_physical_expr::PhysicalSortExpr; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::PhysicalSortExpr; + +use futures::Stream; +use tokio::sync::Barrier; /// Index into the data that has been returned so far #[derive(Debug, Default, Clone)] diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 0a4e49d6c73b..8357e2d1bc65 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -25,28 +25,25 @@ use std::pin::Pin; use std::task::{Context, Poll}; use std::{any::Any, sync::Arc}; -use arrow::{ - datatypes::{Field, Schema, SchemaRef}, - record_batch::RecordBatch, -}; -use datafusion_common::stats::Sharpness; -use datafusion_common::{exec_err, internal_err, DFSchemaRef, DataFusionError}; -use futures::Stream; -use itertools::Itertools; -use log::{debug, trace, warn}; - -use super::DisplayAs; use super::{ expressions::PhysicalSortExpr, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, Statistics, + ColumnStatistics, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, + RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::common::get_meet_of_orderings; use crate::metrics::BaselineMetrics; use crate::stream::ObservedStream; -use datafusion_common::Result; + +use arrow::datatypes::{Field, Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; +use datafusion_common::stats::Sharpness; +use datafusion_common::{exec_err, internal_err, DFSchemaRef, DataFusionError, Result}; use datafusion_execution::TaskContext; + +use futures::Stream; +use itertools::Itertools; +use log::{debug, trace, warn}; use tokio::macros::support::thread_rng_n; /// `UnionExec`: `UNION ALL` execution plan. @@ -598,9 +595,9 @@ fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { #[cfg(test)] mod tests { use super::*; + use crate::collect; use crate::test; - use crate::collect; use arrow::record_batch::RecordBatch; use datafusion_common::ScalarValue; diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index be5fcb40d243..ed64735e5ad6 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -17,6 +17,17 @@ //! Defines the unnest column plan for unnesting values in a column that contains a list //! type, conceptually is like joining each row with all the values in the list column. + +use std::time::Instant; +use std::{any::Any, sync::Arc}; + +use super::DisplayAs; +use crate::{ + expressions::Column, DisplayFormatType, Distribution, EquivalenceProperties, + ExecutionPlan, Partitioning, PhysicalExpr, PhysicalSortExpr, RecordBatchStream, + SendableRecordBatchStream, Statistics, +}; + use arrow::array::{ Array, ArrayRef, ArrowPrimitiveType, FixedSizeListArray, LargeListArray, ListArray, PrimitiveArray, @@ -27,23 +38,12 @@ use arrow::datatypes::{ }; use arrow::record_batch::RecordBatch; use arrow_array::{GenericListArray, OffsetSizeTrait}; -use async_trait::async_trait; -use datafusion_common::UnnestOptions; -use datafusion_common::{exec_err, DataFusionError, Result}; +use datafusion_common::{exec_err, DataFusionError, Result, UnnestOptions}; use datafusion_execution::TaskContext; -use futures::Stream; -use futures::StreamExt; -use log::trace; -use std::time::Instant; -use std::{any::Any, sync::Arc}; - -use crate::{ - expressions::Column, DisplayFormatType, Distribution, EquivalenceProperties, - ExecutionPlan, Partitioning, PhysicalExpr, PhysicalSortExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, -}; -use super::DisplayAs; +use async_trait::async_trait; +use futures::{Stream, StreamExt}; +use log::trace; /// Unnest the given column by joining the row with each value in the /// nested type. diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index e6c71bef76b3..ef03e1c39d84 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -17,20 +17,21 @@ //! Values execution plan +use std::any::Any; +use std::sync::Arc; + use super::expressions::PhysicalSortExpr; use super::{common, DisplayAs, SendableRecordBatchStream, Statistics}; use crate::{ memory::MemoryStream, ColumnarValue, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, }; + use arrow::array::new_null_array; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use datafusion_common::{internal_err, plan_err, ScalarValue}; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{internal_err, plan_err, DataFusionError, Result, ScalarValue}; use datafusion_execution::TaskContext; -use std::any::Any; -use std::sync::Arc; /// Execution plan for values list based relation (produces constant rows) #[derive(Debug)] @@ -200,6 +201,7 @@ impl ExecutionPlan for ValuesExec { mod tests { use super::*; use crate::test::{self, make_partition}; + use arrow_schema::{DataType, Field, Schema}; #[tokio::test] 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 33d6aea8f635..e78c5e5a2ced 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -36,9 +36,6 @@ use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, WindowExpr, }; -use datafusion_common::stats::Sharpness; -use datafusion_common::{exec_err, plan_err, Result}; -use datafusion_execution::TaskContext; use arrow::{ array::{Array, ArrayRef, UInt32Builder}, @@ -46,13 +43,14 @@ use arrow::{ datatypes::{Schema, SchemaBuilder, SchemaRef}, record_batch::RecordBatch, }; - use datafusion_common::hash_utils::create_hashes; +use datafusion_common::stats::Sharpness; use datafusion_common::utils::{ evaluate_partition_ranges, get_arrayref_at_indices, get_at_indices, get_record_batch_at_indices, get_row_at_idx, }; -use datafusion_common::DataFusionError; +use datafusion_common::{exec_err, plan_err, DataFusionError, Result}; +use datafusion_execution::TaskContext; use datafusion_expr::window_state::{PartitionBatchState, WindowAggState}; use datafusion_expr::ColumnarValue; use datafusion_physical_expr::window::{ diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 1fed39e7a582..4cc80d65f64e 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -17,10 +17,12 @@ //! Serde code to convert from protocol buffers to Rust data structures. -use crate::protobuf; +use std::convert::{TryFrom, TryInto}; +use std::ops::Deref; +use std::sync::Arc; + +use arrow::compute::SortOptions; use arrow::datatypes::DataType; -use chrono::TimeZone; -use chrono::Utc; use datafusion::arrow::datatypes::Schema; use datafusion::datasource::listing::{FileRange, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; @@ -29,33 +31,28 @@ use datafusion::execution::context::ExecutionProps; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::window_function::WindowFunction; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; -use datafusion::physical_plan::expressions::{in_list, LikeExpr}; +use datafusion::physical_plan::expressions::{ + in_list, BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, LikeExpr, + Literal, NegativeExpr, NotExpr, TryCastExpr, +}; use datafusion::physical_plan::expressions::{GetFieldAccessExpr, GetIndexedFieldExpr}; +use datafusion::physical_plan::joins::utils::JoinSide; use datafusion::physical_plan::windows::create_window_expr; -use datafusion::physical_plan::WindowExpr; use datafusion::physical_plan::{ - expressions::{ - BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, Literal, - NegativeExpr, NotExpr, TryCastExpr, - }, - functions, Partitioning, + functions, ColumnStatistics, Partitioning, PhysicalExpr, Statistics, WindowExpr, }; -use datafusion::physical_plan::{ColumnStatistics, PhysicalExpr, Statistics}; use datafusion_common::stats::Sharpness; -use datafusion_common::ScalarValue; -use datafusion_common::{not_impl_err, DataFusionError, Result}; -use object_store::path::Path; -use object_store::ObjectMeta; -use std::convert::{TryFrom, TryInto}; -use std::ops::Deref; -use std::sync::Arc; +use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue}; use crate::common::proto_error; use crate::convert_required; use crate::logical_plan; +use crate::protobuf; use crate::protobuf::physical_expr_node::ExprType; -use datafusion::physical_plan::joins::utils::JoinSide; -use datafusion::physical_plan::sorts::sort::SortOptions; + +use chrono::{TimeZone, Utc}; +use object_store::path::Path; +use object_store::ObjectMeta; impl From<&protobuf::PhysicalColumn> for Column { fn from(c: &protobuf::PhysicalColumn) -> Column { @@ -609,9 +606,7 @@ impl From<&protobuf::ColumnStats> for ColumnStatistics { impl From for Sharpness { fn from(s: protobuf::Sharpness) -> Self { - let sharpness_type = if let Ok(s_type) = s.sharpness_info.try_into() { - s_type - } else { + let Ok(sharpness_type) = s.sharpness_info.try_into() else { return Sharpness::Absent; }; match sharpness_type { @@ -648,9 +643,7 @@ impl From for Sharpness { impl From for Sharpness { fn from(s: protobuf::Sharpness) -> Self { - let sharpness_type = if let Ok(s_type) = s.sharpness_info.try_into() { - s_type - } else { + let Ok(sharpness_type) = s.sharpness_info.try_into() else { return Sharpness::Absent; }; match sharpness_type { @@ -695,8 +688,6 @@ impl TryFrom<&protobuf::Statistics> for Statistics { fn try_from(s: &protobuf::Statistics) -> Result { // Keep it sync with Statistics::to_proto - let column_statistics = - s.column_stats.iter().map(|s| s.into()).collect::>(); Ok(Statistics { num_rows: if let Some(nr) = &s.num_rows { nr.clone().into() @@ -709,7 +700,7 @@ impl TryFrom<&protobuf::Statistics> for Statistics { Sharpness::Absent }, // No column statistic (None) is encoded with empty array - column_statistics, + column_statistics: s.column_stats.iter().map(|s| s.into()).collect(), }) } } diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index f8e6332a951d..c50258c8267c 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -22,57 +22,34 @@ use std::{ sync::Arc, }; -use datafusion::physical_plan::{ - expressions::{ - ApproxDistinct, ApproxMedian, ApproxPercentileCont, - ApproxPercentileContWithWeight, ArrayAgg, Correlation, Covariance, CovariancePop, - DistinctArrayAgg, DistinctBitXor, DistinctSum, FirstValue, Grouping, LastValue, - Median, OrderSensitiveArrayAgg, Regr, RegrType, Stddev, StddevPop, Variance, - VariancePop, - }, - windows::BuiltInWindowExpr, - ColumnStatistics, -}; -use datafusion::{ - physical_expr::window::NthValueKind, - physical_plan::{ - expressions::{ - CaseExpr, CumeDist, InListExpr, IsNotNullExpr, IsNullExpr, NegativeExpr, - NotExpr, NthValue, Ntile, Rank, RankType, RowNumber, WindowShift, - }, - Statistics, - }, -}; -use datafusion::{ - physical_expr::window::SlidingAggregateWindowExpr, - physical_plan::{ - expressions::{CastExpr, TryCastExpr}, - windows::PlainAggregateWindowExpr, - WindowExpr, - }, -}; - -use datafusion::datasource::listing::{FileRange, PartitionedFile}; -use datafusion::datasource::physical_plan::FileScanConfig; - -use datafusion::physical_plan::expressions::{Count, DistinctCount, Literal}; - -use datafusion::physical_plan::expressions::{ - Avg, BinaryExpr, BitAnd, BitOr, BitXor, BoolAnd, BoolOr, Column, LikeExpr, Max, Min, - Sum, -}; -use datafusion::physical_plan::{AggregateExpr, PhysicalExpr}; - use crate::protobuf::{self, physical_window_expr_node, scalar_value::Value}; use crate::protobuf::{ physical_aggregate_expr_node, PhysicalSortExprNode, PhysicalSortExprNodeCollection, ScalarValue, }; + +use datafusion::datasource::listing::{FileRange, PartitionedFile}; +use datafusion::datasource::physical_plan::FileScanConfig; use datafusion::logical_expr::BuiltinScalarFunction; use datafusion::physical_expr::expressions::{GetFieldAccessExpr, GetIndexedFieldExpr}; +use datafusion::physical_expr::window::{NthValueKind, SlidingAggregateWindowExpr}; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; +use datafusion::physical_plan::expressions::{ + ApproxDistinct, ApproxMedian, ApproxPercentileCont, ApproxPercentileContWithWeight, + ArrayAgg, Avg, BinaryExpr, BitAnd, BitOr, BitXor, BoolAnd, BoolOr, CaseExpr, + CastExpr, Column, Correlation, Count, Covariance, CovariancePop, CumeDist, + DistinctArrayAgg, DistinctBitXor, DistinctCount, DistinctSum, FirstValue, Grouping, + InListExpr, IsNotNullExpr, IsNullExpr, LastValue, LikeExpr, Literal, Max, Median, + Min, NegativeExpr, NotExpr, NthValue, Ntile, OrderSensitiveArrayAgg, Rank, RankType, + Regr, RegrType, RowNumber, Stddev, StddevPop, Sum, TryCastExpr, Variance, + VariancePop, WindowShift, +}; use datafusion::physical_plan::joins::utils::JoinSide; use datafusion::physical_plan::udaf::AggregateFunctionExpr; +use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; +use datafusion::physical_plan::{ + AggregateExpr, ColumnStatistics, PhysicalExpr, Statistics, WindowExpr, +}; use datafusion_common::{ internal_err, not_impl_err, stats::Sharpness, DataFusionError, Result, }; diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 9547053db13a..a8f84a50b661 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -25,20 +25,19 @@ use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion::execution::context::ExecutionProps; -use datafusion::logical_expr::create_udf; -use datafusion::logical_expr::{BuiltinScalarFunction, Volatility}; -use datafusion::logical_expr::{JoinType, Operator}; -use datafusion::physical_expr::expressions::GetFieldAccessExpr; -use datafusion::physical_expr::expressions::{cast, in_list}; +use datafusion::logical_expr::{ + create_udf, BuiltinScalarFunction, JoinType, Operator, Volatility, +}; use datafusion::physical_expr::window::SlidingAggregateWindowExpr; use datafusion::physical_expr::ScalarFunctionExpr; -use datafusion::physical_plan::aggregates::PhysicalGroupBy; -use datafusion::physical_plan::aggregates::{AggregateExec, AggregateMode}; +use datafusion::physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; use datafusion::physical_plan::analyze::AnalyzeExec; use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::{ - binary, col, like, lit, Avg, BinaryExpr, Column, DistinctCount, GetIndexedFieldExpr, - NotExpr, NthValue, PhysicalSortExpr, Sum, + binary, cast, col, in_list, like, lit, Avg, BinaryExpr, Column, DistinctCount, + GetFieldAccessExpr, GetIndexedFieldExpr, NotExpr, NthValue, PhysicalSortExpr, Sum, }; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::functions::make_scalar_function; @@ -49,8 +48,9 @@ use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::windows::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowAggExec, }; -use datafusion::physical_plan::{functions, udaf}; -use datafusion::physical_plan::{AggregateExpr, ExecutionPlan, PhysicalExpr, Statistics}; +use datafusion::physical_plan::{ + functions, udaf, AggregateExpr, ExecutionPlan, PhysicalExpr, Statistics, +}; use datafusion::prelude::SessionContext; use datafusion::scalar::ScalarValue; use datafusion_common::stats::Sharpness; diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index d005a318b0b2..7788ba0a69de 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -15,8 +15,9 @@ // specific language governing permissions and limitations // under the License. -use async_recursion::async_recursion; -use chrono::DateTime; +use std::collections::HashMap; +use std::sync::Arc; + use datafusion::arrow::datatypes::Schema; use datafusion::common::not_impl_err; use datafusion::datasource::listing::PartitionedFile; @@ -25,9 +26,10 @@ use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion::error::{DataFusionError, Result}; use datafusion::physical_plan::{ExecutionPlan, Statistics}; use datafusion::prelude::SessionContext; + +use async_recursion::async_recursion; +use chrono::DateTime; use object_store::ObjectMeta; -use std::collections::HashMap; -use std::sync::Arc; use substrait::proto::read_rel::local_files::file_or_files::PathType; use substrait::proto::{ expression::MaskExpression, read_rel::ReadType, rel::RelType, Rel, diff --git a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs index 1a9bb21a93a1..3e5e757e4c39 100644 --- a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashMap; +use std::sync::Arc; + use datafusion::arrow::datatypes::Schema; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; @@ -23,8 +26,7 @@ use datafusion::error::Result; use datafusion::physical_plan::{displayable, ExecutionPlan, Statistics}; use datafusion::prelude::SessionContext; use datafusion_substrait::physical_plan::{consumer, producer}; -use std::collections::HashMap; -use std::sync::Arc; + use substrait::proto::extensions; #[tokio::test] From 41b364f70eb4082af0251ac1fb88b9a992546960 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 11 Oct 2023 10:03:36 +0300 Subject: [PATCH 37/53] get_int_range replaced by cardinality function --- datafusion/physical-expr/src/analysis.rs | 5 +-- .../physical-expr/src/expressions/in_list.rs | 17 ++++++---- datafusion/physical-expr/src/lib.rs | 3 -- datafusion/physical-plan/src/joins/utils.rs | 32 ++++++------------- 4 files changed, 22 insertions(+), 35 deletions(-) diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index 0c089ce396ea..97294be7a2bb 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -23,8 +23,9 @@ use std::sync::Arc; use arrow::datatypes::Schema; use arrow_schema::Field; -use datafusion_common::stats::Sharpness; -use datafusion_common::{ColumnStatistics, DataFusionError, Result, ScalarValue}; +use datafusion_common::{ + ColumnStatistics, DataFusionError, Result, ScalarValue, Sharpness, +}; use crate::expressions::Column; use crate::intervals::cp_solver::PropagationResult; diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index bdc476f5b3a1..2a9ed58871ea 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -17,8 +17,14 @@ //! Implementation of `InList` expressions: [`InListExpr`] -use ahash::RandomState; use datafusion_common::exec_err; +use datafusion_common::hash_utils::HashValue; +use datafusion_common::{ + cast::{as_boolean_array, as_generic_binary_array, as_string_array}, + internal_err, not_impl_err, DataFusionError, Result, ScalarValue, +}; +use datafusion_expr::ColumnarValue; + use std::any::Any; use std::fmt::Debug; use std::hash::{Hash, Hasher}; @@ -27,6 +33,7 @@ use std::sync::Arc; use crate::physical_expr::down_cast_any_ref; use crate::utils::expr_list_eq_any_order; use crate::PhysicalExpr; + use arrow::array::*; use arrow::buffer::BooleanBuffer; use arrow::compute::kernels::boolean::{not, or_kleene}; @@ -36,12 +43,8 @@ use arrow::datatypes::*; use arrow::record_batch::RecordBatch; use arrow::util::bit_iterator::BitIndexIterator; use arrow::{downcast_dictionary_array, downcast_primitive_array}; -use datafusion_common::hash_utils::HashValue; -use datafusion_common::{ - cast::{as_boolean_array, as_generic_binary_array, as_string_array}, - internal_err, not_impl_err, DataFusionError, Result, ScalarValue, -}; -use datafusion_expr::ColumnarValue; + +use ahash::RandomState; use hashbrown::hash_map::RawEntryMut; use hashbrown::HashMap; diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 48d5f4e1308b..e670380e59d2 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -48,9 +48,6 @@ pub mod utils; pub mod var_provider; pub mod window; -// For backwards compatibility -pub use datafusion_common::hash_utils; - pub use aggregate::groups_accumulator::{ EmitTo, GroupsAccumulator, GroupsAccumulatorAdapter, }; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 1903eb3f560a..e4ae9bc3b263 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -42,10 +42,10 @@ use datafusion_common::cast::as_boolean_array; use datafusion_common::stats::Sharpness; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ - exec_err, plan_err, DataFusionError, JoinType, Result, ScalarValue, SharedResult, + exec_err, plan_err, DataFusionError, JoinType, Result, SharedResult, }; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::intervals::ExprIntervalGraph; +use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval, IntervalBound}; use datafusion_physical_expr::utils::merge_vectors; use datafusion_physical_expr::{ add_offset_to_lex_ordering, EquivalentClass, LexOrdering, LexOrderingRef, @@ -906,7 +906,13 @@ fn max_distinct_count( // Note that float support is intentionally omitted here, since the computation // of a range between two float values is not trivial and the result would be // highly inaccurate. - let numeric_range = get_int_range(min, max)?; + let numeric_range = Interval::new( + IntervalBound::new(min.clone(), false), + IntervalBound::new(max.clone(), false), + ) + .cardinality() + .ok() + .flatten()? as usize; // The number can never be greater than the number of rows we have (minus // the nulls, since they don't count as distinct values). @@ -927,26 +933,6 @@ fn max_distinct_count( } } -/// Return the numeric range between the given min and max values. -fn get_int_range(min: &ScalarValue, max: &ScalarValue) -> Option { - let delta = &max.sub(min).ok()?; - match delta { - ScalarValue::Int8(Some(delta)) if *delta >= 0 => Some(*delta as usize), - ScalarValue::Int16(Some(delta)) if *delta >= 0 => Some(*delta as usize), - ScalarValue::Int32(Some(delta)) if *delta >= 0 => Some(*delta as usize), - ScalarValue::Int64(Some(delta)) if *delta >= 0 => Some(*delta as usize), - ScalarValue::UInt8(Some(delta)) => Some(*delta as usize), - ScalarValue::UInt16(Some(delta)) => Some(*delta as usize), - ScalarValue::UInt32(Some(delta)) => Some(*delta as usize), - ScalarValue::UInt64(Some(delta)) => Some(*delta as usize), - _ => None, - } - // The delta (directly) is not the real range, since it does not include the - // first term. - // E.g. (min=2, max=4) -> (4 - 2) -> 2, but the actual result should be 3 (1, 2, 3). - .map(|open_ended_range| open_ended_range + 1) -} - enum OnceFutState { Pending(OnceFutPending), Ready(SharedResult>), From 33282800f52de71427c4085005f323348cdd0c52 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 11 Oct 2023 12:50:10 +0300 Subject: [PATCH 38/53] Fix imports --- .../physical-expr/src/expressions/in_list.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 2a9ed58871ea..643bbfd820a6 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -17,14 +17,6 @@ //! Implementation of `InList` expressions: [`InListExpr`] -use datafusion_common::exec_err; -use datafusion_common::hash_utils::HashValue; -use datafusion_common::{ - cast::{as_boolean_array, as_generic_binary_array, as_string_array}, - internal_err, not_impl_err, DataFusionError, Result, ScalarValue, -}; -use datafusion_expr::ColumnarValue; - use std::any::Any; use std::fmt::Debug; use std::hash::{Hash, Hasher}; @@ -43,6 +35,14 @@ use arrow::datatypes::*; use arrow::record_batch::RecordBatch; use arrow::util::bit_iterator::BitIndexIterator; use arrow::{downcast_dictionary_array, downcast_primitive_array}; +use datafusion_common::cast::{ + as_boolean_array, as_generic_binary_array, as_string_array, +}; +use datafusion_common::hash_utils::HashValue; +use datafusion_common::{ + exec_err, internal_err, not_impl_err, DataFusionError, Result, ScalarValue, +}; +use datafusion_expr::ColumnarValue; use ahash::RandomState; use hashbrown::hash_map::RawEntryMut; From ba31d445c11e2e41b0ffa302504d4acc89c93872 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 11 Oct 2023 18:05:03 +0300 Subject: [PATCH 39/53] Statistics display is shortened. --- datafusion/common/src/stats.rs | 6 +----- datafusion/core/tests/sql/explain_analyze.rs | 2 +- datafusion/sqllogictest/test_files/explain.slt | 8 ++++---- 3 files changed, 6 insertions(+), 10 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 47aa60bac6f1..d89dca351c3a 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -253,11 +253,7 @@ impl Statistics { impl Display for Statistics { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!( - f, - "Rows={}, Bytes={}, ColStats={:?}", - self.num_rows, self.total_byte_size, self.column_statistics - )?; + write!(f, "Rows={}, Bytes={}", self.num_rows, self.total_byte_size)?; Ok(()) } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index d20070183ad1..321288522c05 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -827,5 +827,5 @@ async fn csv_explain_analyze_with_statistics() { .to_string(); // should contain scan statistics - assert_contains!(&formatted, ", statistics=[Rows=Absent, Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }]]"); + assert_contains!(&formatted, ", statistics=[Rows=Absent, Bytes=Absent]"); } diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index c27cbd15f4ef..066a31590ccd 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -273,8 +273,8 @@ query TT EXPLAIN SELECT a, b, c FROM simple_explain_test limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(10), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }]] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Absent, Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }]] +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(10), Bytes=Absent] +--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], limit=10, has_header=true, statistics=[Rows=Absent, Bytes=Absent] # Parquet scan with statistics collected statement ok @@ -287,8 +287,8 @@ query TT EXPLAIN SELECT * FROM alltypes_plain limit 10; ---- physical_plan -GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }]] ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, ColStats=[ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }, ColumnStatistics { null_count: Absent, max_value: Absent, min_value: Absent, distinct_count: Absent }]] +GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent] +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent] statement ok set datafusion.execution.collect_statistics = false; From 8479595e636f5c100b1977d9d979af758f815fe3 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 13 Oct 2023 09:53:42 +0300 Subject: [PATCH 40/53] fix after merge --- datafusion/common/src/lib.rs | 2 +- datafusion/common/src/stats.rs | 206 +--------- .../core/src/datasource/file_format/csv.rs | 6 +- .../core/src/datasource/file_format/json.rs | 6 +- .../src/datasource/file_format/parquet.rs | 51 ++- .../core/src/datasource/listing/table.rs | 14 +- .../physical_plan/file_scan_config.rs | 12 +- datafusion/core/src/datasource/statistics.rs | 102 ++--- .../aggregate_statistics.rs | 12 +- .../enforce_distribution.rs | 4 +- .../src/physical_optimizer/join_selection.rs | 79 ++-- datafusion/core/tests/custom_sources.rs | 12 +- .../tests/custom_sources_cases/statistics.rs | 30 +- .../core/tests/parquet/file_statistics.rs | 14 +- datafusion/core/tests/path_partition.rs | 6 +- datafusion/physical-expr/src/analysis.rs | 7 +- .../physical-expr/src/expressions/negative.rs | 4 +- .../physical-plan/src/aggregates/mod.rs | 8 +- datafusion/physical-plan/src/common.rs | 32 +- datafusion/physical-plan/src/filter.rs | 184 ++++----- .../physical-plan/src/joins/cross_join.rs | 124 +++--- datafusion/physical-plan/src/joins/utils.rs | 130 +++--- datafusion/physical-plan/src/limit.rs | 44 +-- datafusion/physical-plan/src/projection.rs | 72 ++-- datafusion/physical-plan/src/union.rs | 88 ++--- .../src/windows/bounded_window_agg_exec.rs | 4 +- .../src/windows/window_agg_exec.rs | 4 +- datafusion/proto/proto/datafusion.proto | 18 +- datafusion/proto/src/generated/pbjson.rs | 370 +++++++++--------- datafusion/proto/src/generated/prost.rs | 28 +- .../proto/src/physical_plan/from_proto.rs | 70 ++-- .../proto/src/physical_plan/to_proto.rs | 46 +-- .../tests/cases/roundtrip_physical_plan.rs | 6 +- 33 files changed, 806 insertions(+), 989 deletions(-) diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 535b4c47ddb3..a939cf73dc9c 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -60,7 +60,7 @@ pub use functional_dependencies::{ pub use join_type::{JoinConstraint, JoinType}; pub use scalar::{ScalarType, ScalarValue}; pub use schema_reference::{OwnedSchemaReference, SchemaReference}; -pub use stats::{ColumnStatistics, Sharpness, Statistics}; +pub use stats::{ColumnStatistics, Statistics}; pub use table_reference::{OwnedTableReference, ResolvedTableReference, TableReference}; pub use unnest::UnnestOptions; pub use utils::project_schema; diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 17d33f65da43..757ca613848d 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -18,7 +18,7 @@ //! This module provides data structures to represent statistics use crate::ScalarValue; -use arrow::datatypes::DataType; +use arrow_schema::Schema; use std::fmt::{self, Debug, Display}; @@ -199,182 +199,6 @@ impl Display for Precision< } } -use arrow::datatypes::Schema; - -/// To deal with information without exactness guarantees, we wrap it inside a -/// [`Sharpness`] object to express its reliability. See [`Statistics`] for a usage. -#[derive(Clone, PartialEq, Eq, Default)] -pub enum Sharpness { - Exact(T), - Inexact(T), - #[default] - Absent, -} - -impl Sharpness { - /// If we have some value (exact or inexact), it returns that value. - /// Otherwise, it returns `None`. - pub fn get_value(&self) -> Option<&T> { - match self { - Sharpness::Exact(value) | Sharpness::Inexact(value) => Some(value), - Sharpness::Absent => None, - } - } - - /// Transform the value in this [`Sharpness`] object, if one exists, using - /// the given function. Preserves the exactness state. - pub fn map(self, f: F) -> Sharpness - where - F: Fn(T) -> T, - { - match self { - Sharpness::Exact(val) => Sharpness::Exact(f(val)), - Sharpness::Inexact(val) => Sharpness::Inexact(f(val)), - _ => self, - } - } - - /// Returns `Some(true)` if we have an exact value, `Some(false)` if we - /// have an inexact value, and `None` if there is no value. - pub fn is_exact(&self) -> Option { - match self { - Sharpness::Exact(_) => Some(true), - Sharpness::Inexact(_) => Some(false), - _ => None, - } - } - - /// Returns the maximum of two (possibly inexact) values, conservatively - /// propagating exactness information. If one of the input values is - /// [`Sharpness::Absent`], the result is `Absent` too. - pub fn max(&self, other: &Sharpness) -> Sharpness { - match (self, other) { - (Sharpness::Exact(a), Sharpness::Exact(b)) => { - Sharpness::Exact(if a >= b { a.clone() } else { b.clone() }) - } - (Sharpness::Inexact(a), Sharpness::Exact(b)) - | (Sharpness::Exact(a), Sharpness::Inexact(b)) - | (Sharpness::Inexact(a), Sharpness::Inexact(b)) => { - Sharpness::Inexact(if a >= b { a.clone() } else { b.clone() }) - } - (_, _) => Sharpness::Absent, - } - } - - /// Returns the minimum of two (possibly inexact) values, conservatively - /// propagating exactness information. If one of the input values is - /// [`Sharpness::Absent`], the result is `Absent` too. - pub fn min(&self, other: &Sharpness) -> Sharpness { - match (self, other) { - (Sharpness::Exact(a), Sharpness::Exact(b)) => { - Sharpness::Exact(if a >= b { b.clone() } else { a.clone() }) - } - (Sharpness::Inexact(a), Sharpness::Exact(b)) - | (Sharpness::Exact(a), Sharpness::Inexact(b)) - | (Sharpness::Inexact(a), Sharpness::Inexact(b)) => { - Sharpness::Inexact(if a >= b { b.clone() } else { a.clone() }) - } - (_, _) => Sharpness::Absent, - } - } - - /// Demotes the sharpness state to inexact (if present). - pub fn to_inexact(self) -> Self { - match self { - Sharpness::Exact(value) => Sharpness::Inexact(value), - _ => self, - } - } -} - -impl Sharpness { - /// Calculates the sum of two (possibly inexact) [`usize`] values, - /// conservatively propagating exactness information. If one of the input - /// values is [`Sharpness::Absent`], the result is `Absent` too. - pub fn add(&self, other: &Sharpness) -> Sharpness { - match (self, other) { - (Sharpness::Exact(a), Sharpness::Exact(b)) => Sharpness::Exact(a + b), - (Sharpness::Inexact(a), Sharpness::Exact(b)) - | (Sharpness::Exact(a), Sharpness::Inexact(b)) - | (Sharpness::Inexact(a), Sharpness::Inexact(b)) => Sharpness::Inexact(a + b), - (_, _) => Sharpness::Absent, - } - } - - /// Calculates the difference of two (possibly inexact) [`usize`] values, - /// conservatively propagating exactness information. If one of the input - /// values is [`Sharpness::Absent`], the result is `Absent` too. - pub fn sub(&self, other: &Sharpness) -> Sharpness { - match (self, other) { - (Sharpness::Exact(a), Sharpness::Exact(b)) => Sharpness::Exact(a - b), - (Sharpness::Inexact(a), Sharpness::Exact(b)) - | (Sharpness::Exact(a), Sharpness::Inexact(b)) - | (Sharpness::Inexact(a), Sharpness::Inexact(b)) => Sharpness::Inexact(a - b), - (_, _) => Sharpness::Absent, - } - } - - /// Calculates the multiplication of two (possibly inexact) [`usize`] values, - /// conservatively propagating exactness information. If one of the input - /// values is [`Sharpness::Absent`], the result is `Absent` too. - pub fn multiply(&self, other: &Sharpness) -> Sharpness { - match (self, other) { - (Sharpness::Exact(a), Sharpness::Exact(b)) => Sharpness::Exact(a * b), - (Sharpness::Inexact(a), Sharpness::Exact(b)) - | (Sharpness::Exact(a), Sharpness::Inexact(b)) - | (Sharpness::Inexact(a), Sharpness::Inexact(b)) => Sharpness::Inexact(a * b), - (_, _) => Sharpness::Absent, - } - } -} - -impl Sharpness { - /// Calculates the sum of two (possibly inexact) [`ScalarValue`] values, - /// conservatively propagating exactness information. If one of the input - /// values is [`Sharpness::Absent`], the result is `Absent` too. - pub fn add(&self, other: &Sharpness) -> Sharpness { - match (self, other) { - (Sharpness::Exact(a), Sharpness::Exact(b)) => { - if let Ok(result) = a.add(b) { - Sharpness::Exact(result) - } else { - Sharpness::Absent - } - } - (Sharpness::Inexact(a), Sharpness::Exact(b)) - | (Sharpness::Exact(a), Sharpness::Inexact(b)) - | (Sharpness::Inexact(a), Sharpness::Inexact(b)) => { - if let Ok(result) = a.add(b) { - Sharpness::Inexact(result) - } else { - Sharpness::Absent - } - } - (_, _) => Sharpness::Absent, - } - } -} - -impl Debug for Sharpness { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - Sharpness::Exact(inner) => write!(f, "Exact({:?})", inner), - Sharpness::Inexact(inner) => write!(f, "Inexact({:?})", inner), - Sharpness::Absent => write!(f, "Absent"), - } - } -} - -impl Display for Sharpness { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - Sharpness::Exact(inner) => write!(f, "Exact({:?})", inner), - Sharpness::Inexact(inner) => write!(f, "Inexact({:?})", inner), - Sharpness::Absent => write!(f, "Absent"), - } - } -} - /// Statistics for a relation /// Fields are optional and can be inexact because the sources /// sometimes provide approximate estimates for performance reasons @@ -382,9 +206,9 @@ impl Display for Sharpness< #[derive(Debug, Clone, PartialEq, Eq)] pub struct Statistics { /// The number of table rows - pub num_rows: Sharpness, + pub num_rows: Precision, /// total bytes of the table rows - pub total_byte_size: Sharpness, + pub total_byte_size: Precision, /// Statistics on a column level pub column_statistics: Vec, } @@ -394,8 +218,8 @@ impl Statistics { /// unknown statistics to each column in the schema. pub fn new_unknown(schema: &Schema) -> Self { Self { - num_rows: Sharpness::Absent, - total_byte_size: Sharpness::Absent, + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, column_statistics: Statistics::unknown_column(schema), } } @@ -410,7 +234,7 @@ impl Statistics { } /// If the exactness of a [`Statistics`] instance is lost, this function relaxes - /// the exactness of all information by converting them [`Sharpness::Inexact`]. + /// the exactness of all information by converting them [`Precision::Inexact`]. pub fn make_inexact(self) -> Self { Statistics { num_rows: self.num_rows.to_inexact(), @@ -441,13 +265,13 @@ impl Display for Statistics { #[derive(Clone, Debug, PartialEq, Eq, Default)] pub struct ColumnStatistics { /// Number of null values on column - pub null_count: Sharpness, + pub null_count: Precision, /// Maximum value of column - pub max_value: Sharpness, + pub max_value: Precision, /// Minimum value of column - pub min_value: Sharpness, + pub min_value: Precision, /// Number of distinct values - pub distinct_count: Sharpness, + pub distinct_count: Precision, } impl ColumnStatistics { @@ -460,13 +284,13 @@ impl ColumnStatistics { } } - /// Returns a [`ColumnStatistics`] instance having all [`Sharpness::Absent`] parameters. + /// Returns a [`ColumnStatistics`] instance having all [`Precision::Absent`] parameters. pub fn new_unknown() -> ColumnStatistics { ColumnStatistics { - null_count: Sharpness::Absent, - max_value: Sharpness::Absent, - min_value: Sharpness::Absent, - distinct_count: Sharpness::Absent, + null_count: Precision::Absent, + max_value: Precision::Absent, + min_value: Precision::Absent, + distinct_count: Precision::Absent, } } } diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index ebe3d24f731d..41265ede7fc7 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -625,7 +625,7 @@ mod tests { use chrono::DateTime; use datafusion_common::cast::as_string_array; use datafusion_common::internal_err; - use datafusion_common::stats::Sharpness; + use datafusion_common::stats::Precision; use datafusion_common::FileType; use datafusion_common::GetExt; use datafusion_expr::{col, lit}; @@ -658,8 +658,8 @@ mod tests { assert_eq!(tt_batches, 50 /* 100/2 */); // test metadata - assert_eq!(exec.statistics()?.num_rows, Sharpness::Absent); - assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Absent); + assert_eq!(exec.statistics()?.num_rows, Precision::Absent); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Absent); Ok(()) } diff --git a/datafusion/core/src/datasource/file_format/json.rs b/datafusion/core/src/datasource/file_format/json.rs index 97fd6c4dfba3..dc5b24b2ea10 100644 --- a/datafusion/core/src/datasource/file_format/json.rs +++ b/datafusion/core/src/datasource/file_format/json.rs @@ -381,7 +381,7 @@ impl DataSink for JsonSink { mod tests { use super::super::test_util::scan_format; use datafusion_common::cast::as_int64_array; - use datafusion_common::stats::Sharpness; + use datafusion_common::stats::Precision; use futures::StreamExt; use object_store::local::LocalFileSystem; @@ -412,8 +412,8 @@ mod tests { assert_eq!(tt_batches, 6 /* 12/2 */); // test metadata - assert_eq!(exec.statistics()?.num_rows, Sharpness::Absent); - assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Absent); + assert_eq!(exec.statistics()?.num_rows, Precision::Absent); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Absent); Ok(()) } diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index 371c9adb7de0..859bff7ae46f 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -44,9 +44,8 @@ use crate::physical_plan::{ use arrow::array::{BooleanArray, Float32Array, Float64Array, Int32Array, Int64Array}; use arrow::datatypes::{DataType, Fields, Schema, SchemaRef}; -use datafusion_common::{ - exec_err, not_impl_err, plan_err, DataFusionError, FileType, Sharpness, -}; +use datafusion_common::stats::Precision; +use datafusion_common::{exec_err, not_impl_err, plan_err, DataFusionError, FileType}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; use datafusion_physical_plan::metrics::MetricsSet; @@ -512,7 +511,7 @@ async fn fetch_statistics( let mut num_rows = 0; let mut total_byte_size = 0; - let mut null_counts = vec![Sharpness::Exact(0); num_fields]; + let mut null_counts = vec![Precision::Exact(0); num_fields]; let mut has_statistics = false; let schema_adapter = SchemaAdapter::new(table_schema.clone()); @@ -538,7 +537,7 @@ async fn fetch_statistics( schema_adapter.map_column_index(table_idx, &file_schema) { if let Some((null_count, stats)) = column_stats.get(&file_idx) { - *null_cnt = null_cnt.add(&Sharpness::Exact(*null_count as usize)); + *null_cnt = null_cnt.add(&Precision::Exact(*null_count as usize)); summarize_min_max( &mut max_values, &mut min_values, @@ -552,7 +551,7 @@ async fn fetch_statistics( min_values[table_idx] = None; } } else { - *null_cnt = null_cnt.add(&Sharpness::Exact(num_rows as usize)); + *null_cnt = null_cnt.add(&Precision::Exact(num_rows as usize)); } } } @@ -565,8 +564,8 @@ async fn fetch_statistics( }; let statistics = Statistics { - num_rows: Sharpness::Exact(num_rows as usize), - total_byte_size: Sharpness::Exact(total_byte_size as usize), + num_rows: Precision::Exact(num_rows as usize), + total_byte_size: Precision::Exact(total_byte_size as usize), column_statistics: column_stats, }; @@ -1177,25 +1176,25 @@ mod tests { let stats = fetch_statistics(store.as_ref(), schema.clone(), &meta[0], None).await?; - assert_eq!(stats.num_rows, Sharpness::Exact(3)); + assert_eq!(stats.num_rows, Precision::Exact(3)); let c1_stats = &stats.column_statistics[0]; let c2_stats = &stats.column_statistics[1]; - assert_eq!(c1_stats.null_count, Sharpness::Exact(1)); - assert_eq!(c2_stats.null_count, Sharpness::Exact(3)); + assert_eq!(c1_stats.null_count, Precision::Exact(1)); + assert_eq!(c2_stats.null_count, Precision::Exact(3)); let stats = fetch_statistics(store.as_ref(), schema, &meta[1], None).await?; - assert_eq!(stats.num_rows, Sharpness::Exact(3)); + assert_eq!(stats.num_rows, Precision::Exact(3)); let c1_stats = &stats.column_statistics[0]; let c2_stats = &stats.column_statistics[1]; - assert_eq!(c1_stats.null_count, Sharpness::Exact(3)); - assert_eq!(c2_stats.null_count, Sharpness::Exact(1)); + assert_eq!(c1_stats.null_count, Precision::Exact(3)); + assert_eq!(c2_stats.null_count, Precision::Exact(1)); assert_eq!( c2_stats.max_value, - Sharpness::Exact(ScalarValue::Int64(Some(2))) + Precision::Exact(ScalarValue::Int64(Some(2))) ); assert_eq!( c2_stats.min_value, - Sharpness::Exact(ScalarValue::Int64(Some(1))) + Precision::Exact(ScalarValue::Int64(Some(1))) ); Ok(()) @@ -1332,11 +1331,11 @@ mod tests { fetch_statistics(store.upcast().as_ref(), schema.clone(), &meta[0], Some(9)) .await?; - assert_eq!(stats.num_rows, Sharpness::Exact(3)); + assert_eq!(stats.num_rows, Precision::Exact(3)); let c1_stats = &stats.column_statistics[0]; let c2_stats = &stats.column_statistics[1]; - assert_eq!(c1_stats.null_count, Sharpness::Exact(1)); - assert_eq!(c2_stats.null_count, Sharpness::Exact(3)); + assert_eq!(c1_stats.null_count, Precision::Exact(1)); + assert_eq!(c2_stats.null_count, Precision::Exact(3)); let store = Arc::new(RequestCountingObjectStore::new(Arc::new( LocalFileSystem::new(), @@ -1365,11 +1364,11 @@ mod tests { ) .await?; - assert_eq!(stats.num_rows, Sharpness::Exact(3)); + assert_eq!(stats.num_rows, Precision::Exact(3)); let c1_stats = &stats.column_statistics[0]; let c2_stats = &stats.column_statistics[1]; - assert_eq!(c1_stats.null_count, Sharpness::Exact(1)); - assert_eq!(c2_stats.null_count, Sharpness::Exact(3)); + assert_eq!(c1_stats.null_count, Precision::Exact(1)); + assert_eq!(c2_stats.null_count, Precision::Exact(3)); let store = Arc::new(RequestCountingObjectStore::new(Arc::new( LocalFileSystem::new(), @@ -1409,8 +1408,8 @@ mod tests { assert_eq!(tt_batches, 4 /* 8/2 */); // test metadata - assert_eq!(exec.statistics()?.num_rows, Sharpness::Exact(8)); - assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Exact(671)); + assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8)); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671)); Ok(()) } @@ -1451,8 +1450,8 @@ mod tests { get_exec(&state, "alltypes_plain.parquet", projection, Some(1)).await?; // note: even if the limit is set, the executor rounds up to the batch size - assert_eq!(exec.statistics()?.num_rows, Sharpness::Exact(8)); - assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Exact(671)); + assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8)); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671)); let batches = collect(exec, task_ctx).await?; assert_eq!(1, batches.len()); assert_eq!(11, batches[0].num_columns()); diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 83e62b685f32..9bbdf2fc5392 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1049,7 +1049,7 @@ mod tests { use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; - use datafusion_common::stats::Sharpness; + use datafusion_common::stats::Precision; use datafusion_common::{assert_contains, GetExt, ScalarValue}; use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator}; use rstest::*; @@ -1098,8 +1098,8 @@ mod tests { assert_eq!(exec.output_partitioning().partition_count(), 1); // test metadata - assert_eq!(exec.statistics()?.num_rows, Sharpness::Exact(8)); - assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Exact(671)); + assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8)); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671)); Ok(()) } @@ -1121,8 +1121,8 @@ mod tests { let table = ListingTable::try_new(config)?; let exec = table.scan(&state, None, &[], None).await?; - assert_eq!(exec.statistics()?.num_rows, Sharpness::Exact(8)); - assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Exact(671)); + assert_eq!(exec.statistics()?.num_rows, Precision::Exact(8)); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Exact(671)); Ok(()) } @@ -1145,8 +1145,8 @@ mod tests { let table = ListingTable::try_new(config)?; let exec = table.scan(&state, None, &[], None).await?; - assert_eq!(exec.statistics()?.num_rows, Sharpness::Absent); - assert_eq!(exec.statistics()?.total_byte_size, Sharpness::Absent); + assert_eq!(exec.statistics()?.num_rows, Precision::Absent); + assert_eq!(exec.statistics()?.total_byte_size, Precision::Absent); Ok(()) } 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 8e4782a59e94..c1a19b745b8d 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -38,7 +38,7 @@ use arrow::buffer::Buffer; use arrow::datatypes::{ArrowNativeType, UInt16Type}; use arrow_array::{ArrayRef, DictionaryArray, RecordBatch}; use arrow_schema::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::{exec_err, ColumnStatistics, Statistics}; use datafusion_physical_expr::LexOrdering; @@ -145,7 +145,7 @@ impl FileScanConfig { let table_stats = Statistics { num_rows: self.statistics.num_rows.clone(), // TODO correct byte size? - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, column_statistics: table_cols_stats, }; @@ -534,16 +534,16 @@ mod tests { Arc::clone(&file_schema), Some(vec![file_schema.fields().len(), 0]), Statistics { - num_rows: Sharpness::Inexact(10), + num_rows: Precision::Inexact(10), // assign the column index to distinct_count to help assert // the source statistic after the projection column_statistics: (0..file_schema.fields().len()) .map(|i| ColumnStatistics { - distinct_count: Sharpness::Inexact(i), + distinct_count: Precision::Inexact(i), ..Default::default() }) .collect(), - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }, vec![( "date".to_owned(), @@ -560,7 +560,7 @@ mod tests { assert_eq!(proj_stat_cols.len(), 2); // TODO implement tests for proj_stat_cols[0] once partition column // statistics are implemented - assert_eq!(proj_stat_cols[1].distinct_count, Sharpness::Inexact(0)); + assert_eq!(proj_stat_cols[1].distinct_count, Precision::Inexact(0)); let col_names = conf.projected_file_column_names(); assert_eq!(col_names, Some(vec!["c1".to_owned()])); diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 9905e5bf774a..df487beff58c 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -21,7 +21,7 @@ use crate::error::Result; use crate::physical_plan::expressions::{MaxAccumulator, MinAccumulator}; use crate::physical_plan::{Accumulator, ColumnStatistics, Statistics}; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::ScalarValue; use futures::{Stream, StreamExt}; @@ -36,16 +36,16 @@ pub async fn get_statistics_with_limit( limit: Option, ) -> Result<(Vec, Statistics)> { let mut result_files = vec![]; - let mut null_counts: Option>> = None; - let mut max_values: Option>> = None; - let mut min_values: Option>> = None; + let mut null_counts: Option>> = None; + let mut max_values: Option>> = None; + let mut min_values: Option>> = None; // The number of rows and the total byte size can be calculated as long as // at least one file has them. If none of the files provide them, then they // will be omitted from the statistics. The missing values will be counted // as zero. - let mut num_rows: Option> = None; - let mut total_byte_size: Option> = None; + let mut num_rows: Option> = None; + let mut total_byte_size: Option> = None; // Fusing the stream allows us to call next safely even once it is finished. let mut all_files = Box::pin(all_files.fuse()); @@ -55,11 +55,11 @@ pub async fn get_statistics_with_limit( // Number of rows, total byte size and null counts are added for each file. // In case of an absent information or inexact value coming from the file, - // it changes the statistic sharpness to inexact. + // it changes the statistic precision to inexact. num_rows = Some(if let Some(some_num_rows) = num_rows { match (file_stats.num_rows, &some_num_rows) { - (Sharpness::Absent, _) => some_num_rows.to_inexact(), - (lhs, Sharpness::Absent) => lhs.to_inexact(), + (Precision::Absent, _) => some_num_rows.to_inexact(), + (lhs, Precision::Absent) => lhs.to_inexact(), (lhs, rhs) => lhs.add(rhs), } } else { @@ -68,8 +68,8 @@ pub async fn get_statistics_with_limit( total_byte_size = Some(if let Some(some_total_byte_size) = total_byte_size { match (file_stats.total_byte_size, &some_total_byte_size) { - (Sharpness::Absent, _) => some_total_byte_size.to_inexact(), - (lhs, Sharpness::Absent) => lhs.to_inexact(), + (Precision::Absent, _) => some_total_byte_size.to_inexact(), + (lhs, Precision::Absent) => lhs.to_inexact(), (lhs, rhs) => lhs.add(rhs), } } else { @@ -81,7 +81,7 @@ pub async fn get_statistics_with_limit( .iter_mut() .zip(file_stats.column_statistics.iter()) { - *target = if cs.null_count == Sharpness::Absent { + *target = if cs.null_count == Precision::Absent { // Downcast to inexact: target.clone().to_inexact() } else { @@ -99,7 +99,7 @@ pub async fn get_statistics_with_limit( // as partitions, guaranteed to be at the end). Hence, rest of the // fields are initialized with `Absent`. for _ in 0..file_schema.fields().len() - file_stats.column_statistics.len() { - new_col_stats_nulls.push(Sharpness::Absent) + new_col_stats_nulls.push(Precision::Absent) } null_counts = Some(new_col_stats_nulls); }; @@ -118,7 +118,7 @@ pub async fn get_statistics_with_limit( // file schema may have additional fields other than each file (such as partition, guaranteed to be at the end) // Hence, push rest of the fields with information Absent. for _ in 0..file_schema.fields().len() - file_stats.column_statistics.len() { - new_col_stats_max.push(Sharpness::Absent) + new_col_stats_max.push(Precision::Absent) } max_values = Some(new_col_stats_max); }; @@ -137,7 +137,7 @@ pub async fn get_statistics_with_limit( // file schema may have additional fields other than each file (such as partition, guaranteed to be at the end) // Hence, push rest of the fields with information Absent. for _ in 0..file_schema.fields().len() - file_stats.column_statistics.len() { - new_col_stats_min.push(Sharpness::Absent) + new_col_stats_min.push(Precision::Absent) } min_values = Some(new_col_stats_min); }; @@ -148,7 +148,7 @@ pub async fn get_statistics_with_limit( // number of rows. if num_rows .as_ref() - .unwrap_or(&Sharpness::Absent) + .unwrap_or(&Precision::Absent) .get_value() .unwrap_or(&usize::MIN) > &limit.unwrap_or(usize::MAX) @@ -158,13 +158,13 @@ pub async fn get_statistics_with_limit( } let size = file_schema.fields().len(); - let null_counts = null_counts.unwrap_or(vec![Sharpness::Absent; size]); - let max_values = max_values.unwrap_or(vec![Sharpness::Absent; size]); - let min_values = min_values.unwrap_or(vec![Sharpness::Absent; size]); + let null_counts = null_counts.unwrap_or(vec![Precision::Absent; size]); + let max_values = max_values.unwrap_or(vec![Precision::Absent; size]); + let min_values = min_values.unwrap_or(vec![Precision::Absent; size]); let mut statistics = Statistics { - num_rows: num_rows.unwrap_or(Sharpness::Absent), - total_byte_size: total_byte_size.unwrap_or(Sharpness::Absent), + num_rows: num_rows.unwrap_or(Precision::Absent), + total_byte_size: total_byte_size.unwrap_or(Precision::Absent), column_statistics: get_col_stats_vec(null_counts, max_values, min_values), }; if all_files.next().await.is_some() { @@ -194,23 +194,23 @@ pub(crate) fn create_max_min_accs( } pub(crate) fn get_col_stats_vec( - null_counts: Vec>, - max_values: Vec>, - min_values: Vec>, + null_counts: Vec>, + max_values: Vec>, + min_values: Vec>, ) -> Vec { izip!(null_counts, max_values, min_values) .map(|(null_count, max_value, min_value)| ColumnStatistics { null_count, max_value, min_value, - distinct_count: Sharpness::Absent, + distinct_count: Precision::Absent, }) .collect() } pub(crate) fn get_col_stats( schema: &Schema, - null_counts: Vec>, + null_counts: Vec>, max_values: &mut [Option], min_values: &mut [Option], ) -> Vec { @@ -226,9 +226,9 @@ pub(crate) fn get_col_stats( }; ColumnStatistics { null_count: null_counts[i].clone(), - max_value: max_value.map(Sharpness::Exact).unwrap_or(Sharpness::Absent), - min_value: min_value.map(Sharpness::Exact).unwrap_or(Sharpness::Absent), - distinct_count: Sharpness::Absent, + max_value: max_value.map(Precision::Exact).unwrap_or(Precision::Absent), + min_value: min_value.map(Precision::Exact).unwrap_or(Precision::Absent), + distinct_count: Precision::Absent, } }) .collect() @@ -237,63 +237,63 @@ pub(crate) fn get_col_stats( /// If the given value is numerically greater than the original maximum value, /// set the new maximum value with appropriate exactness information. fn set_max_if_greater( - max_values: &mut [Sharpness], - max_nominee: Sharpness, + max_values: &mut [Precision], + max_nominee: Precision, index: usize, ) { match (&max_values[index], &max_nominee) { - (Sharpness::Exact(val1), Sharpness::Exact(val2)) => { + (Precision::Exact(val1), Precision::Exact(val2)) => { if val1 < val2 { max_values[index] = max_nominee; } } - (Sharpness::Exact(val1), Sharpness::Inexact(val2)) - | (Sharpness::Inexact(val1), Sharpness::Inexact(val2)) - | (Sharpness::Inexact(val1), Sharpness::Exact(val2)) => { + (Precision::Exact(val1), Precision::Inexact(val2)) + | (Precision::Inexact(val1), Precision::Inexact(val2)) + | (Precision::Inexact(val1), Precision::Exact(val2)) => { if val1 < val2 { max_values[index] = max_nominee.to_inexact() } } - (Sharpness::Inexact(_), Sharpness::Absent) - | (Sharpness::Exact(_), Sharpness::Absent) => { + (Precision::Inexact(_), Precision::Absent) + | (Precision::Exact(_), Precision::Absent) => { max_values[index] = max_values[index].clone().to_inexact() } - (Sharpness::Absent, Sharpness::Exact(_)) - | (Sharpness::Absent, Sharpness::Inexact(_)) => { + (Precision::Absent, Precision::Exact(_)) + | (Precision::Absent, Precision::Inexact(_)) => { max_values[index] = max_nominee.to_inexact() } - (Sharpness::Absent, Sharpness::Absent) => max_values[index] = Sharpness::Absent, + (Precision::Absent, Precision::Absent) => max_values[index] = Precision::Absent, } } /// If the given value is numerically lesser than the original minimum value, /// set the new minimum value with appropriate exactness information. fn set_min_if_lesser( - min_values: &mut [Sharpness], - min_nominee: Sharpness, + min_values: &mut [Precision], + min_nominee: Precision, index: usize, ) { match (&min_values[index], &min_nominee) { - (Sharpness::Exact(val1), Sharpness::Exact(val2)) => { + (Precision::Exact(val1), Precision::Exact(val2)) => { if val1 > val2 { min_values[index] = min_nominee; } } - (Sharpness::Exact(val1), Sharpness::Inexact(val2)) - | (Sharpness::Inexact(val1), Sharpness::Inexact(val2)) - | (Sharpness::Inexact(val1), Sharpness::Exact(val2)) => { + (Precision::Exact(val1), Precision::Inexact(val2)) + | (Precision::Inexact(val1), Precision::Inexact(val2)) + | (Precision::Inexact(val1), Precision::Exact(val2)) => { if val1 > val2 { min_values[index] = min_nominee.to_inexact() } } - (Sharpness::Inexact(_), Sharpness::Absent) - | (Sharpness::Exact(_), Sharpness::Absent) => { + (Precision::Inexact(_), Precision::Absent) + | (Precision::Exact(_), Precision::Absent) => { min_values[index] = min_values[index].clone().to_inexact() } - (Sharpness::Absent, Sharpness::Exact(_)) - | (Sharpness::Absent, Sharpness::Inexact(_)) => { + (Precision::Absent, Precision::Exact(_)) + | (Precision::Absent, Precision::Inexact(_)) => { min_values[index] = min_nominee.to_inexact() } - (Sharpness::Absent, Sharpness::Absent) => min_values[index] = Sharpness::Absent, + (Precision::Absent, Precision::Absent) => min_values[index] = Precision::Absent, } } diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 7a11c24095eb..43def5d73f73 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -27,7 +27,7 @@ use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::{expressions, AggregateExpr, ExecutionPlan, Statistics}; use crate::scalar::ScalarValue; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::tree_node::TreeNode; use datafusion_expr::utils::COUNT_STAR_EXPANSION; @@ -141,7 +141,7 @@ fn take_optimizable_table_count( agg_expr: &dyn AggregateExpr, stats: &Statistics, ) -> Option<(ScalarValue, &'static str)> { - if let (&Sharpness::Exact(num_rows), Some(casted_expr)) = ( + if let (&Precision::Exact(num_rows), Some(casted_expr)) = ( &stats.num_rows, agg_expr.as_any().downcast_ref::(), ) { @@ -169,7 +169,7 @@ fn take_optimizable_column_count( stats: &Statistics, ) -> Option<(ScalarValue, String)> { let col_stats = &stats.column_statistics; - if let (&Sharpness::Exact(num_rows), Some(casted_expr)) = ( + if let (&Precision::Exact(num_rows), Some(casted_expr)) = ( &stats.num_rows, agg_expr.as_any().downcast_ref::(), ) { @@ -180,7 +180,7 @@ fn take_optimizable_column_count( .downcast_ref::() { let current_val = &col_stats[col_expr.index()].null_count; - if let &Sharpness::Exact(val) = current_val { + if let &Precision::Exact(val) = current_val { return Some(( ScalarValue::Int64(Some((num_rows - val) as i64)), casted_expr.name().to_string(), @@ -205,7 +205,7 @@ fn take_optimizable_min( .as_any() .downcast_ref::() { - if let Sharpness::Exact(val) = &col_stats[col_expr.index()].min_value { + if let Precision::Exact(val) = &col_stats[col_expr.index()].min_value { if !val.is_null() { return Some((val.clone(), casted_expr.name().to_string())); } @@ -229,7 +229,7 @@ fn take_optimizable_max( .as_any() .downcast_ref::() { - if let Sharpness::Exact(val) = &col_stats[col_expr.index()].max_value { + if let Precision::Exact(val) = &col_stats[col_expr.index()].max_value { if !val.is_null() { return Some((val.clone(), casted_expr.name().to_string())); } diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 15bfa97e5176..d1e09a16095a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -48,7 +48,7 @@ use crate::physical_plan::{ }; use arrow::compute::SortOptions; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::equivalence::EquivalenceProperties; @@ -1238,7 +1238,7 @@ fn ensure_distribution( // Don't need to apply when the returned row count is not greater than 1: let stats = dist_context.plan.statistics()?; let mut repartition_beneficial_stat = true; - if let Sharpness::Exact(num_rows) = stats.num_rows { + if let Precision::Exact(num_rows) = stats.num_rows { // If the number of rows is surely less than one, then repartitioning // is not beneficial. repartition_beneficial_stat = num_rows > 1; diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index ec37d168bf74..876a464257cc 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -601,15 +601,15 @@ mod tests_statistical { }; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::{stats::Sharpness, JoinType, ScalarValue}; + use datafusion_common::{stats::Precision, JoinType, ScalarValue}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::PhysicalExpr; fn create_big_and_small() -> (Arc, Arc) { let big = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(10), - total_byte_size: Sharpness::Inexact(100000), + num_rows: Precision::Inexact(10), + total_byte_size: Precision::Inexact(100000), column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), @@ -617,8 +617,8 @@ mod tests_statistical { let small = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(100000), - total_byte_size: Sharpness::Inexact(10), + num_rows: Precision::Inexact(100000), + total_byte_size: Precision::Inexact(10), column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), @@ -638,14 +638,14 @@ mod tests_statistical { ) -> Vec { vec![ColumnStatistics { distinct_count: distinct_count - .map(Sharpness::Inexact) - .unwrap_or(Sharpness::Absent), + .map(Precision::Inexact) + .unwrap_or(Precision::Absent), min_value: min - .map(|size| Sharpness::Inexact(ScalarValue::UInt64(Some(size)))) - .unwrap_or(Sharpness::Absent), + .map(|size| Precision::Inexact(ScalarValue::UInt64(Some(size)))) + .unwrap_or(Precision::Absent), max_value: max - .map(|size| Sharpness::Inexact(ScalarValue::UInt64(Some(size)))) - .unwrap_or(Sharpness::Absent), + .map(|size| Precision::Inexact(ScalarValue::UInt64(Some(size)))) + .unwrap_or(Precision::Absent), ..Default::default() }] } @@ -661,39 +661,39 @@ mod tests_statistical { ) { let big = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(100_000), + num_rows: Precision::Inexact(100_000), column_statistics: create_column_stats( Some(0), Some(50_000), Some(50_000), ), - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), )); let medium = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(10_000), + num_rows: Precision::Inexact(10_000), column_statistics: create_column_stats( Some(1000), Some(5000), Some(1000), ), - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }, Schema::new(vec![Field::new("medium_col", DataType::Int32, false)]), )); let small = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(1000), + num_rows: Precision::Inexact(1000), column_statistics: create_column_stats( Some(0), Some(100_000), Some(1000), ), - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), )); @@ -744,11 +744,11 @@ mod tests_statistical { assert_eq!( swapped_join.left().statistics().unwrap().total_byte_size, - Sharpness::Inexact(10) + Precision::Inexact(10) ); assert_eq!( swapped_join.right().statistics().unwrap().total_byte_size, - Sharpness::Inexact(100000) + Precision::Inexact(100000) ); } @@ -795,11 +795,11 @@ mod tests_statistical { assert_eq!( swapped_join.left().statistics().unwrap().total_byte_size, - Sharpness::Inexact(100000) + Precision::Inexact(100000) ); assert_eq!( swapped_join.right().statistics().unwrap().total_byte_size, - Sharpness::Inexact(10) + Precision::Inexact(10) ); } @@ -840,11 +840,11 @@ mod tests_statistical { assert_eq!( swapped_join.left().statistics().unwrap().total_byte_size, - Sharpness::Inexact(10) + Precision::Inexact(10) ); assert_eq!( swapped_join.right().statistics().unwrap().total_byte_size, - Sharpness::Inexact(100000) + Precision::Inexact(100000) ); assert_eq!(original_schema, swapped_join.schema()); @@ -955,11 +955,11 @@ mod tests_statistical { assert_eq!( swapped_join.left().statistics().unwrap().total_byte_size, - Sharpness::Inexact(10) + Precision::Inexact(10) ); assert_eq!( swapped_join.right().statistics().unwrap().total_byte_size, - Sharpness::Inexact(100000) + Precision::Inexact(100000) ); } @@ -1002,8 +1002,8 @@ mod tests_statistical { async fn test_join_selection_collect_left() { let big = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(10000000), - total_byte_size: Sharpness::Inexact(10000000), + num_rows: Precision::Inexact(10000000), + total_byte_size: Precision::Inexact(10000000), column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), @@ -1011,8 +1011,8 @@ mod tests_statistical { let small = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(10), - total_byte_size: Sharpness::Inexact(10), + num_rows: Precision::Inexact(10), + total_byte_size: Precision::Inexact(10), column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), @@ -1020,8 +1020,8 @@ mod tests_statistical { let empty = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Absent, - total_byte_size: Sharpness::Absent, + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), @@ -1080,8 +1080,8 @@ mod tests_statistical { async fn test_join_selection_partitioned() { let big1 = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(10000000), - total_byte_size: Sharpness::Inexact(10000000), + num_rows: Precision::Inexact(10000000), + total_byte_size: Precision::Inexact(10000000), column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("big_col1", DataType::Int32, false)]), @@ -1089,8 +1089,8 @@ mod tests_statistical { let big2 = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(20000000), - total_byte_size: Sharpness::Inexact(20000000), + num_rows: Precision::Inexact(20000000), + total_byte_size: Precision::Inexact(20000000), column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("big_col2", DataType::Int32, false)]), @@ -1098,8 +1098,8 @@ mod tests_statistical { let empty = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Absent, - total_byte_size: Sharpness::Absent, + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, column_statistics: vec![ColumnStatistics::new_unknown()], }, Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), @@ -1205,11 +1205,8 @@ mod util_tests { use std::sync::Arc; use arrow_schema::{DataType, Field, Schema}; - use datafusion_common::ScalarValue; use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::{ - BinaryExpr, Column, Literal, NegativeExpr, - }; + use datafusion_physical_expr::expressions::{BinaryExpr, Column, NegativeExpr}; use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::PhysicalExpr; diff --git a/datafusion/core/tests/custom_sources.rs b/datafusion/core/tests/custom_sources.rs index 0f7036e51df7..daf1ef41a297 100644 --- a/datafusion/core/tests/custom_sources.rs +++ b/datafusion/core/tests/custom_sources.rs @@ -39,7 +39,7 @@ use datafusion::physical_plan::{ use datafusion::scalar::ScalarValue; use datafusion_common::cast::as_primitive_array; use datafusion_common::project_schema; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use async_trait::async_trait; use futures::stream::Stream; @@ -154,19 +154,19 @@ impl ExecutionPlan for CustomExecutionPlan { fn statistics(&self) -> Result { let batch = TEST_CUSTOM_RECORD_BATCH!().unwrap(); Ok(Statistics { - num_rows: Sharpness::Exact(batch.num_rows()), - total_byte_size: Sharpness::Absent, + num_rows: Precision::Exact(batch.num_rows()), + total_byte_size: Precision::Absent, column_statistics: self .projection .clone() .unwrap_or_else(|| (0..batch.columns().len()).collect()) .iter() .map(|i| ColumnStatistics { - null_count: Sharpness::Exact(batch.column(*i).null_count()), - min_value: Sharpness::Exact(ScalarValue::Int32(aggregate::min( + null_count: Precision::Exact(batch.column(*i).null_count()), + min_value: Precision::Exact(ScalarValue::Int32(aggregate::min( as_primitive_array::(batch.column(*i)).unwrap(), ))), - max_value: Sharpness::Exact(ScalarValue::Int32(aggregate::max( + max_value: Precision::Exact(ScalarValue::Int32(aggregate::max( as_primitive_array::(batch.column(*i)).unwrap(), ))), ..Default::default() diff --git a/datafusion/core/tests/custom_sources_cases/statistics.rs b/datafusion/core/tests/custom_sources_cases/statistics.rs index bad6a22bafd2..f0985f554654 100644 --- a/datafusion/core/tests/custom_sources_cases/statistics.rs +++ b/datafusion/core/tests/custom_sources_cases/statistics.rs @@ -34,7 +34,7 @@ use datafusion::{ use async_trait::async_trait; use datafusion::execution::context::{SessionState, TaskContext}; -use datafusion_common::{project_schema, stats::Sharpness}; +use datafusion_common::{project_schema, stats::Precision}; /// This is a testing structure for statistics /// It will act both as a table provider and execution plan @@ -100,7 +100,7 @@ impl TableProvider for StatisticsValidation { num_rows: current_stat.num_rows, column_statistics: proj_col_stats, // TODO stats: knowing the type of the new columns we can guess the output size - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }, projected_schema, ))) @@ -178,20 +178,20 @@ fn init_ctx(stats: Statistics, schema: Schema) -> Result { fn fully_defined() -> (Statistics, Schema) { ( Statistics { - num_rows: Sharpness::Exact(13), - total_byte_size: Sharpness::Absent, // ignore byte size for now + num_rows: Precision::Exact(13), + total_byte_size: Precision::Absent, // ignore byte size for now column_statistics: vec![ ColumnStatistics { - distinct_count: Sharpness::Exact(2), - max_value: Sharpness::Exact(ScalarValue::Int32(Some(1023))), - min_value: Sharpness::Exact(ScalarValue::Int32(Some(-24))), - null_count: Sharpness::Exact(0), + distinct_count: Precision::Exact(2), + max_value: Precision::Exact(ScalarValue::Int32(Some(1023))), + min_value: Precision::Exact(ScalarValue::Int32(Some(-24))), + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: Sharpness::Exact(13), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(5486))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(-6783))), - null_count: Sharpness::Exact(5), + distinct_count: Precision::Exact(13), + max_value: Precision::Exact(ScalarValue::Int64(Some(5486))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-6783))), + null_count: Precision::Exact(5), }, ], }, @@ -228,7 +228,7 @@ async fn sql_filter() -> Result<()> { let physical_plan = df.create_physical_plan().await.unwrap(); let stats = physical_plan.statistics()?; - assert_eq!(stats.num_rows, Sharpness::Inexact(1)); + assert_eq!(stats.num_rows, Precision::Inexact(1)); Ok(()) } @@ -245,9 +245,9 @@ async fn sql_limit() -> Result<()> { // we loose all statistics except the for number of rows which becomes the limit assert_eq!( Statistics { - num_rows: Sharpness::Exact(5), + num_rows: Precision::Exact(5), column_statistics: col_stats, - total_byte_size: Sharpness::Absent + total_byte_size: Precision::Absent }, physical_plan.statistics()? ); diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index 6bb53b0c8322..1ea154303d69 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -26,7 +26,7 @@ use datafusion::datasource::physical_plan::ParquetExec; use datafusion::datasource::TableProvider; use datafusion::execution::context::SessionState; use datafusion::prelude::SessionContext; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_execution::cache::cache_manager::CacheManagerConfig; use datafusion_execution::cache::cache_unit; use datafusion_execution::cache::cache_unit::{ @@ -57,10 +57,10 @@ async fn load_table_stats_with_session_level_cache() { assert_eq!(get_static_cache_size(&state1), 0); let exec1 = table1.scan(&state1, None, &[], None).await.unwrap(); - assert_eq!(exec1.statistics().unwrap().num_rows, Sharpness::Exact(8)); + assert_eq!(exec1.statistics().unwrap().num_rows, Precision::Exact(8)); assert_eq!( exec1.statistics().unwrap().total_byte_size, - Sharpness::Exact(671) + Precision::Exact(671) ); assert_eq!(get_static_cache_size(&state1), 1); @@ -68,10 +68,10 @@ async fn load_table_stats_with_session_level_cache() { //check session 1 cache result not show in session 2 assert_eq!(get_static_cache_size(&state2), 0); let exec2 = table2.scan(&state2, None, &[], None).await.unwrap(); - assert_eq!(exec2.statistics().unwrap().num_rows, Sharpness::Exact(8)); + assert_eq!(exec2.statistics().unwrap().num_rows, Precision::Exact(8)); assert_eq!( exec2.statistics().unwrap().total_byte_size, - Sharpness::Exact(671) + Precision::Exact(671) ); assert_eq!(get_static_cache_size(&state2), 1); @@ -79,10 +79,10 @@ async fn load_table_stats_with_session_level_cache() { //check session 1 cache result not show in session 2 assert_eq!(get_static_cache_size(&state1), 1); let exec3 = table1.scan(&state1, None, &[], None).await.unwrap(); - assert_eq!(exec3.statistics().unwrap().num_rows, Sharpness::Exact(8)); + assert_eq!(exec3.statistics().unwrap().num_rows, Precision::Exact(8)); assert_eq!( exec3.statistics().unwrap().total_byte_size, - Sharpness::Exact(671) + Precision::Exact(671) ); // List same file no increase assert_eq!(get_static_cache_size(&state1), 1); diff --git a/datafusion/core/tests/path_partition.rs b/datafusion/core/tests/path_partition.rs index 9e13f83fd473..27d146de798d 100644 --- a/datafusion/core/tests/path_partition.rs +++ b/datafusion/core/tests/path_partition.rs @@ -36,7 +36,7 @@ use datafusion::{ prelude::SessionContext, test_util::{self, arrow_test_data, parquet_test_data}, }; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::ScalarValue; use async_trait::async_trait; @@ -466,7 +466,7 @@ async fn parquet_statistics() -> Result<()> { let stat_cols = physical_plan.statistics()?.column_statistics; assert_eq!(stat_cols.len(), 4); // stats for the first col are read from the parquet file - assert_eq!(stat_cols[0].null_count, Sharpness::Exact(3)); + assert_eq!(stat_cols[0].null_count, Precision::Exact(3)); // TODO assert partition column (1,2,3) stats once implemented (#1186) assert_eq!(stat_cols[1], ColumnStatistics::new_unknown(),); assert_eq!(stat_cols[2], ColumnStatistics::new_unknown(),); @@ -481,7 +481,7 @@ async fn parquet_statistics() -> Result<()> { let stat_cols = physical_plan.statistics()?.column_statistics; assert_eq!(stat_cols.len(), 2); // stats for the first col are read from the parquet file - assert_eq!(stat_cols[0].null_count, Sharpness::Exact(1)); + assert_eq!(stat_cols[0].null_count, Precision::Exact(1)); // TODO assert partition column stats once implemented (#1186) assert_eq!(stat_cols[1], ColumnStatistics::new_unknown(),); diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index 97294be7a2bb..11442f2fd969 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -23,9 +23,8 @@ use std::sync::Arc; use arrow::datatypes::Schema; use arrow_schema::Field; -use datafusion_common::{ - ColumnStatistics, DataFusionError, Result, ScalarValue, Sharpness, -}; +use datafusion_common::stats::Precision; +use datafusion_common::{ColumnStatistics, DataFusionError, Result, ScalarValue}; use crate::expressions::Column; use crate::intervals::cp_solver::PropagationResult; @@ -105,7 +104,7 @@ pub struct ExprBoundaries { /// Minimum and maximum values this expression can have. pub interval: Interval, /// Maximum number of distinct values this expression can produce, if known. - pub distinct_count: Sharpness, + pub distinct_count: Precision, } /// Attempts to refine column boundaries and compute a selectivity value. diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index 906658634bc0..86b000e76a32 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -31,11 +31,9 @@ use datafusion_expr::{ type_coercion::{is_interval, is_null, is_signed_numeric}, ColumnarValue, }; -use std::any::Any; + use std::any::Any; use std::hash::{Hash, Hasher}; -use std::hash::{Hash, Hasher}; -use std::sync::Arc; use std::sync::Arc; /// Negative expression diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 6d0391de51c1..b6129f24c5fb 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -35,7 +35,7 @@ use crate::{ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; @@ -1015,16 +1015,16 @@ impl ExecutionPlan for AggregateExec { if self.group_by.expr.is_empty() => { Ok(Statistics { - num_rows: Sharpness::Exact(1), + num_rows: Precision::Exact(1), column_statistics, - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }) } _ => Ok(Statistics { // the output row count is surely not larger than its input row count num_rows: self.input.statistics()?.num_rows, column_statistics, - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }), } } diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index 37cfcd28ce5c..649f3a31aa7e 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -30,7 +30,7 @@ use crate::{ColumnStatistics, ExecutionPlan, Statistics}; use arrow::datatypes::Schema; use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; use arrow::record_batch::RecordBatch; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; use datafusion_physical_expr::expressions::{BinaryExpr, Column}; @@ -156,14 +156,14 @@ pub fn compute_record_batch_statistics( for batch in partition { for (stat_index, col_index) in projection.iter().enumerate() { column_statistics[stat_index].null_count = - Sharpness::Exact(batch.column(*col_index).null_count()); + Precision::Exact(batch.column(*col_index).null_count()); } } } Statistics { - num_rows: Sharpness::Exact(nb_rows), - total_byte_size: Sharpness::Exact(total_byte_size), + num_rows: Precision::Exact(nb_rows), + total_byte_size: Precision::Exact(total_byte_size), column_statistics, } } @@ -675,8 +675,8 @@ mod tests { ])); let stats = compute_record_batch_statistics(&[], &schema, Some(vec![0, 1])); - assert_eq!(stats.num_rows, Sharpness::Exact(0)); - assert_eq!(stats.total_byte_size, Sharpness::Exact(0)); + assert_eq!(stats.num_rows, Precision::Exact(0)); + assert_eq!(stats.total_byte_size, Precision::Exact(0)); Ok(()) } @@ -697,20 +697,20 @@ mod tests { compute_record_batch_statistics(&[vec![batch]], &schema, Some(vec![0, 1])); let mut expected = Statistics { - num_rows: Sharpness::Exact(3), - total_byte_size: Sharpness::Exact(464), // this might change a bit if the way we compute the size changes + num_rows: Precision::Exact(3), + total_byte_size: Precision::Exact(464), // this might change a bit if the way we compute the size changes column_statistics: vec![ ColumnStatistics { - distinct_count: Sharpness::Absent, - max_value: Sharpness::Absent, - min_value: Sharpness::Absent, - null_count: Sharpness::Exact(0), + distinct_count: Precision::Absent, + max_value: Precision::Absent, + min_value: Precision::Absent, + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: Sharpness::Absent, - max_value: Sharpness::Absent, - min_value: Sharpness::Absent, - null_count: Sharpness::Exact(0), + distinct_count: Precision::Absent, + max_value: Precision::Absent, + min_value: Precision::Absent, + null_count: Precision::Exact(0), }, ], }; diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index b63c9751a171..12be05641bb3 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -37,7 +37,7 @@ use arrow::compute::filter_record_batch; use arrow::datatypes::{DataType, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::cast::as_boolean_array; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Operator; @@ -219,12 +219,12 @@ impl ExecutionPlan for FilterExec { let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); let num_rows = match num_rows.get_value() { - Some(nr) => Sharpness::Inexact((*nr as f64 * selectivity).ceil() as usize), - None => Sharpness::Absent, + Some(nr) => Precision::Inexact((*nr as f64 * selectivity).ceil() as usize), + None => Precision::Absent, }; let total_byte_size = match total_byte_size.get_value() { - Some(tbs) => Sharpness::Inexact((*tbs as f64 * selectivity).ceil() as usize), - None => Sharpness::Absent, + Some(tbs) => Precision::Inexact((*tbs as f64 * selectivity).ceil() as usize), + None => Precision::Absent, }; if let Some(analysis_boundaries) = analysis_ctx.boundaries { @@ -270,14 +270,14 @@ fn collect_new_statistics( let closed_interval = interval.close_bounds(); ColumnStatistics { null_count: match input_column_stats[idx].null_count.get_value() { - Some(nc) => Sharpness::Inexact(*nc), - None => Sharpness::Absent, + Some(nc) => Precision::Inexact(*nc), + None => Precision::Absent, }, - max_value: Sharpness::Inexact(closed_interval.upper.value), - min_value: Sharpness::Inexact(closed_interval.lower.value), + max_value: Precision::Inexact(closed_interval.upper.value), + min_value: Precision::Inexact(closed_interval.lower.value), distinct_count: match distinct_count.get_value() { - Some(dc) => Sharpness::Inexact(*dc), - None => Sharpness::Absent, + Some(dc) => Precision::Inexact(*dc), + None => Precision::Absent, }, } }, @@ -458,11 +458,11 @@ mod tests { let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(100), - total_byte_size: Sharpness::Inexact(100 * bytes_per_row), + num_rows: Precision::Inexact(100), + total_byte_size: Precision::Inexact(100 * bytes_per_row), column_statistics: vec![ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }], }, @@ -478,16 +478,16 @@ mod tests { Arc::new(FilterExec::try_new(predicate, input)?); let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Sharpness::Inexact(25)); + assert_eq!(statistics.num_rows, Precision::Inexact(25)); assert_eq!( statistics.total_byte_size, - Sharpness::Inexact(25 * bytes_per_row) + Precision::Inexact(25 * bytes_per_row) ); assert_eq!( statistics.column_statistics, vec![ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(25))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(25))), ..Default::default() }] ); @@ -502,13 +502,13 @@ mod tests { let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(100), + num_rows: Precision::Inexact(100), column_statistics: vec![ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }], - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }, schema.clone(), )); @@ -528,12 +528,12 @@ mod tests { )?); let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Sharpness::Inexact(16)); + assert_eq!(statistics.num_rows, Precision::Inexact(16)); assert_eq!( statistics.column_statistics, vec![ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(10))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(25))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(10))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(25))), ..Default::default() }] ); @@ -552,20 +552,20 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(100), + num_rows: Precision::Inexact(100), column_statistics: vec![ ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(50))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(50))), ..Default::default() }, ], - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }, schema.clone(), )); @@ -594,18 +594,18 @@ mod tests { // // Which would result with a selectivity of '15/100 * 5/50' or 0.015 // and that means about %1.5 of the all rows (rounded up to 2 rows). - assert_eq!(statistics.num_rows, Sharpness::Inexact(2)); + assert_eq!(statistics.num_rows, Precision::Inexact(2)); assert_eq!( statistics.column_statistics, vec![ ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(10))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(25))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(10))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(25))), ..Default::default() }, ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(46))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(50))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(46))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(50))), ..Default::default() } ] @@ -633,7 +633,7 @@ mod tests { Arc::new(FilterExec::try_new(predicate, input)?); let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Sharpness::Absent); + assert_eq!(statistics.num_rows, Precision::Absent); Ok(()) } @@ -651,22 +651,22 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(1000), - total_byte_size: Sharpness::Inexact(4000), + num_rows: Precision::Inexact(1000), + total_byte_size: Precision::Inexact(4000), column_statistics: vec![ ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(3))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Float32(Some(1000.0))), - max_value: Sharpness::Inexact(ScalarValue::Float32(Some(1100.0))), + min_value: Precision::Inexact(ScalarValue::Float32(Some(1000.0))), + max_value: Precision::Inexact(ScalarValue::Float32(Some(1100.0))), ..Default::default() }, ], @@ -709,22 +709,22 @@ mod tests { // 0.5 (from a) * 0.333333... (from b) * 0.798387... (from c) ≈ 0.1330... // num_rows after ceil => 133.0... => 134 // total_byte_size after ceil => 532.0... => 533 - assert_eq!(statistics.num_rows, Sharpness::Inexact(134)); - assert_eq!(statistics.total_byte_size, Sharpness::Inexact(533)); + assert_eq!(statistics.num_rows, Precision::Inexact(134)); + assert_eq!(statistics.total_byte_size, Precision::Inexact(533)); let exp_col_stats = vec![ ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(4))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(53))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(4))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(53))), ..Default::default() }, ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(3))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(3))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(3))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Float32(Some(1000.0))), - max_value: Sharpness::Inexact(ScalarValue::Float32(Some(1075.0))), + min_value: Precision::Inexact(ScalarValue::Float32(Some(1000.0))), + max_value: Precision::Inexact(ScalarValue::Float32(Some(1075.0))), ..Default::default() }, ]; @@ -769,17 +769,17 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(1000), - total_byte_size: Sharpness::Inexact(4000), + num_rows: Precision::Inexact(1000), + total_byte_size: Precision::Inexact(4000), column_statistics: vec![ ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(3))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, ], @@ -806,8 +806,8 @@ mod tests { Arc::new(FilterExec::try_new(predicate, input)?); let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Sharpness::Inexact(1000)); - assert_eq!(statistics.total_byte_size, Sharpness::Inexact(4000)); + assert_eq!(statistics.num_rows, Precision::Inexact(1000)); + assert_eq!(statistics.total_byte_size, Precision::Inexact(4000)); assert_eq!(statistics.column_statistics, expected); Ok(()) @@ -824,17 +824,17 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(1000), - total_byte_size: Sharpness::Inexact(4000), + num_rows: Precision::Inexact(1000), + total_byte_size: Precision::Inexact(4000), column_statistics: vec![ ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(3))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, ], @@ -859,19 +859,19 @@ mod tests { Arc::new(FilterExec::try_new(predicate, input)?); let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Sharpness::Inexact(0)); - assert_eq!(statistics.total_byte_size, Sharpness::Inexact(0)); + assert_eq!(statistics.num_rows, Precision::Inexact(0)); + assert_eq!(statistics.total_byte_size, Precision::Inexact(0)); assert_eq!( statistics.column_statistics, vec![ ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(3))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(3))), ..Default::default() }, ] @@ -888,17 +888,17 @@ mod tests { ]); let input = Arc::new(StatisticsExec::new( Statistics { - num_rows: Sharpness::Inexact(1000), - total_byte_size: Sharpness::Inexact(4000), + num_rows: Precision::Inexact(1000), + total_byte_size: Precision::Inexact(4000), column_statistics: vec![ ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ], @@ -915,19 +915,19 @@ mod tests { Arc::new(FilterExec::try_new(predicate, input)?); let statistics = filter.statistics()?; - assert_eq!(statistics.num_rows, Sharpness::Inexact(490)); - assert_eq!(statistics.total_byte_size, Sharpness::Inexact(1960)); + assert_eq!(statistics.num_rows, Precision::Inexact(490)); + assert_eq!(statistics.total_byte_size, Precision::Inexact(1960)); assert_eq!( statistics.column_statistics, vec![ ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(49))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(49))), ..Default::default() }, ColumnStatistics { - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(1))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(100))), + min_value: Precision::Inexact(ScalarValue::Int32(Some(1))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(100))), ..Default::default() }, ] @@ -966,13 +966,13 @@ mod tests { let filter_statistics = filter.statistics()?; let expected_filter_statistics = Statistics { - num_rows: Sharpness::Absent, - total_byte_size: Sharpness::Absent, + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, column_statistics: vec![ColumnStatistics { - null_count: Sharpness::Absent, - min_value: Sharpness::Inexact(ScalarValue::Int32(Some(5))), - max_value: Sharpness::Inexact(ScalarValue::Int32(Some(10))), - distinct_count: Sharpness::Absent, + null_count: Precision::Absent, + min_value: Precision::Inexact(ScalarValue::Int32(Some(5))), + max_value: Precision::Inexact(ScalarValue::Int32(Some(10))), + distinct_count: Precision::Absent, }], }; diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 8637da252cab..1ffd9ad1c18a 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -35,7 +35,7 @@ use crate::{ use arrow::datatypes::{Fields, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::{plan_err, DataFusionError, Result, ScalarValue}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; @@ -279,7 +279,7 @@ fn stats_cartesian_product( let total_byte_size = left_stats .total_byte_size .multiply(&right_stats.total_byte_size) - .multiply(&Sharpness::Exact(2)); + .multiply(&Precision::Exact(2)); let left_col_stats = left_stats.column_statistics; let right_col_stats = right_stats.column_statistics; @@ -463,66 +463,66 @@ mod tests { let right_bytes = 27; let left = Statistics { - num_rows: Sharpness::Exact(left_row_count), - total_byte_size: Sharpness::Exact(left_bytes), + num_rows: Precision::Exact(left_row_count), + total_byte_size: Precision::Exact(left_bytes), column_statistics: vec![ ColumnStatistics { - distinct_count: Sharpness::Exact(5), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), - null_count: Sharpness::Exact(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: Sharpness::Exact(1), - max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "x", )))), - min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "a", )))), - null_count: Sharpness::Exact(3), + null_count: Precision::Exact(3), }, ], }; let right = Statistics { - num_rows: Sharpness::Exact(right_row_count), - total_byte_size: Sharpness::Exact(right_bytes), + num_rows: Precision::Exact(right_row_count), + total_byte_size: Precision::Exact(right_bytes), column_statistics: vec![ColumnStatistics { - distinct_count: Sharpness::Exact(3), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(12))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(0))), - null_count: Sharpness::Exact(2), + distinct_count: Precision::Exact(3), + max_value: Precision::Exact(ScalarValue::Int64(Some(12))), + min_value: Precision::Exact(ScalarValue::Int64(Some(0))), + null_count: Precision::Exact(2), }], }; let result = stats_cartesian_product(left, right); let expected = Statistics { - num_rows: Sharpness::Exact(left_row_count * right_row_count), - total_byte_size: Sharpness::Exact(2 * left_bytes * right_bytes), + num_rows: Precision::Exact(left_row_count * right_row_count), + total_byte_size: Precision::Exact(2 * left_bytes * right_bytes), column_statistics: vec![ ColumnStatistics { - distinct_count: Sharpness::Exact(5), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), - null_count: Sharpness::Exact(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: Sharpness::Exact(1), - max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "x", )))), - min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "a", )))), - null_count: Sharpness::Exact(3 * right_row_count), + null_count: Precision::Exact(3 * right_row_count), }, ColumnStatistics { - distinct_count: Sharpness::Exact(3), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(12))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(0))), - null_count: Sharpness::Exact(2 * left_row_count), + distinct_count: Precision::Exact(3), + max_value: Precision::Exact(ScalarValue::Int64(Some(12))), + min_value: Precision::Exact(ScalarValue::Int64(Some(0))), + null_count: Precision::Exact(2 * left_row_count), }, ], }; @@ -535,66 +535,66 @@ mod tests { let left_row_count = 11; let left = Statistics { - num_rows: Sharpness::Exact(left_row_count), - total_byte_size: Sharpness::Exact(23), + num_rows: Precision::Exact(left_row_count), + total_byte_size: Precision::Exact(23), column_statistics: vec![ ColumnStatistics { - distinct_count: Sharpness::Exact(5), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), - null_count: Sharpness::Exact(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: Sharpness::Exact(1), - max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "x", )))), - min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "a", )))), - null_count: Sharpness::Exact(3), + null_count: Precision::Exact(3), }, ], }; let right = Statistics { - num_rows: Sharpness::Absent, - total_byte_size: Sharpness::Absent, + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, column_statistics: vec![ColumnStatistics { - distinct_count: Sharpness::Exact(3), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(12))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(0))), - null_count: Sharpness::Exact(2), + distinct_count: Precision::Exact(3), + max_value: Precision::Exact(ScalarValue::Int64(Some(12))), + min_value: Precision::Exact(ScalarValue::Int64(Some(0))), + null_count: Precision::Exact(2), }], }; let result = stats_cartesian_product(left, right); let expected = Statistics { - num_rows: Sharpness::Absent, - total_byte_size: Sharpness::Absent, + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, column_statistics: vec![ ColumnStatistics { - distinct_count: Sharpness::Exact(5), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), - null_count: Sharpness::Absent, // we don't know the row count on the right + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Absent, // we don't know the row count on the right }, ColumnStatistics { - distinct_count: Sharpness::Exact(1), - max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "x", )))), - min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "a", )))), - null_count: Sharpness::Absent, // we don't know the row count on the right + null_count: Precision::Absent, // we don't know the row count on the right }, ColumnStatistics { - distinct_count: Sharpness::Exact(3), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(12))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(0))), - null_count: Sharpness::Exact(2 * left_row_count), + distinct_count: Precision::Exact(3), + max_value: Precision::Exact(ScalarValue::Int64(Some(12))), + min_value: Precision::Exact(ScalarValue::Int64(Some(0))), + null_count: Precision::Exact(2 * left_row_count), }, ], }; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index deb21c8172e3..b2c5f16c9fd1 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -39,7 +39,7 @@ use arrow::compute; use arrow::datatypes::{Field, Schema, SchemaBuilder}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::cast::as_boolean_array; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ exec_err, plan_err, DataFusionError, JoinType, Result, SharedResult, @@ -733,12 +733,12 @@ pub(crate) fn estimate_join_statistics( let join_stats = estimate_join_cardinality(join_type, left_stats, right_stats, &on); let (num_rows, column_statistics) = match join_stats { - Some(stats) => (Sharpness::Inexact(stats.num_rows), stats.column_statistics), - None => (Sharpness::Absent, Statistics::unknown_column(schema)), + Some(stats) => (Precision::Inexact(stats.num_rows), stats.column_statistics), + None => (Precision::Absent, Statistics::unknown_column(schema)), }; Ok(Statistics { num_rows, - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, column_statistics, }) } @@ -765,12 +765,12 @@ fn estimate_join_cardinality( let ij_cardinality = estimate_inner_join_cardinality( Statistics { num_rows: left_stats.num_rows.clone(), - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, column_statistics: left_col_stats, }, Statistics { num_rows: right_stats.num_rows.clone(), - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, column_statistics: right_col_stats, }, )?; @@ -818,10 +818,10 @@ fn estimate_join_cardinality( fn estimate_inner_join_cardinality( left_stats: Statistics, right_stats: Statistics, -) -> Option> { +) -> Option> { // The algorithm here is partly based on the non-histogram selectivity estimation // from Spark's Catalyst optimizer. - let mut join_selectivity = Sharpness::Absent; + let mut join_selectivity = Precision::Absent; for (left_stat, right_stat) in left_stats .column_statistics .iter() @@ -835,9 +835,9 @@ fn estimate_inner_join_cardinality( if left_stat.min_value.is_exact().unwrap_or(false) && right_stat.max_value.is_exact().unwrap_or(false) { - Sharpness::Exact(0) + Precision::Exact(0) } else { - Sharpness::Inexact(0) + Precision::Inexact(0) }, ); } @@ -846,9 +846,9 @@ fn estimate_inner_join_cardinality( if left_stat.max_value.is_exact().unwrap_or(false) && right_stat.min_value.is_exact().unwrap_or(false) { - Sharpness::Exact(0) + Precision::Exact(0) } else { - Sharpness::Inexact(0) + Precision::Inexact(0) }, ); } @@ -870,11 +870,11 @@ fn estimate_inner_join_cardinality( let left_num_rows = left_stats.num_rows.get_value()?; let right_num_rows = right_stats.num_rows.get_value()?; match join_selectivity { - Sharpness::Exact(value) if value > 0 => { - Some(Sharpness::Exact((left_num_rows * right_num_rows) / value)) + Precision::Exact(value) if value > 0 => { + Some(Precision::Exact((left_num_rows * right_num_rows) / value)) } - Sharpness::Inexact(value) if value > 0 => { - Some(Sharpness::Inexact((left_num_rows * right_num_rows) / value)) + Precision::Inexact(value) if value > 0 => { + Some(Precision::Inexact((left_num_rows * right_num_rows) / value)) } // Since we don't have any information about the selectivity (which is derived // from the number of distinct rows information) we can give up here for now. @@ -891,15 +891,15 @@ fn estimate_inner_join_cardinality( /// has min/max values, then they might be used as a fallback option. Otherwise, /// returns None. fn max_distinct_count( - num_rows: &Sharpness, + num_rows: &Precision, stats: &ColumnStatistics, -) -> Option> { +) -> Option> { match ( &stats.distinct_count, stats.max_value.get_value(), stats.min_value.get_value(), ) { - (Sharpness::Exact(_), _, _) | (Sharpness::Inexact(_), _, _) => { + (Precision::Exact(_), _, _) | (Precision::Inexact(_), _, _) => { Some(stats.distinct_count.clone()) } (_, Some(max), Some(min)) => { @@ -920,9 +920,9 @@ fn max_distinct_count( && stats.max_value.is_exact().unwrap_or(false) && stats.min_value.is_exact().unwrap_or(false) { - Sharpness::Exact(numeric_range.min(ceiling)) + Precision::Exact(numeric_range.min(ceiling)) } else { - Sharpness::Inexact(numeric_range.min(ceiling)) + Precision::Inexact(numeric_range.min(ceiling)) }, ) } @@ -1572,13 +1572,13 @@ mod tests { ) -> Statistics { Statistics { num_rows: if is_exact { - num_rows.map(Sharpness::Exact) + num_rows.map(Precision::Exact) } else { - num_rows.map(Sharpness::Inexact) + num_rows.map(Precision::Inexact) } - .unwrap_or(Sharpness::Absent), + .unwrap_or(Precision::Absent), column_statistics: column_stats, - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, } } @@ -1589,14 +1589,14 @@ mod tests { ) -> ColumnStatistics { ColumnStatistics { distinct_count: distinct_count - .map(Sharpness::Inexact) - .unwrap_or(Sharpness::Absent), + .map(Precision::Inexact) + .unwrap_or(Precision::Absent), min_value: min - .map(|size| Sharpness::Inexact(ScalarValue::from(size))) - .unwrap_or(Sharpness::Absent), + .map(|size| Precision::Inexact(ScalarValue::from(size))) + .unwrap_or(Precision::Absent), max_value: max - .map(|size| Sharpness::Inexact(ScalarValue::from(size))) - .unwrap_or(Sharpness::Absent), + .map(|size| Precision::Inexact(ScalarValue::from(size))) + .unwrap_or(Precision::Absent), ..Default::default() } } @@ -1608,7 +1608,7 @@ mod tests { // over the expected output (since it depends on join type to join type). #[test] fn test_inner_join_cardinality_single_column() -> Result<()> { - let cases: Vec<(PartialStats, PartialStats, Option>)> = vec![ + let cases: Vec<(PartialStats, PartialStats, Option>)> = vec![ // ----------------------------------------------------------------------------- // | left(rows, min, max, distinct), right(rows, min, max, distinct), expected | // ----------------------------------------------------------------------------- @@ -1620,55 +1620,55 @@ mod tests { ( (10, Some(1), Some(10), None), (10, Some(1), Some(10), None), - Some(Sharpness::Inexact(10)), + Some(Precision::Inexact(10)), ), // range(left) > range(right) ( (10, Some(6), Some(10), None), (10, Some(8), Some(10), None), - Some(Sharpness::Inexact(20)), + Some(Precision::Inexact(20)), ), // range(right) > range(left) ( (10, Some(8), Some(10), None), (10, Some(6), Some(10), None), - Some(Sharpness::Inexact(20)), + Some(Precision::Inexact(20)), ), // range(left) > len(left), range(right) > len(right) ( (10, Some(1), Some(15), None), (20, Some(1), Some(40), None), - Some(Sharpness::Inexact(10)), + Some(Precision::Inexact(10)), ), // When we have distinct count. ( (10, Some(1), Some(10), Some(10)), (10, Some(1), Some(10), Some(10)), - Some(Sharpness::Inexact(10)), + Some(Precision::Inexact(10)), ), // distinct(left) > distinct(right) ( (10, Some(1), Some(10), Some(5)), (10, Some(1), Some(10), Some(2)), - Some(Sharpness::Inexact(20)), + Some(Precision::Inexact(20)), ), // distinct(right) > distinct(left) ( (10, Some(1), Some(10), Some(2)), (10, Some(1), Some(10), Some(5)), - Some(Sharpness::Inexact(20)), + Some(Precision::Inexact(20)), ), // min(left) < 0 (range(left) > range(right)) ( (10, Some(-5), Some(5), None), (10, Some(1), Some(5), None), - Some(Sharpness::Inexact(10)), + Some(Precision::Inexact(10)), ), // min(right) < 0, max(right) < 0 (range(right) > range(left)) ( (10, Some(-25), Some(-20), None), (10, Some(-25), Some(-15), None), - Some(Sharpness::Inexact(10)), + Some(Precision::Inexact(10)), ), // range(left) < 0, range(right) >= 0 // (there isn't a case where both left and right ranges are negative @@ -1677,13 +1677,13 @@ mod tests { ( (10, Some(-10), Some(0), None), (10, Some(0), Some(10), Some(5)), - Some(Sharpness::Inexact(10)), + Some(Precision::Inexact(10)), ), // range(left) = 1, range(right) = 1 ( (10, Some(1), Some(1), None), (10, Some(1), Some(1), None), - Some(Sharpness::Inexact(100)), + Some(Precision::Inexact(100)), ), // // Edge cases @@ -1708,12 +1708,12 @@ mod tests { ( (10, Some(0), Some(10), None), (10, Some(11), Some(20), None), - Some(Sharpness::Inexact(0)), + Some(Precision::Inexact(0)), ), ( (10, Some(11), Some(20), None), (10, Some(0), Some(10), None), - Some(Sharpness::Inexact(0)), + Some(Precision::Inexact(0)), ), // distinct(left) = 0, distinct(right) = 0 ( @@ -1738,13 +1738,13 @@ mod tests { assert_eq!( estimate_inner_join_cardinality( Statistics { - num_rows: Sharpness::Inexact(left_num_rows), - total_byte_size: Sharpness::Absent, + num_rows: Precision::Inexact(left_num_rows), + total_byte_size: Precision::Absent, column_statistics: left_col_stats.clone(), }, Statistics { - num_rows: Sharpness::Inexact(right_num_rows), - total_byte_size: Sharpness::Absent, + num_rows: Precision::Inexact(right_num_rows), + total_byte_size: Precision::Absent, column_statistics: right_col_stats.clone(), }, ), @@ -1764,7 +1764,7 @@ mod tests { assert_eq!( partial_join_stats .clone() - .map(|s| Sharpness::Inexact(s.num_rows)), + .map(|s| Precision::Inexact(s.num_rows)), expected_cardinality.clone() ); assert_eq!( @@ -1794,17 +1794,17 @@ mod tests { assert_eq!( estimate_inner_join_cardinality( Statistics { - num_rows: Sharpness::Inexact(400), - total_byte_size: Sharpness::Absent, + num_rows: Precision::Inexact(400), + total_byte_size: Precision::Absent, column_statistics: left_col_stats, }, Statistics { - num_rows: Sharpness::Inexact(400), - total_byte_size: Sharpness::Absent, + num_rows: Precision::Inexact(400), + total_byte_size: Precision::Absent, column_statistics: right_col_stats, }, ), - Some(Sharpness::Inexact((400 * 400) / 200)) + Some(Precision::Inexact((400 * 400) / 200)) ); Ok(()) } @@ -1812,29 +1812,29 @@ mod tests { #[test] fn test_inner_join_cardinality_decimal_range() -> Result<()> { let left_col_stats = vec![ColumnStatistics { - distinct_count: Sharpness::Absent, - min_value: Sharpness::Inexact(ScalarValue::Decimal128(Some(32500), 14, 4)), - max_value: Sharpness::Inexact(ScalarValue::Decimal128(Some(35000), 14, 4)), + distinct_count: Precision::Absent, + min_value: Precision::Inexact(ScalarValue::Decimal128(Some(32500), 14, 4)), + max_value: Precision::Inexact(ScalarValue::Decimal128(Some(35000), 14, 4)), ..Default::default() }]; let right_col_stats = vec![ColumnStatistics { - distinct_count: Sharpness::Absent, - min_value: Sharpness::Inexact(ScalarValue::Decimal128(Some(33500), 14, 4)), - max_value: Sharpness::Inexact(ScalarValue::Decimal128(Some(34000), 14, 4)), + distinct_count: Precision::Absent, + min_value: Precision::Inexact(ScalarValue::Decimal128(Some(33500), 14, 4)), + max_value: Precision::Inexact(ScalarValue::Decimal128(Some(34000), 14, 4)), ..Default::default() }]; assert_eq!( estimate_inner_join_cardinality( Statistics { - num_rows: Sharpness::Inexact(100), - total_byte_size: Sharpness::Absent, + num_rows: Precision::Inexact(100), + total_byte_size: Precision::Absent, column_statistics: left_col_stats, }, Statistics { - num_rows: Sharpness::Inexact(100), - total_byte_size: Sharpness::Absent, + num_rows: Precision::Inexact(100), + total_byte_size: Precision::Absent, column_statistics: right_col_stats, }, ), diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index f2e2cc93afd7..9fe1eb0763ba 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -32,7 +32,7 @@ use crate::{ use arrow::array::ArrayRef; use arrow::datatypes::SchemaRef; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::OrderingEquivalenceProperties; @@ -207,26 +207,26 @@ impl ExecutionPlan for GlobalLimitExec { let col_stats = Statistics::unknown_column(&self.schema()); let fetched_row_number_stats = Statistics { - num_rows: Sharpness::Exact(max_row_num), + num_rows: Precision::Exact(max_row_num), column_statistics: col_stats.clone(), - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }; let stats = match input_stats { Statistics { - num_rows: Sharpness::Exact(nr), + num_rows: Precision::Exact(nr), .. } | Statistics { - num_rows: Sharpness::Inexact(nr), + num_rows: Precision::Inexact(nr), .. } => { if nr <= skip { // if all input data will be skipped, return 0 Statistics { - num_rows: Sharpness::Exact(0), + num_rows: Precision::Exact(0), column_statistics: col_stats, - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, } } else if nr <= max_row_num { // if the input does not reach the "fetch" globally, return input stats @@ -374,44 +374,44 @@ impl ExecutionPlan for LocalLimitExec { let stats = match input_stats { // if the input does not reach the limit globally, return input stats Statistics { - num_rows: Sharpness::Exact(nr), + num_rows: Precision::Exact(nr), .. } | Statistics { - num_rows: Sharpness::Inexact(nr), + num_rows: Precision::Inexact(nr), .. } if nr <= self.fetch => input_stats, // if the input is greater than the limit, the num_row will be greater // than the limit because the partitions will be limited separatly // the statistic Statistics { - num_rows: Sharpness::Exact(nr), + num_rows: Precision::Exact(nr), .. } if nr > self.fetch => Statistics { - num_rows: Sharpness::Exact(self.fetch), + num_rows: Precision::Exact(self.fetch), // this is not actually exact, but will be when GlobalLimit is applied // TODO stats: find a more explicit way to vehiculate this information column_statistics: col_stats, - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }, Statistics { - num_rows: Sharpness::Inexact(nr), + num_rows: Precision::Inexact(nr), .. } if nr > self.fetch => Statistics { - num_rows: Sharpness::Inexact(self.fetch), + num_rows: Precision::Inexact(self.fetch), // this is not actually exact, but will be when GlobalLimit is applied // TODO stats: find a more explicit way to vehiculate this information column_statistics: col_stats, - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }, _ => Statistics { // the result output row number will always be no greater than the limit number - num_rows: Sharpness::Inexact( + num_rows: Precision::Inexact( self.fetch * self.output_partitioning().partition_count(), ), column_statistics: col_stats, - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }, }; Ok(stats) @@ -754,10 +754,10 @@ mod tests { #[tokio::test] async fn test_row_number_statistics_for_global_limit() -> Result<()> { let row_count = row_number_statistics_for_global_limit(0, Some(10)).await?; - assert_eq!(row_count, Sharpness::Exact(10)); + assert_eq!(row_count, Precision::Exact(10)); let row_count = row_number_statistics_for_global_limit(5, Some(10)).await?; - assert_eq!(row_count, Sharpness::Exact(15)); + assert_eq!(row_count, Precision::Exact(15)); Ok(()) } @@ -765,7 +765,7 @@ mod tests { #[tokio::test] async fn test_row_number_statistics_for_local_limit() -> Result<()> { let row_count = row_number_statistics_for_local_limit(4, 10).await?; - assert_eq!(row_count, Sharpness::Exact(10)); + assert_eq!(row_count, Precision::Exact(10)); Ok(()) } @@ -773,7 +773,7 @@ mod tests { async fn row_number_statistics_for_global_limit( skip: usize, fetch: Option, - ) -> Result> { + ) -> Result> { let num_partitions = 4; let csv = test::scan_partitioned(num_partitions); @@ -788,7 +788,7 @@ mod tests { async fn row_number_statistics_for_local_limit( num_partitions: usize, fetch: usize, - ) -> Result> { + ) -> Result> { let csv = test::scan_partitioned(num_partitions); assert_eq!(csv.output_partitioning().partition_count(), num_partitions); diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index e27193aa2378..029dd24d7d11 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -36,7 +36,7 @@ use crate::{ use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; @@ -420,7 +420,7 @@ fn stats_projection( if primitive_row_size_possible { stats.total_byte_size = - Sharpness::Exact(primitive_row_size).multiply(&stats.num_rows); + Precision::Exact(primitive_row_size).multiply(&stats.num_rows); } stats.column_statistics = column_statistics; stats @@ -510,30 +510,30 @@ mod tests { fn get_stats() -> Statistics { Statistics { - num_rows: Sharpness::Exact(5), - total_byte_size: Sharpness::Exact(23), + num_rows: Precision::Exact(5), + total_byte_size: Precision::Exact(23), column_statistics: vec![ ColumnStatistics { - distinct_count: Sharpness::Exact(5), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), - null_count: Sharpness::Exact(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: Sharpness::Exact(1), - max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "x", )))), - min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "a", )))), - null_count: Sharpness::Exact(3), + null_count: Precision::Exact(3), }, ColumnStatistics { - distinct_count: Sharpness::Absent, - max_value: Sharpness::Exact(ScalarValue::Float32(Some(1.1))), - min_value: Sharpness::Exact(ScalarValue::Float32(Some(0.1))), - null_count: Sharpness::Absent, + distinct_count: Precision::Absent, + max_value: Precision::Exact(ScalarValue::Float32(Some(1.1))), + min_value: Precision::Exact(ScalarValue::Float32(Some(0.1))), + null_count: Precision::Absent, }, ], } @@ -558,24 +558,24 @@ mod tests { let result = stats_projection(source, exprs.into_iter(), Arc::new(schema)); let expected = Statistics { - num_rows: Sharpness::Exact(5), - total_byte_size: Sharpness::Exact(23), + num_rows: Precision::Exact(5), + total_byte_size: Precision::Exact(23), column_statistics: vec![ ColumnStatistics { - distinct_count: Sharpness::Exact(1), - max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "x", )))), - min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "a", )))), - null_count: Sharpness::Exact(3), + null_count: Precision::Exact(3), }, ColumnStatistics { - distinct_count: Sharpness::Exact(5), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), - null_count: Sharpness::Exact(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, ], }; @@ -596,20 +596,20 @@ mod tests { let result = stats_projection(source, exprs.into_iter(), Arc::new(schema)); let expected = Statistics { - num_rows: Sharpness::Exact(5), - total_byte_size: Sharpness::Exact(60), + num_rows: Precision::Exact(5), + total_byte_size: Precision::Exact(60), column_statistics: vec![ ColumnStatistics { - distinct_count: Sharpness::Absent, - max_value: Sharpness::Exact(ScalarValue::Float32(Some(1.1))), - min_value: Sharpness::Exact(ScalarValue::Float32(Some(0.1))), - null_count: Sharpness::Absent, + distinct_count: Precision::Absent, + max_value: Precision::Exact(ScalarValue::Float32(Some(1.1))), + min_value: Precision::Exact(ScalarValue::Float32(Some(0.1))), + null_count: Precision::Absent, }, ColumnStatistics { - distinct_count: Sharpness::Exact(5), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), - null_count: Sharpness::Exact(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, ], }; diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 8357e2d1bc65..2727a6e161be 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -37,7 +37,7 @@ use crate::stream::ObservedStream; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::{exec_err, internal_err, DFSchemaRef, DataFusionError, Result}; use datafusion_execution::TaskContext; @@ -572,7 +572,7 @@ fn col_stats_union( mut left: ColumnStatistics, right: ColumnStatistics, ) -> ColumnStatistics { - left.distinct_count = Sharpness::Absent; + left.distinct_count = Precision::Absent; left.min_value = left.min_value.min(&right.min_value); left.max_value = left.max_value.max(&right.max_value); left.null_count = left.null_count.add(&right.null_count); @@ -623,89 +623,89 @@ mod tests { #[tokio::test] async fn test_stats_union() { let left = Statistics { - num_rows: Sharpness::Exact(5), - total_byte_size: Sharpness::Exact(23), + num_rows: Precision::Exact(5), + total_byte_size: Precision::Exact(23), column_statistics: vec![ ColumnStatistics { - distinct_count: Sharpness::Exact(5), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(21))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), - null_count: Sharpness::Exact(0), + distinct_count: Precision::Exact(5), + max_value: Precision::Exact(ScalarValue::Int64(Some(21))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(0), }, ColumnStatistics { - distinct_count: Sharpness::Exact(1), - max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + distinct_count: Precision::Exact(1), + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "x", )))), - min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "a", )))), - null_count: Sharpness::Exact(3), + null_count: Precision::Exact(3), }, ColumnStatistics { - distinct_count: Sharpness::Absent, - max_value: Sharpness::Exact(ScalarValue::Float32(Some(1.1))), - min_value: Sharpness::Exact(ScalarValue::Float32(Some(0.1))), - null_count: Sharpness::Absent, + distinct_count: Precision::Absent, + max_value: Precision::Exact(ScalarValue::Float32(Some(1.1))), + min_value: Precision::Exact(ScalarValue::Float32(Some(0.1))), + null_count: Precision::Absent, }, ], }; let right = Statistics { - num_rows: Sharpness::Exact(7), - total_byte_size: Sharpness::Exact(29), + num_rows: Precision::Exact(7), + total_byte_size: Precision::Exact(29), column_statistics: vec![ ColumnStatistics { - distinct_count: Sharpness::Exact(3), - max_value: Sharpness::Exact(ScalarValue::Int64(Some(34))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(1))), - null_count: Sharpness::Exact(1), + distinct_count: Precision::Exact(3), + max_value: Precision::Exact(ScalarValue::Int64(Some(34))), + min_value: Precision::Exact(ScalarValue::Int64(Some(1))), + null_count: Precision::Exact(1), }, ColumnStatistics { - distinct_count: Sharpness::Absent, - max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + distinct_count: Precision::Absent, + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "c", )))), - min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "b", )))), - null_count: Sharpness::Absent, + null_count: Precision::Absent, }, ColumnStatistics { - distinct_count: Sharpness::Absent, - max_value: Sharpness::Absent, - min_value: Sharpness::Absent, - null_count: Sharpness::Absent, + distinct_count: Precision::Absent, + max_value: Precision::Absent, + min_value: Precision::Absent, + null_count: Precision::Absent, }, ], }; let result = stats_union(left, right); let expected = Statistics { - num_rows: Sharpness::Exact(12), - total_byte_size: Sharpness::Exact(52), + num_rows: Precision::Exact(12), + total_byte_size: Precision::Exact(52), column_statistics: vec![ ColumnStatistics { - distinct_count: Sharpness::Absent, - max_value: Sharpness::Exact(ScalarValue::Int64(Some(34))), - min_value: Sharpness::Exact(ScalarValue::Int64(Some(-4))), - null_count: Sharpness::Exact(1), + distinct_count: Precision::Absent, + max_value: Precision::Exact(ScalarValue::Int64(Some(34))), + min_value: Precision::Exact(ScalarValue::Int64(Some(-4))), + null_count: Precision::Exact(1), }, ColumnStatistics { - distinct_count: Sharpness::Absent, - max_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + distinct_count: Precision::Absent, + max_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "x", )))), - min_value: Sharpness::Exact(ScalarValue::Utf8(Some(String::from( + min_value: Precision::Exact(ScalarValue::Utf8(Some(String::from( "a", )))), - null_count: Sharpness::Absent, + null_count: Precision::Absent, }, ColumnStatistics { - distinct_count: Sharpness::Absent, - max_value: Sharpness::Absent, - min_value: Sharpness::Absent, - null_count: Sharpness::Absent, + distinct_count: Precision::Absent, + max_value: Precision::Absent, + min_value: Precision::Absent, + null_count: Precision::Absent, }, ], }; 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 54876d57d8a4..eab47886c764 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -44,7 +44,7 @@ use arrow::{ record_batch::RecordBatch, }; use datafusion_common::hash_utils::create_hashes; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::utils::{ evaluate_partition_ranges, get_arrayref_at_indices, get_at_indices, get_record_batch_at_indices, get_row_at_idx, @@ -334,7 +334,7 @@ impl ExecutionPlan for BoundedWindowAggExec { Ok(Statistics { num_rows: input_stat.num_rows, column_statistics, - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }) } } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index cef134e0e62b..4873778425d3 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -42,7 +42,7 @@ use arrow::{ datatypes::{Schema, SchemaRef}, record_batch::RecordBatch, }; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::utils::{evaluate_partition_ranges, get_at_indices}; use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; @@ -261,7 +261,7 @@ impl ExecutionPlan for WindowAggExec { Ok(Statistics { num_rows: input_stat.num_rows, column_statistics, - total_byte_size: Sharpness::Absent, + total_byte_size: Precision::Absent, }) } } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 09516e0e9a48..b41419d6615a 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1545,28 +1545,28 @@ message PartitionStats { repeated ColumnStats column_stats = 4; } -message Sharpness{ - SharpnessInfo sharpness_info = 1; +message Precision{ + PrecisionInfo precision_info = 1; ScalarValue val = 2; } -enum SharpnessInfo { +enum PrecisionInfo { EXACT = 0; INEXACT = 1; ABSENT = 2; } message Statistics { - Sharpness num_rows = 1; - Sharpness total_byte_size = 2; + Precision num_rows = 1; + Precision total_byte_size = 2; repeated ColumnStats column_stats = 3; } message ColumnStats { - Sharpness min_value = 1; - Sharpness max_value = 2; - Sharpness null_count = 3; - Sharpness distinct_count = 4; + Precision min_value = 1; + Precision max_value = 2; + Precision null_count = 3; + Precision distinct_count = 4; } message NamedStructFieldExpr { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index dba38b5de169..99c2f4f1dc82 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -18316,6 +18316,191 @@ impl<'de> serde::Deserialize<'de> for PlanType { deserializer.deserialize_struct("datafusion.PlanType", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for Precision { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.precision_info != 0 { + len += 1; + } + if self.val.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.Precision", len)?; + if self.precision_info != 0 { + let v = PrecisionInfo::try_from(self.precision_info) + .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.precision_info)))?; + struct_ser.serialize_field("precisionInfo", &v)?; + } + if let Some(v) = self.val.as_ref() { + struct_ser.serialize_field("val", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for Precision { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "precision_info", + "precisionInfo", + "val", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + PrecisionInfo, + Val, + } + 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 { + "precisionInfo" | "precision_info" => Ok(GeneratedField::PrecisionInfo), + "val" => Ok(GeneratedField::Val), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = Precision; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.Precision") + } + + fn visit_map(self, mut map_: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut precision_info__ = None; + let mut val__ = None; + while let Some(k) = map_.next_key()? { + match k { + GeneratedField::PrecisionInfo => { + if precision_info__.is_some() { + return Err(serde::de::Error::duplicate_field("precisionInfo")); + } + precision_info__ = Some(map_.next_value::()? as i32); + } + GeneratedField::Val => { + if val__.is_some() { + return Err(serde::de::Error::duplicate_field("val")); + } + val__ = map_.next_value()?; + } + } + } + Ok(Precision { + precision_info: precision_info__.unwrap_or_default(), + val: val__, + }) + } + } + deserializer.deserialize_struct("datafusion.Precision", FIELDS, GeneratedVisitor) + } +} +impl serde::Serialize for PrecisionInfo { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + let variant = match self { + Self::Exact => "EXACT", + Self::Inexact => "INEXACT", + Self::Absent => "ABSENT", + }; + serializer.serialize_str(variant) + } +} +impl<'de> serde::Deserialize<'de> for PrecisionInfo { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "EXACT", + "INEXACT", + "ABSENT", + ]; + + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = PrecisionInfo; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + fn visit_i64(self, v: i64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) + }) + } + + fn visit_u64(self, v: u64) -> std::result::Result + where + E: serde::de::Error, + { + i32::try_from(v) + .ok() + .and_then(|x| x.try_into().ok()) + .ok_or_else(|| { + serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) + }) + } + + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "EXACT" => Ok(PrecisionInfo::Exact), + "INEXACT" => Ok(PrecisionInfo::Inexact), + "ABSENT" => Ok(PrecisionInfo::Absent), + _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), + } + } + } + deserializer.deserialize_any(GeneratedVisitor) + } +} impl serde::Serialize for PrepareNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result @@ -21550,191 +21735,6 @@ impl<'de> serde::Deserialize<'de> for SelectionNode { deserializer.deserialize_struct("datafusion.SelectionNode", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for Sharpness { - #[allow(deprecated)] - fn serialize(&self, serializer: S) -> std::result::Result - where - S: serde::Serializer, - { - use serde::ser::SerializeStruct; - let mut len = 0; - if self.sharpness_info != 0 { - len += 1; - } - if self.val.is_some() { - len += 1; - } - let mut struct_ser = serializer.serialize_struct("datafusion.Sharpness", len)?; - if self.sharpness_info != 0 { - let v = SharpnessInfo::try_from(self.sharpness_info) - .map_err(|_| serde::ser::Error::custom(format!("Invalid variant {}", self.sharpness_info)))?; - struct_ser.serialize_field("sharpnessInfo", &v)?; - } - if let Some(v) = self.val.as_ref() { - struct_ser.serialize_field("val", v)?; - } - struct_ser.end() - } -} -impl<'de> serde::Deserialize<'de> for Sharpness { - #[allow(deprecated)] - fn deserialize(deserializer: D) -> std::result::Result - where - D: serde::Deserializer<'de>, - { - const FIELDS: &[&str] = &[ - "sharpness_info", - "sharpnessInfo", - "val", - ]; - - #[allow(clippy::enum_variant_names)] - enum GeneratedField { - SharpnessInfo, - Val, - } - 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 { - "sharpnessInfo" | "sharpness_info" => Ok(GeneratedField::SharpnessInfo), - "val" => Ok(GeneratedField::Val), - _ => Err(serde::de::Error::unknown_field(value, FIELDS)), - } - } - } - deserializer.deserialize_identifier(GeneratedVisitor) - } - } - struct GeneratedVisitor; - impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = Sharpness; - - fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct datafusion.Sharpness") - } - - fn visit_map(self, mut map_: V) -> std::result::Result - where - V: serde::de::MapAccess<'de>, - { - let mut sharpness_info__ = None; - let mut val__ = None; - while let Some(k) = map_.next_key()? { - match k { - GeneratedField::SharpnessInfo => { - if sharpness_info__.is_some() { - return Err(serde::de::Error::duplicate_field("sharpnessInfo")); - } - sharpness_info__ = Some(map_.next_value::()? as i32); - } - GeneratedField::Val => { - if val__.is_some() { - return Err(serde::de::Error::duplicate_field("val")); - } - val__ = map_.next_value()?; - } - } - } - Ok(Sharpness { - sharpness_info: sharpness_info__.unwrap_or_default(), - val: val__, - }) - } - } - deserializer.deserialize_struct("datafusion.Sharpness", FIELDS, GeneratedVisitor) - } -} -impl serde::Serialize for SharpnessInfo { - #[allow(deprecated)] - fn serialize(&self, serializer: S) -> std::result::Result - where - S: serde::Serializer, - { - let variant = match self { - Self::Exact => "EXACT", - Self::Inexact => "INEXACT", - Self::Absent => "ABSENT", - }; - serializer.serialize_str(variant) - } -} -impl<'de> serde::Deserialize<'de> for SharpnessInfo { - #[allow(deprecated)] - fn deserialize(deserializer: D) -> std::result::Result - where - D: serde::Deserializer<'de>, - { - const FIELDS: &[&str] = &[ - "EXACT", - "INEXACT", - "ABSENT", - ]; - - struct GeneratedVisitor; - - impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = SharpnessInfo; - - fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(formatter, "expected one of: {:?}", &FIELDS) - } - - fn visit_i64(self, v: i64) -> std::result::Result - where - E: serde::de::Error, - { - i32::try_from(v) - .ok() - .and_then(|x| x.try_into().ok()) - .ok_or_else(|| { - serde::de::Error::invalid_value(serde::de::Unexpected::Signed(v), &self) - }) - } - - fn visit_u64(self, v: u64) -> std::result::Result - where - E: serde::de::Error, - { - i32::try_from(v) - .ok() - .and_then(|x| x.try_into().ok()) - .ok_or_else(|| { - serde::de::Error::invalid_value(serde::de::Unexpected::Unsigned(v), &self) - }) - } - - fn visit_str(self, value: &str) -> std::result::Result - where - E: serde::de::Error, - { - match value { - "EXACT" => Ok(SharpnessInfo::Exact), - "INEXACT" => Ok(SharpnessInfo::Inexact), - "ABSENT" => Ok(SharpnessInfo::Absent), - _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), - } - } - } - deserializer.deserialize_any(GeneratedVisitor) - } -} impl serde::Serialize for SimilarToNode { #[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 9e882cfc4d4d..ddd5d7b66965 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2208,9 +2208,9 @@ pub struct PartitionStats { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct Sharpness { - #[prost(enumeration = "SharpnessInfo", tag = "1")] - pub sharpness_info: i32, +pub struct Precision { + #[prost(enumeration = "PrecisionInfo", tag = "1")] + pub precision_info: i32, #[prost(message, optional, tag = "2")] pub val: ::core::option::Option, } @@ -2218,9 +2218,9 @@ pub struct Sharpness { #[derive(Clone, PartialEq, ::prost::Message)] pub struct Statistics { #[prost(message, optional, tag = "1")] - pub num_rows: ::core::option::Option, + pub num_rows: ::core::option::Option, #[prost(message, optional, tag = "2")] - pub total_byte_size: ::core::option::Option, + pub total_byte_size: ::core::option::Option, #[prost(message, repeated, tag = "3")] pub column_stats: ::prost::alloc::vec::Vec, } @@ -2228,13 +2228,13 @@ pub struct Statistics { #[derive(Clone, PartialEq, ::prost::Message)] pub struct ColumnStats { #[prost(message, optional, tag = "1")] - pub min_value: ::core::option::Option, + pub min_value: ::core::option::Option, #[prost(message, optional, tag = "2")] - pub max_value: ::core::option::Option, + pub max_value: ::core::option::Option, #[prost(message, optional, tag = "3")] - pub null_count: ::core::option::Option, + pub null_count: ::core::option::Option, #[prost(message, optional, tag = "4")] - pub distinct_count: ::core::option::Option, + pub distinct_count: ::core::option::Option, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -3168,21 +3168,21 @@ impl JoinSide { } #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] -pub enum SharpnessInfo { +pub enum PrecisionInfo { Exact = 0, Inexact = 1, Absent = 2, } -impl SharpnessInfo { +impl PrecisionInfo { /// String value of the enum field names used in the ProtoBuf definition. /// /// The values are not transformed in any way and thus are considered stable /// (if the ProtoBuf definition does not change) and safe for programmatic use. pub fn as_str_name(&self) -> &'static str { match self { - SharpnessInfo::Exact => "EXACT", - SharpnessInfo::Inexact => "INEXACT", - SharpnessInfo::Absent => "ABSENT", + PrecisionInfo::Exact => "EXACT", + PrecisionInfo::Inexact => "INEXACT", + PrecisionInfo::Absent => "ABSENT", } } /// Creates an enum from field names used in the ProtoBuf definition. diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 4cc80d65f64e..ff02b8052136 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -41,7 +41,7 @@ use datafusion::physical_plan::windows::create_window_expr; use datafusion::physical_plan::{ functions, ColumnStatistics, Partitioning, PhysicalExpr, Statistics, WindowExpr, }; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::{not_impl_err, DataFusionError, Result, ScalarValue}; use crate::common::proto_error; @@ -583,93 +583,93 @@ impl From<&protobuf::ColumnStats> for ColumnStatistics { null_count: if let Some(nc) = &cs.null_count { nc.clone().into() } else { - Sharpness::Absent + Precision::Absent }, max_value: if let Some(max) = &cs.max_value { max.clone().into() } else { - Sharpness::Absent + Precision::Absent }, min_value: if let Some(min) = &cs.min_value { min.clone().into() } else { - Sharpness::Absent + Precision::Absent }, distinct_count: if let Some(dc) = &cs.distinct_count { dc.clone().into() } else { - Sharpness::Absent + Precision::Absent }, } } } -impl From for Sharpness { - fn from(s: protobuf::Sharpness) -> Self { - let Ok(sharpness_type) = s.sharpness_info.try_into() else { - return Sharpness::Absent; +impl From for Precision { + fn from(s: protobuf::Precision) -> Self { + let Ok(precision_type) = s.precision_info.try_into() else { + return Precision::Absent; }; - match sharpness_type { - protobuf::SharpnessInfo::Exact => { + match precision_type { + protobuf::PrecisionInfo::Exact => { if let Some(val) = s.val { if let Ok(ScalarValue::UInt64(Some(val))) = ScalarValue::try_from(&val) { - Sharpness::Exact(val as usize) + Precision::Exact(val as usize) } else { - Sharpness::Absent + Precision::Absent } } else { - Sharpness::Absent + Precision::Absent } } - protobuf::SharpnessInfo::Inexact => { + protobuf::PrecisionInfo::Inexact => { if let Some(val) = s.val { if let Ok(ScalarValue::UInt64(Some(val))) = ScalarValue::try_from(&val) { - Sharpness::Inexact(val as usize) + Precision::Inexact(val as usize) } else { - Sharpness::Absent + Precision::Absent } } else { - Sharpness::Absent + Precision::Absent } } - protobuf::SharpnessInfo::Absent => Sharpness::Absent, + protobuf::PrecisionInfo::Absent => Precision::Absent, } } } -impl From for Sharpness { - fn from(s: protobuf::Sharpness) -> Self { - let Ok(sharpness_type) = s.sharpness_info.try_into() else { - return Sharpness::Absent; +impl From for Precision { + fn from(s: protobuf::Precision) -> Self { + let Ok(precision_type) = s.precision_info.try_into() else { + return Precision::Absent; }; - match sharpness_type { - protobuf::SharpnessInfo::Exact => { + match precision_type { + protobuf::PrecisionInfo::Exact => { if let Some(val) = s.val { if let Ok(val) = ScalarValue::try_from(&val) { - Sharpness::Exact(val) + Precision::Exact(val) } else { - Sharpness::Absent + Precision::Absent } } else { - Sharpness::Absent + Precision::Absent } } - protobuf::SharpnessInfo::Inexact => { + protobuf::PrecisionInfo::Inexact => { if let Some(val) = s.val { if let Ok(val) = ScalarValue::try_from(&val) { - Sharpness::Inexact(val) + Precision::Inexact(val) } else { - Sharpness::Absent + Precision::Absent } } else { - Sharpness::Absent + Precision::Absent } } - protobuf::SharpnessInfo::Absent => Sharpness::Absent, + protobuf::PrecisionInfo::Absent => Precision::Absent, } } } @@ -692,12 +692,12 @@ impl TryFrom<&protobuf::Statistics> for Statistics { num_rows: if let Some(nr) = &s.num_rows { nr.clone().into() } else { - Sharpness::Absent + Precision::Absent }, total_byte_size: if let Some(tbs) = &s.total_byte_size { tbs.clone().into() } else { - Sharpness::Absent + Precision::Absent }, // No column statistic (None) is encoded with empty array column_statistics: s.column_stats.iter().map(|s| s.into()).collect(), diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index c50258c8267c..db97c393255a 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -51,7 +51,7 @@ use datafusion::physical_plan::{ AggregateExpr, ColumnStatistics, PhysicalExpr, Statistics, WindowExpr, }; use datafusion_common::{ - internal_err, not_impl_err, stats::Sharpness, DataFusionError, Result, + internal_err, not_impl_err, stats::Precision, DataFusionError, Result, }; impl TryFrom> for protobuf::PhysicalExprNode { @@ -623,42 +623,42 @@ impl TryFrom<&[PartitionedFile]> for protobuf::FileGroup { } } -impl From<&Sharpness> for protobuf::Sharpness { - fn from(s: &Sharpness) -> protobuf::Sharpness { +impl From<&Precision> for protobuf::Precision { + fn from(s: &Precision) -> protobuf::Precision { match s { - Sharpness::Exact(val) => protobuf::Sharpness { - sharpness_info: protobuf::SharpnessInfo::Exact.into(), + Precision::Exact(val) => protobuf::Precision { + precision_info: protobuf::PrecisionInfo::Exact.into(), val: Some(ScalarValue { value: Some(Value::Uint64Value(*val as u64)), }), }, - Sharpness::Inexact(val) => protobuf::Sharpness { - sharpness_info: protobuf::SharpnessInfo::Inexact.into(), + Precision::Inexact(val) => protobuf::Precision { + precision_info: protobuf::PrecisionInfo::Inexact.into(), val: Some(ScalarValue { value: Some(Value::Uint64Value(*val as u64)), }), }, - Sharpness::Absent => protobuf::Sharpness { - sharpness_info: protobuf::SharpnessInfo::Absent.into(), + Precision::Absent => protobuf::Precision { + precision_info: protobuf::PrecisionInfo::Absent.into(), val: Some(ScalarValue { value: None }), }, } } } -impl From<&Sharpness> for protobuf::Sharpness { - fn from(s: &Sharpness) -> protobuf::Sharpness { +impl From<&Precision> for protobuf::Precision { + fn from(s: &Precision) -> protobuf::Precision { match s { - Sharpness::Exact(val) => protobuf::Sharpness { - sharpness_info: protobuf::SharpnessInfo::Exact.into(), + Precision::Exact(val) => protobuf::Precision { + precision_info: protobuf::PrecisionInfo::Exact.into(), val: val.try_into().ok(), }, - Sharpness::Inexact(val) => protobuf::Sharpness { - sharpness_info: protobuf::SharpnessInfo::Inexact.into(), + Precision::Inexact(val) => protobuf::Precision { + precision_info: protobuf::PrecisionInfo::Inexact.into(), val: val.try_into().ok(), }, - Sharpness::Absent => protobuf::Sharpness { - sharpness_info: protobuf::SharpnessInfo::Absent.into(), + Precision::Absent => protobuf::Precision { + precision_info: protobuf::PrecisionInfo::Absent.into(), val: Some(ScalarValue { value: None }), }, } @@ -669,8 +669,8 @@ impl From<&Statistics> for protobuf::Statistics { fn from(s: &Statistics) -> protobuf::Statistics { let column_stats = s.column_statistics.iter().map(|s| s.into()).collect(); protobuf::Statistics { - num_rows: Some(protobuf::Sharpness::from(&s.num_rows)), - total_byte_size: Some(protobuf::Sharpness::from(&s.total_byte_size)), + num_rows: Some(protobuf::Precision::from(&s.num_rows)), + total_byte_size: Some(protobuf::Precision::from(&s.total_byte_size)), column_stats, } } @@ -679,10 +679,10 @@ impl From<&Statistics> for protobuf::Statistics { impl From<&ColumnStatistics> for protobuf::ColumnStats { fn from(s: &ColumnStatistics) -> protobuf::ColumnStats { protobuf::ColumnStats { - min_value: Some(protobuf::Sharpness::from(&s.min_value)), - max_value: Some(protobuf::Sharpness::from(&s.max_value)), - null_count: Some(protobuf::Sharpness::from(&s.null_count)), - distinct_count: Some(protobuf::Sharpness::from(&s.distinct_count)), + min_value: Some(protobuf::Precision::from(&s.min_value)), + max_value: Some(protobuf::Precision::from(&s.max_value)), + null_count: Some(protobuf::Precision::from(&s.null_count)), + distinct_count: Some(protobuf::Precision::from(&s.distinct_count)), } } } diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 092525583921..01a0916d8cd2 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -53,7 +53,7 @@ use datafusion::physical_plan::{ }; use datafusion::prelude::SessionContext; use datafusion::scalar::ScalarValue; -use datafusion_common::stats::Sharpness; +use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_expr::{ Accumulator, AccumulatorFactoryFunction, AggregateUDF, ReturnTypeFunction, Signature, @@ -473,8 +473,8 @@ fn roundtrip_parquet_exec_with_pruning_predicate() -> Result<()> { 1024, )]], statistics: Statistics { - num_rows: Sharpness::Inexact(100), - total_byte_size: Sharpness::Inexact(1024), + num_rows: Precision::Inexact(100), + total_byte_size: Precision::Inexact(1024), column_statistics: Statistics::unknown_column(&Arc::new(Schema::new(vec![ Field::new("col", DataType::Utf8, false), ]))), From 10b9100dcde7d68e95dcd3dfe1b7db65f2bd32fe Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 13 Oct 2023 15:47:34 +0200 Subject: [PATCH 41/53] Harmonize imports --- datafusion/common/src/stats.rs | 5 +++-- .../physical-expr/src/intervals/interval_aritmetic.rs | 9 +++++---- datafusion/physical-expr/src/intervals/utils.rs | 5 +++-- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 757ca613848d..43be25e173d2 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -17,10 +17,11 @@ //! This module provides data structures to represent statistics +use std::fmt::{self, Debug, Display}; + use crate::ScalarValue; -use arrow_schema::Schema; -use std::fmt::{self, Debug, Display}; +use arrow_schema::Schema; /// Represents a value with a degree of certainty. `Precision` is used to /// propagate information the precision of statistical values. diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index 9612c47cf7a0..344ac2b4584a 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -17,18 +17,19 @@ //! Interval arithmetic library +use std::borrow::Borrow; +use std::fmt::{self, Display, Formatter}; +use std::ops::{AddAssign, SubAssign}; + use crate::aggregate::min_max::{max, min}; use crate::intervals::rounding::{alter_fp_rounding_mode, next_down, next_up}; + use arrow::compute::{cast_with_options, CastOptions}; use arrow::datatypes::DataType; use arrow_array::ArrowNativeTypeOp; use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::type_coercion::binary::get_result_type; use datafusion_expr::Operator; -use std::borrow::Borrow; -use std::fmt; -use std::fmt::{Display, Formatter}; -use std::ops::{AddAssign, SubAssign}; /// This type represents a single endpoint of an [`Interval`]. An /// endpoint can be open (does not include the endpoint) or closed diff --git a/datafusion/physical-expr/src/intervals/utils.rs b/datafusion/physical-expr/src/intervals/utils.rs index 2ddbca40733c..7a4ccff950e6 100644 --- a/datafusion/physical-expr/src/intervals/utils.rs +++ b/datafusion/physical-expr/src/intervals/utils.rs @@ -17,17 +17,18 @@ //! Utility functions for the interval arithmetic library +use std::sync::Arc; + use super::{Interval, IntervalBound}; use crate::{ expressions::{BinaryExpr, CastExpr, Column, Literal, NegativeExpr}, PhysicalExpr, }; + use arrow_schema::{DataType, SchemaRef}; use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::Operator; -use std::sync::Arc; - const MDN_DAY_MASK: i128 = 0xFFFF_FFFF_0000_0000_0000_0000; const MDN_NS_MASK: i128 = 0xFFFF_FFFF_FFFF_FFFF; const DT_MS_MASK: i64 = 0xFFFF_FFFF; From 19dbd4666614126fcadfea3dd226e1b3d89c525f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Sat, 14 Oct 2023 22:45:10 +0300 Subject: [PATCH 42/53] Update datafusion/physical-expr/src/intervals/interval_aritmetic.rs Co-authored-by: Andrew Lamb --- datafusion/physical-expr/src/intervals/interval_aritmetic.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index 344ac2b4584a..8ecf9ad43802 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -708,8 +708,8 @@ fn next_value(value: ScalarValue) -> ScalarValue { } } -/// This function computes the cardinality ratio of the given intervals. -/// If it cannot be calculated, it returns 1.0 meaning full selective. +/// This function computes the selectivity by computing the cardinality ratio of the given intervals. +/// If this can not be calculated for some reasons, returns `1.0` meaning full selective / no filtering. pub fn cardinality_ratio( initial_interval: &Interval, final_interval: &Interval, From e62e54535b52e7c52f2e2f09127811729a0da975 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Sat, 14 Oct 2023 23:44:51 +0300 Subject: [PATCH 43/53] Addresses the reviews --- datafusion/common/src/stats.rs | 9 +- datafusion/core/src/datasource/statistics.rs | 2 +- datafusion/physical-expr/src/analysis.rs | 86 +++++++++++-------- .../src/intervals/interval_aritmetic.rs | 8 +- 4 files changed, 63 insertions(+), 42 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 43be25e173d2..fbf639a32182 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -206,11 +206,12 @@ impl Display for Precision< /// and the transformations output are not always predictable. #[derive(Debug, Clone, PartialEq, Eq)] pub struct Statistics { - /// The number of table rows + /// The number of table rows. pub num_rows: Precision, - /// total bytes of the table rows + /// Total bytes of the table rows. pub total_byte_size: Precision, - /// Statistics on a column level + /// Statistics on a column level. It contains a [`ColumnStatistics`] for + /// each field in the schema of the the table to which the [`Statistics`] refer. pub column_statistics: Vec, } @@ -236,7 +237,7 @@ impl Statistics { /// If the exactness of a [`Statistics`] instance is lost, this function relaxes /// the exactness of all information by converting them [`Precision::Inexact`]. - pub fn make_inexact(self) -> Self { + pub fn into_inexact(self) -> Self { Statistics { num_rows: self.num_rows.to_inexact(), total_byte_size: self.total_byte_size.to_inexact(), diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index df487beff58c..71597f711d85 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -171,7 +171,7 @@ pub async fn get_statistics_with_limit( // If we still have files in the stream, it means that the limit kicked // in, and the statistic could have been different had we processed the // files in a different order. - statistics = statistics.make_inexact() + statistics = statistics.into_inexact() } Ok((result_files, statistics)) diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index 11442f2fd969..2f71169cc0c4 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -21,7 +21,6 @@ use std::fmt::Debug; use std::sync::Arc; use arrow::datatypes::Schema; -use arrow_schema::Field; use datafusion_common::stats::Precision; use datafusion_common::{ColumnStatistics, DataFusionError, Result, ScalarValue}; @@ -63,35 +62,12 @@ impl AnalysisContext { input_schema: &Schema, statistics: &[ColumnStatistics], ) -> Self { - let mut column_boundaries = vec![]; - for (idx, stats) in statistics.iter().enumerate() { - let field: &Arc = &input_schema.fields()[idx]; - if let Ok(inf_field) = ScalarValue::try_from(field.data_type()) { - let interval = Interval::new( - IntervalBound::new_closed( - stats - .min_value - .get_value() - .cloned() - .unwrap_or(inf_field.clone()), - ), - IntervalBound::new_closed( - stats.max_value.get_value().cloned().unwrap_or(inf_field), - ), - ); - let column = Column::new(field.name(), idx); - column_boundaries.push(ExprBoundaries { - column, - interval, - distinct_count: stats.distinct_count.clone(), - }); - } else { - return AnalysisContext { - boundaries: None, - selectivity: None, - }; - } - } + let column_boundaries = statistics + .iter() + .enumerate() + .map(|(idx, stats)| ExprBoundaries::from_column(input_schema, stats, idx)) + .collect::>(); + Self::new(column_boundaries) } } @@ -107,6 +83,45 @@ pub struct ExprBoundaries { pub distinct_count: Precision, } +impl ExprBoundaries { + /// Create a new `ExprBoundaries` object from column level statistics. + pub fn from_column( + schema: &Schema, + col_stats: &ColumnStatistics, + col_index: usize, + ) -> Self { + let field = &schema.fields()[col_index]; + let inf_field = ScalarValue::try_from(field.data_type()).unwrap_or_else(|_| { + panic!( + "There is no equivalent for the DataType {} among ScalarValues.", + field.data_type() + ); + }); + let interval = Interval::new( + IntervalBound::new_closed( + col_stats + .min_value + .get_value() + .cloned() + .unwrap_or(inf_field.clone()), + ), + IntervalBound::new_closed( + col_stats + .max_value + .get_value() + .cloned() + .unwrap_or(inf_field), + ), + ); + let column = Column::new(field.name(), col_index); + ExprBoundaries { + column, + interval, + distinct_count: col_stats.distinct_count.clone(), + } + } +} + /// Attempts to refine column boundaries and compute a selectivity value. /// /// The function accepts boundaries of the input columns in the `context` parameter. @@ -192,10 +207,13 @@ fn shrink_boundaries( } }); let graph_nodes = graph.gather_node_indices(&[expr.clone()]); - // Since propagation result was successful, the graph has at least one element. - // An empty check is also done at the outer scope, do not repeat it here. - let (_, root_index) = graph_nodes[0]; - let final_result = graph.get_interval(root_index); + let Some((_, root_index)) = graph_nodes.get(0) else { + return Err(DataFusionError::Internal( + "The ExprIntervalGraph under investigation does not have any nodes." + .to_owned(), + )); + }; + let final_result = graph.get_interval(*root_index); let selectivity = calculate_selectivity( &final_result.lower.value, diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index 344ac2b4584a..d67c1043a025 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -1760,9 +1760,11 @@ mod tests { assert_eq!(interval.cardinality()?.unwrap(), distinct_f32); } - // If the floating numbers has showned a homogeneous distribution pattern, the result would to be - // (distinct_f64 * 2_048 = 9223372036854775808); however, due to subnormal numbers around 0, - // the result will be a specific value, close to the expected one. + // The regular logarithmic distribution of floating-point numbers are only applicable + // outside of the `(-phi, phi)` interval, where `phi` denotes the largest positive + // subnormal floating-point number. Since the following intervals include these subnormal + // points, we cannot use the constant number that remains the same in powers of 2. Therefore, + // we manually supply the actual expected cardinality. let interval = Interval::new( IntervalBound::new(ScalarValue::from(-0.0625), false), IntervalBound::new(ScalarValue::from(0.0625), true), From ab8c0c7c0d6ae5a615fb307f8014c75e03d920a0 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Sun, 15 Oct 2023 17:04:54 +0300 Subject: [PATCH 44/53] Update tests --- datafusion/sqllogictest/test_files/window.slt | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 0111ed7fa4ba..7226182cf3d0 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2010,13 +2010,10 @@ Projection: ARRAY_AGG(aggregate_test_100.c13) AS array_agg1 --------TableScan: aggregate_test_100 projection=[c13] physical_plan ProjectionExec: expr=[ARRAY_AGG(aggregate_test_100.c13)@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------GlobalLimitExec: skip=0, fetch=1 -------------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST] ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true +--AggregateExec: mode=Single, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] +----GlobalLimitExec: skip=0, fetch=1 +------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST] +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true query ? SELECT ARRAY_AGG(c13) as array_agg1 FROM (SELECT * FROM aggregate_test_100 ORDER BY c13 LIMIT 1) From 34719787d53f9aa394f301840d23bf7eae71f5fc Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 16 Oct 2023 13:42:20 +0300 Subject: [PATCH 45/53] Remove panics --- datafusion/physical-expr/src/analysis.rs | 48 ++++++++----------- .../src/intervals/interval_aritmetic.rs | 15 +++--- datafusion/physical-plan/src/filter.rs | 4 +- 3 files changed, 32 insertions(+), 35 deletions(-) diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index 2f71169cc0c4..eb3d051ac60a 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -20,17 +20,18 @@ use std::fmt::Debug; use std::sync::Arc; -use arrow::datatypes::Schema; - -use datafusion_common::stats::Precision; -use datafusion_common::{ColumnStatistics, DataFusionError, Result, ScalarValue}; - use crate::expressions::Column; use crate::intervals::cp_solver::PropagationResult; use crate::intervals::{cardinality_ratio, ExprIntervalGraph, Interval, IntervalBound}; use crate::utils::collect_columns; use crate::PhysicalExpr; +use arrow::datatypes::Schema; +use datafusion_common::stats::Precision; +use datafusion_common::{ + internal_err, ColumnStatistics, DataFusionError, Result, ScalarValue, +}; + /// The shared context used during the analysis of an expression. Includes /// the boundaries for all known columns. #[derive(Clone, Debug, PartialEq)] @@ -58,17 +59,16 @@ impl AnalysisContext { } /// Create a new analysis context from column statistics. - pub fn from_statistics( + pub fn try_from_statistics( input_schema: &Schema, statistics: &[ColumnStatistics], - ) -> Self { - let column_boundaries = statistics + ) -> Result { + statistics .iter() .enumerate() - .map(|(idx, stats)| ExprBoundaries::from_column(input_schema, stats, idx)) - .collect::>(); - - Self::new(column_boundaries) + .map(|(idx, stats)| ExprBoundaries::try_from_column(input_schema, stats, idx)) + .collect::>>() + .map(Self::new) } } @@ -85,40 +85,35 @@ pub struct ExprBoundaries { impl ExprBoundaries { /// Create a new `ExprBoundaries` object from column level statistics. - pub fn from_column( + pub fn try_from_column( schema: &Schema, col_stats: &ColumnStatistics, col_index: usize, - ) -> Self { + ) -> Result { let field = &schema.fields()[col_index]; - let inf_field = ScalarValue::try_from(field.data_type()).unwrap_or_else(|_| { - panic!( - "There is no equivalent for the DataType {} among ScalarValues.", - field.data_type() - ); - }); + let empty_field = ScalarValue::try_from(field.data_type())?; let interval = Interval::new( IntervalBound::new_closed( col_stats .min_value .get_value() .cloned() - .unwrap_or(inf_field.clone()), + .unwrap_or(empty_field.clone()), ), IntervalBound::new_closed( col_stats .max_value .get_value() .cloned() - .unwrap_or(inf_field), + .unwrap_or(empty_field), ), ); let column = Column::new(field.name(), col_index); - ExprBoundaries { + Ok(ExprBoundaries { column, interval, distinct_count: col_stats.distinct_count.clone(), - } + }) } } @@ -208,10 +203,9 @@ fn shrink_boundaries( }); let graph_nodes = graph.gather_node_indices(&[expr.clone()]); let Some((_, root_index)) = graph_nodes.get(0) else { - return Err(DataFusionError::Internal( + return internal_err!( "The ExprIntervalGraph under investigation does not have any nodes." - .to_owned(), - )); + ); }; let final_result = graph.get_interval(*root_index); diff --git a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs index b7c3b99623c5..1ea9b2d9aee6 100644 --- a/datafusion/physical-expr/src/intervals/interval_aritmetic.rs +++ b/datafusion/physical-expr/src/intervals/interval_aritmetic.rs @@ -708,8 +708,10 @@ fn next_value(value: ScalarValue) -> ScalarValue { } } -/// This function computes the selectivity by computing the cardinality ratio of the given intervals. -/// If this can not be calculated for some reasons, returns `1.0` meaning full selective / no filtering. +/// This function computes the selectivity of an operation by computing the +/// cardinality ratio of the given input/output intervals. If this can not be +/// calculated for some reason, it returns `1.0` meaning fullly selective (no +/// filtering). pub fn cardinality_ratio( initial_interval: &Interval, final_interval: &Interval, @@ -1760,10 +1762,11 @@ mod tests { assert_eq!(interval.cardinality()?.unwrap(), distinct_f32); } - // The regular logarithmic distribution of floating-point numbers are only applicable - // outside of the `(-phi, phi)` interval, where `phi` denotes the largest positive - // subnormal floating-point number. Since the following intervals include these subnormal - // points, we cannot use the constant number that remains the same in powers of 2. Therefore, + // The regular logarithmic distribution of floating-point numbers are + // only applicable outside of the `(-phi, phi)` interval where `phi` + // denotes the largest positive subnormal floating-point number. Since + // the following intervals include such subnormal points, we cannot use + // a simple powers-of-two type formula for our expectations. Therefore, // we manually supply the actual expected cardinality. let interval = Interval::new( IntervalBound::new(ScalarValue::from(-0.0625), false), diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 12be05641bb3..cff2a81ba8db 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -211,10 +211,10 @@ impl ExecutionPlan for FilterExec { let num_rows = input_stats.num_rows; let total_byte_size = input_stats.total_byte_size; - let input_analysis_ctx = AnalysisContext::from_statistics( + let input_analysis_ctx = AnalysisContext::try_from_statistics( &self.input.schema(), &input_stats.column_statistics, - ); + )?; let analysis_ctx = analyze(predicate, input_analysis_ctx)?; let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); From 23c564b093f8a60b95bbf9f774a1afe9b2908bc7 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 16 Oct 2023 15:29:48 +0300 Subject: [PATCH 46/53] 1 bug-fix, 2 code simplifications 1) get_statistics_with_limit() is simplified 2) AnalysisContext { boundaries } does not need to be an Option, it is removed. 3) Aggregation statistics() has a special handling for 0 and 1 row cases. --- datafusion/core/src/datasource/statistics.rs | 222 +++++++++--------- datafusion/physical-expr/src/analysis.rs | 8 +- .../physical-plan/src/aggregates/mod.rs | 33 ++- datafusion/physical-plan/src/filter.rs | 26 +- 4 files changed, 145 insertions(+), 144 deletions(-) diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 71597f711d85..9c0a066c8561 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -36,135 +36,87 @@ pub async fn get_statistics_with_limit( limit: Option, ) -> Result<(Vec, Statistics)> { let mut result_files = vec![]; - let mut null_counts: Option>> = None; - let mut max_values: Option>> = None; - let mut min_values: Option>> = None; - - // The number of rows and the total byte size can be calculated as long as - // at least one file has them. If none of the files provide them, then they - // will be omitted from the statistics. The missing values will be counted - // as zero. - let mut num_rows: Option> = None; - let mut total_byte_size: Option> = None; + // These statistics can be calculated as long as at least one file has them. + // If none of the files provide them, then they will become an absent precision. + // The missing values will be counted as + // - zero for summations, + // - neutral element for extreme points. + let mut null_counts: Vec> = + vec![Precision::Absent; file_schema.fields().len()]; + let mut max_values: Vec> = + vec![Precision::Absent; file_schema.fields().len()]; + let mut min_values: Vec> = + vec![Precision::Absent; file_schema.fields().len()]; + let mut num_rows: Precision = Precision::Absent; + let mut total_byte_size: Precision = Precision::Absent; // Fusing the stream allows us to call next safely even once it is finished. let mut all_files = Box::pin(all_files.fuse()); - while let Some(current) = all_files.next().await { - let (file, file_stats) = current?; - result_files.push(file); - - // Number of rows, total byte size and null counts are added for each file. - // In case of an absent information or inexact value coming from the file, - // it changes the statistic precision to inexact. - num_rows = Some(if let Some(some_num_rows) = num_rows { - match (file_stats.num_rows, &some_num_rows) { - (Precision::Absent, _) => some_num_rows.to_inexact(), - (lhs, Precision::Absent) => lhs.to_inexact(), - (lhs, rhs) => lhs.add(rhs), - } - } else { - file_stats.num_rows - }); - - total_byte_size = Some(if let Some(some_total_byte_size) = total_byte_size { - match (file_stats.total_byte_size, &some_total_byte_size) { - (Precision::Absent, _) => some_total_byte_size.to_inexact(), - (lhs, Precision::Absent) => lhs.to_inexact(), - (lhs, rhs) => lhs.add(rhs), - } - } else { - file_stats.total_byte_size - }); - - if let Some(some_null_counts) = &mut null_counts { - for (target, cs) in some_null_counts - .iter_mut() - .zip(file_stats.column_statistics.iter()) - { - *target = if cs.null_count == Precision::Absent { - // Downcast to inexact: - target.clone().to_inexact() - } else { - target.add(&cs.null_count) - }; - } - } else { - // If it is the first file, we set it directly from the file statistics. - let mut new_col_stats_nulls = file_stats - .column_statistics - .iter() - .map(|cs| cs.null_count.clone()) - .collect::>(); - // File schema may have additional fields other than each file (such - // as partitions, guaranteed to be at the end). Hence, rest of the - // fields are initialized with `Absent`. - for _ in 0..file_schema.fields().len() - file_stats.column_statistics.len() { - new_col_stats_nulls.push(Precision::Absent) - } - null_counts = Some(new_col_stats_nulls); - }; - if let Some(some_max_values) = &mut max_values { - for (i, cs) in file_stats.column_statistics.iter().enumerate() { - set_max_if_greater(some_max_values, cs.max_value.clone(), i); - } - } else { - // If it is the first file, we set it directly from the file statistics. - let mut new_col_stats_max = file_stats - .column_statistics - .iter() - .map(|cs| cs.max_value.clone()) - .collect::>(); - // file schema may have additional fields other than each file (such as partition, guaranteed to be at the end) - // Hence, push rest of the fields with information Absent. - for _ in 0..file_schema.fields().len() - file_stats.column_statistics.len() { - new_col_stats_max.push(Precision::Absent) - } - max_values = Some(new_col_stats_max); - }; + if let Some(first_file) = all_files.next().await { + let (file, file_stats) = first_file?; + result_files.push(file); - if let Some(some_min_values) = &mut min_values { - for (i, cs) in file_stats.column_statistics.iter().enumerate() { - set_min_if_lesser(some_min_values, cs.min_value.clone(), i); - } - } else { - // If it is the first file, we set it directly from the file statistics. - let mut new_col_stats_min = file_stats - .column_statistics - .iter() - .map(|cs| cs.min_value.clone()) - .collect::>(); - // file schema may have additional fields other than each file (such as partition, guaranteed to be at the end) - // Hence, push rest of the fields with information Absent. - for _ in 0..file_schema.fields().len() - file_stats.column_statistics.len() { - new_col_stats_min.push(Precision::Absent) - } - min_values = Some(new_col_stats_min); - }; + // First file, we set them directly from the file statistics. + set_from_file_statistics( + &mut num_rows, + &mut total_byte_size, + &mut null_counts, + &mut max_values, + &mut min_values, + file_stats, + ); // If the number of rows exceeds the limit, we can stop processing // files. This only applies when we know the number of rows. It also // currently ignores tables that have no statistics regarding the // number of rows. - if num_rows - .as_ref() - .unwrap_or(&Precision::Absent) - .get_value() - .unwrap_or(&usize::MIN) - > &limit.unwrap_or(usize::MAX) - { - break; - } - } + if num_rows.get_value().unwrap_or(&usize::MIN) <= &limit.unwrap_or(usize::MAX) { + while let Some(current) = all_files.next().await { + let (file, file_stats) = current?; + result_files.push(file); + + // Number of rows, total byte size and null counts are added for each file. + // In case of an absent information or inexact value coming from the file, + // it changes the statistic precision to inexact. + num_rows = add_row_stats(file_stats.num_rows, num_rows); - let size = file_schema.fields().len(); - let null_counts = null_counts.unwrap_or(vec![Precision::Absent; size]); - let max_values = max_values.unwrap_or(vec![Precision::Absent; size]); - let min_values = min_values.unwrap_or(vec![Precision::Absent; size]); + total_byte_size = + add_row_stats(file_stats.total_byte_size, total_byte_size); + + for (cs, target) in file_stats + .column_statistics + .iter() + .map(|cs| cs.null_count.clone()) + .zip(null_counts.iter_mut()) + { + *target = add_row_stats(cs, target.clone()); + } + + for (i, cs) in file_stats.column_statistics.iter().enumerate() { + set_max_if_greater(&mut max_values, cs.max_value.clone(), i); + } + + for (i, cs) in file_stats.column_statistics.iter().enumerate() { + set_min_if_lesser(&mut min_values, cs.min_value.clone(), i); + } + + // If the number of rows exceeds the limit, we can stop processing + // files. This only applies when we know the number of rows. It also + // currently ignores tables that have no statistics regarding the + // number of rows. + if num_rows.get_value().unwrap_or(&usize::MIN) + > &limit.unwrap_or(usize::MAX) + { + break; + } + } + } + }; let mut statistics = Statistics { - num_rows: num_rows.unwrap_or(Precision::Absent), - total_byte_size: total_byte_size.unwrap_or(Precision::Absent), + num_rows, + total_byte_size, column_statistics: get_col_stats_vec(null_counts, max_values, min_values), }; if all_files.next().await.is_some() { @@ -193,6 +145,44 @@ pub(crate) fn create_max_min_accs( (max_values, min_values) } +fn set_from_file_statistics( + num_rows: &mut Precision, + total_byte_size: &mut Precision, + null_counts: &mut Vec>, + max_values: &mut Vec>, + min_values: &mut Vec>, + file_stats: Statistics, +) { + *num_rows = file_stats.num_rows; + *total_byte_size = file_stats.total_byte_size; + *null_counts = file_stats + .column_statistics + .iter() + .map(|cs| cs.null_count.clone()) + .collect::>(); + *max_values = file_stats + .column_statistics + .iter() + .map(|cs| cs.max_value.clone()) + .collect::>(); + *min_values = file_stats + .column_statistics + .iter() + .map(|cs| cs.min_value.clone()) + .collect::>(); +} + +fn add_row_stats( + file_num_rows: Precision, + num_rows: Precision, +) -> Precision { + match (file_num_rows, &num_rows) { + (Precision::Absent, _) => num_rows.to_inexact(), + (lhs, Precision::Absent) => lhs.to_inexact(), + (lhs, rhs) => lhs.add(rhs), + } +} + pub(crate) fn get_col_stats_vec( null_counts: Vec>, max_values: Vec>, diff --git a/datafusion/physical-expr/src/analysis.rs b/datafusion/physical-expr/src/analysis.rs index eb3d051ac60a..93c24014fd3e 100644 --- a/datafusion/physical-expr/src/analysis.rs +++ b/datafusion/physical-expr/src/analysis.rs @@ -38,7 +38,7 @@ use datafusion_common::{ pub struct AnalysisContext { // A list of known column boundaries, ordered by the index // of the column in the current schema. - pub boundaries: Option>, + pub boundaries: Vec, /// The estimated percentage of rows that this expression would select, if /// it were to be used as a boolean predicate on a filter. The value will be /// between 0.0 (selects nothing) and 1.0 (selects everything). @@ -48,7 +48,7 @@ pub struct AnalysisContext { impl AnalysisContext { pub fn new(boundaries: Vec) -> Self { Self { - boundaries: Some(boundaries), + boundaries, selectivity: None, } } @@ -136,9 +136,7 @@ pub fn analyze( expr: &Arc, context: AnalysisContext, ) -> Result { - let target_boundaries = context.boundaries.ok_or_else(|| { - DataFusionError::Internal("No column exists at the input to filter".to_string()) - })?; + let target_boundaries = context.boundaries; let mut graph = ExprIntervalGraph::try_new(expr.clone())?; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 04e2a101a37d..7191d51fb7f0 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1022,12 +1022,33 @@ impl ExecutionPlan for AggregateExec { total_byte_size: Precision::Absent, }) } - _ => Ok(Statistics { - // the output row count is surely not larger than its input row count - num_rows: self.input.statistics()?.num_rows, - column_statistics, - total_byte_size: Precision::Absent, - }), + _ => { + // When the input row count is 0 or 1, we can adopt that statistic keeping its reliability. + // When it is larger than 1, we degrade the precision since it may decrease after aggregation. + let num_rows = if let Some(value) = + self.input().statistics()?.num_rows.get_value() + { + if *value > 1 { + self.input().statistics()?.num_rows.to_inexact() + } else if *value == 0 { + // Aggregation on an empty table creates a null row. + self.input() + .statistics()? + .num_rows + .add(&Precision::Exact(1)) + } else { + // num_rows = 1 case + self.input().statistics()?.num_rows + } + } else { + Precision::Absent + }; + Ok(Statistics { + num_rows, + column_statistics, + total_byte_size: Precision::Absent, + }) + } } } } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index cff2a81ba8db..6bdd96f2d01c 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -227,23 +227,15 @@ impl ExecutionPlan for FilterExec { None => Precision::Absent, }; - if let Some(analysis_boundaries) = analysis_ctx.boundaries { - let column_statistics = collect_new_statistics( - &input_stats.column_statistics, - analysis_boundaries, - ); - Ok(Statistics { - num_rows, - total_byte_size, - column_statistics, - }) - } else { - Ok(Statistics { - num_rows, - total_byte_size, - column_statistics: input_stats.column_statistics, - }) - } + let column_statistics = collect_new_statistics( + &input_stats.column_statistics, + analysis_ctx.boundaries, + ); + Ok(Statistics { + num_rows, + total_byte_size, + column_statistics, + }) } } From e104a212fab28de19364d12f74b0523950079b83 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 16 Oct 2023 18:08:12 +0300 Subject: [PATCH 47/53] conflict resolved --- datafusion/physical-plan/src/joins/utils.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 6d8550c45296..c523a8fe8135 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -43,7 +43,7 @@ use datafusion_common::stats::Precision; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ exec_err, plan_datafusion_err, plan_err, DataFusionError, JoinType, Result, - ScalarValue, SharedResult, + SharedResult, }; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval, IntervalBound}; From 81baf4c2e25bf801ce21a3fc21be3fc1d5f48054 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 16 Oct 2023 18:28:01 +0300 Subject: [PATCH 48/53] conflict resolved --- .../physical-plan/src/sorts/sort_preserving_merge.rs | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 49d2f23f9794..ee044d9172fb 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -271,8 +271,6 @@ impl ExecutionPlan for SortPreservingMergeExec { #[cfg(test)] mod tests { - use std::iter::FromIterator; - use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::expressions::col; @@ -283,17 +281,16 @@ mod tests { use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::test::{self, assert_is_pending, make_partition}; use crate::{collect, common}; + use arrow::array::ArrayRef; use arrow::array::{Int32Array, StringArray, TimestampNanosecondArray}; - use datafusion_common::{assert_batches_eq, assert_contains}; - - use arrow::array::{ArrayRef, Int32Array, StringArray, TimestampNanosecondArray}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; - use datafusion_common::assert_batches_eq; + use datafusion_common::{assert_batches_eq, assert_contains}; use datafusion_execution::config::SessionConfig; use futures::{FutureExt, StreamExt}; + use std::iter::FromIterator; #[tokio::test] async fn test_merge_interleave() { From a5ae0e2d8ece74cd1e5af3b40fa4a69fafb85f27 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Berkay=20=C5=9Eahin?= <124376117+berkaysynnada@users.noreply.github.com> Date: Tue, 17 Oct 2023 09:48:53 +0300 Subject: [PATCH 49/53] Update datafusion/physical-plan/src/filter.rs Co-authored-by: Andrew Lamb --- datafusion/physical-plan/src/filter.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 6bdd96f2d01c..59f9928d0e21 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -217,6 +217,7 @@ impl ExecutionPlan for FilterExec { )?; let analysis_ctx = analyze(predicate, input_analysis_ctx)?; + // Estimate (inexact) selectivity of predicate let selectivity = analysis_ctx.selectivity.unwrap_or(1.0); let num_rows = match num_rows.get_value() { Some(nr) => Precision::Inexact((*nr as f64 * selectivity).ceil() as usize), From ed4f759c251172fd68a97e6654cbfb6f980351a6 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 17 Oct 2023 11:10:57 +0300 Subject: [PATCH 50/53] Simplify set_max/min helpers --- datafusion/core/src/datasource/statistics.rs | 64 ++++++++++---------- datafusion/physical-plan/src/lib.rs | 2 - 2 files changed, 31 insertions(+), 35 deletions(-) diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 9c0a066c8561..79dd040a6c38 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -36,19 +36,18 @@ pub async fn get_statistics_with_limit( limit: Option, ) -> Result<(Vec, Statistics)> { let mut result_files = vec![]; - // These statistics can be calculated as long as at least one file has them. - // If none of the files provide them, then they will become an absent precision. - // The missing values will be counted as - // - zero for summations, + // These statistics can be calculated as long as at least one file provides + // useful information. If none of the files provides any information, then + // they will end up having `Precision::Absent` values. Throughout calculations, + // missing values will be imputed as: + // - zero for summations, and // - neutral element for extreme points. - let mut null_counts: Vec> = - vec![Precision::Absent; file_schema.fields().len()]; - let mut max_values: Vec> = - vec![Precision::Absent; file_schema.fields().len()]; - let mut min_values: Vec> = - vec![Precision::Absent; file_schema.fields().len()]; - let mut num_rows: Precision = Precision::Absent; - let mut total_byte_size: Precision = Precision::Absent; + let size = file_schema.fields().len(); + let mut null_counts: Vec> = vec![Precision::Absent; size]; + let mut max_values: Vec> = vec![Precision::Absent; size]; + let mut min_values: Vec> = vec![Precision::Absent; size]; + let mut num_rows = Precision::::Absent; + let mut total_byte_size = Precision::::Absent; // Fusing the stream allows us to call next safely even once it is finished. let mut all_files = Box::pin(all_files.fuse()); @@ -76,9 +75,10 @@ pub async fn get_statistics_with_limit( let (file, file_stats) = current?; result_files.push(file); - // Number of rows, total byte size and null counts are added for each file. - // In case of an absent information or inexact value coming from the file, - // it changes the statistic precision to inexact. + // We accumulate the number of rows, total byte size and null + // counts across all the files in question. If any file does not + // provide any information or provides an inexact value, we demote + // the statistic precision to inexact. num_rows = add_row_stats(file_stats.num_rows, num_rows); total_byte_size = @@ -94,11 +94,11 @@ pub async fn get_statistics_with_limit( } for (i, cs) in file_stats.column_statistics.iter().enumerate() { - set_max_if_greater(&mut max_values, cs.max_value.clone(), i); + set_max_if_greater(&mut max_values[i], cs.max_value.clone()); } for (i, cs) in file_stats.column_statistics.iter().enumerate() { - set_min_if_lesser(&mut min_values, cs.min_value.clone(), i); + set_min_if_lesser(&mut min_values[i], cs.min_value.clone()); } // If the number of rows exceeds the limit, we can stop processing @@ -227,63 +227,61 @@ pub(crate) fn get_col_stats( /// If the given value is numerically greater than the original maximum value, /// set the new maximum value with appropriate exactness information. fn set_max_if_greater( - max_values: &mut [Precision], + max_values: &mut Precision, max_nominee: Precision, - index: usize, ) { - match (&max_values[index], &max_nominee) { + match (&max_values, &max_nominee) { (Precision::Exact(val1), Precision::Exact(val2)) => { if val1 < val2 { - max_values[index] = max_nominee; + *max_values = max_nominee; } } (Precision::Exact(val1), Precision::Inexact(val2)) | (Precision::Inexact(val1), Precision::Inexact(val2)) | (Precision::Inexact(val1), Precision::Exact(val2)) => { if val1 < val2 { - max_values[index] = max_nominee.to_inexact() + *max_values = max_nominee.to_inexact() } } (Precision::Inexact(_), Precision::Absent) | (Precision::Exact(_), Precision::Absent) => { - max_values[index] = max_values[index].clone().to_inexact() + *max_values = max_values.clone().to_inexact() } (Precision::Absent, Precision::Exact(_)) | (Precision::Absent, Precision::Inexact(_)) => { - max_values[index] = max_nominee.to_inexact() + *max_values = max_nominee.to_inexact() } - (Precision::Absent, Precision::Absent) => max_values[index] = Precision::Absent, + (Precision::Absent, Precision::Absent) => *max_values = Precision::Absent, } } /// If the given value is numerically lesser than the original minimum value, /// set the new minimum value with appropriate exactness information. fn set_min_if_lesser( - min_values: &mut [Precision], + min_values: &mut Precision, min_nominee: Precision, - index: usize, ) { - match (&min_values[index], &min_nominee) { + match (&min_values, &min_nominee) { (Precision::Exact(val1), Precision::Exact(val2)) => { if val1 > val2 { - min_values[index] = min_nominee; + *min_values = min_nominee; } } (Precision::Exact(val1), Precision::Inexact(val2)) | (Precision::Inexact(val1), Precision::Inexact(val2)) | (Precision::Inexact(val1), Precision::Exact(val2)) => { if val1 > val2 { - min_values[index] = min_nominee.to_inexact() + *min_values = min_nominee.to_inexact() } } (Precision::Inexact(_), Precision::Absent) | (Precision::Exact(_), Precision::Absent) => { - min_values[index] = min_values[index].clone().to_inexact() + *min_values = min_values.clone().to_inexact() } (Precision::Absent, Precision::Exact(_)) | (Precision::Absent, Precision::Inexact(_)) => { - min_values[index] = min_nominee.to_inexact() + *min_values = min_nominee.to_inexact() } - (Precision::Absent, Precision::Absent) => min_values[index] = Precision::Absent, + (Precision::Absent, Precision::Absent) => *min_values = Precision::Absent, } } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 2252d8dc94c6..d7987ba95abf 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -42,8 +42,6 @@ use datafusion_physical_expr::{ use futures::stream::TryStreamExt; use tokio::task::JoinSet; -// Interfaces: - mod topk; mod visitor; From 540105116309d60d05b52a44e924dabb303e1fb4 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 17 Oct 2023 11:30:40 +0300 Subject: [PATCH 51/53] fix vector copy, remove clones --- datafusion/core/src/datasource/statistics.rs | 62 ++++---------------- 1 file changed, 13 insertions(+), 49 deletions(-) diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index 79dd040a6c38..fbd7c3466ab2 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -57,14 +57,13 @@ pub async fn get_statistics_with_limit( result_files.push(file); // First file, we set them directly from the file statistics. - set_from_file_statistics( - &mut num_rows, - &mut total_byte_size, - &mut null_counts, - &mut max_values, - &mut min_values, - file_stats, - ); + num_rows = file_stats.num_rows; + total_byte_size = file_stats.total_byte_size; + for (index, file_column) in file_stats.column_statistics.into_iter().enumerate() { + null_counts[index] = file_column.null_count; + max_values[index] = file_column.max_value; + min_values[index] = file_column.min_value; + } // If the number of rows exceeds the limit, we can stop processing // files. This only applies when we know the number of rows. It also @@ -84,21 +83,13 @@ pub async fn get_statistics_with_limit( total_byte_size = add_row_stats(file_stats.total_byte_size, total_byte_size); - for (cs, target) in file_stats - .column_statistics - .iter() - .map(|cs| cs.null_count.clone()) - .zip(null_counts.iter_mut()) + for (index, file_cs) in + file_stats.column_statistics.into_iter().enumerate() { - *target = add_row_stats(cs, target.clone()); - } - - for (i, cs) in file_stats.column_statistics.iter().enumerate() { - set_max_if_greater(&mut max_values[i], cs.max_value.clone()); - } - - for (i, cs) in file_stats.column_statistics.iter().enumerate() { - set_min_if_lesser(&mut min_values[i], cs.min_value.clone()); + null_counts[index] = + add_row_stats(file_cs.null_count, null_counts[index].clone()); + set_max_if_greater(&mut max_values[index], file_cs.max_value); + set_min_if_lesser(&mut min_values[index], file_cs.min_value); } // If the number of rows exceeds the limit, we can stop processing @@ -145,33 +136,6 @@ pub(crate) fn create_max_min_accs( (max_values, min_values) } -fn set_from_file_statistics( - num_rows: &mut Precision, - total_byte_size: &mut Precision, - null_counts: &mut Vec>, - max_values: &mut Vec>, - min_values: &mut Vec>, - file_stats: Statistics, -) { - *num_rows = file_stats.num_rows; - *total_byte_size = file_stats.total_byte_size; - *null_counts = file_stats - .column_statistics - .iter() - .map(|cs| cs.null_count.clone()) - .collect::>(); - *max_values = file_stats - .column_statistics - .iter() - .map(|cs| cs.max_value.clone()) - .collect::>(); - *min_values = file_stats - .column_statistics - .iter() - .map(|cs| cs.min_value.clone()) - .collect::>(); -} - fn add_row_stats( file_num_rows: Precision, num_rows: Precision, From 426462b61d5f111eb58890ca302b352ef78fe994 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 17 Oct 2023 11:49:06 +0300 Subject: [PATCH 52/53] resolving conflict --- .../src/physical_optimizer/enforce_distribution.rs | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 70c4ab18e0c3..89036e9f8ccc 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -48,7 +48,6 @@ use crate::physical_plan::{ }; use arrow::compute::SortOptions; -use datafusion_common::stats::Precision; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::equivalence::EquivalenceProperties; @@ -1291,11 +1290,11 @@ fn ensure_distribution( child_idx, )| { // Don't need to apply when the returned row count is not greater than 1: - let stats = child.statistics()?; - let repartition_beneficial_stats = if stats.is_exact { - stats - .num_rows - .map(|num_rows| num_rows > batch_size) + let num_rows = child.statistics()?.num_rows; + let repartition_beneficial_stats = if num_rows.is_exact().unwrap_or(false) { + num_rows + .get_value() + .map(|value| value > &batch_size) .unwrap_or(true) } else { true From 16d7882ec5edd5a41bf9f2e6d24dd782d3dc1112 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 17 Oct 2023 16:25:34 +0300 Subject: [PATCH 53/53] remove clone --- datafusion/core/src/datasource/statistics.rs | 104 ++++++++++--------- 1 file changed, 54 insertions(+), 50 deletions(-) diff --git a/datafusion/core/src/datasource/statistics.rs b/datafusion/core/src/datasource/statistics.rs index fbd7c3466ab2..3d8248dfdeb2 100644 --- a/datafusion/core/src/datasource/statistics.rs +++ b/datafusion/core/src/datasource/statistics.rs @@ -26,6 +26,7 @@ use datafusion_common::ScalarValue; use futures::{Stream, StreamExt}; use itertools::izip; +use itertools::multiunzip; /// Get all files as well as the file level summary statistics (no statistic for partition columns). /// If the optional `limit` is provided, includes only sufficient files. @@ -83,14 +84,33 @@ pub async fn get_statistics_with_limit( total_byte_size = add_row_stats(file_stats.total_byte_size, total_byte_size); - for (index, file_cs) in - file_stats.column_statistics.into_iter().enumerate() - { - null_counts[index] = - add_row_stats(file_cs.null_count, null_counts[index].clone()); - set_max_if_greater(&mut max_values[index], file_cs.max_value); - set_min_if_lesser(&mut min_values[index], file_cs.min_value); - } + (null_counts, max_values, min_values) = multiunzip( + izip!( + file_stats.column_statistics.into_iter(), + null_counts.into_iter(), + max_values.into_iter(), + min_values.into_iter() + ) + .map( + |( + ColumnStatistics { + null_count: file_nc, + max_value: file_max, + min_value: file_min, + distinct_count: _, + }, + null_count, + max_value, + min_value, + )| { + ( + add_row_stats(file_nc, null_count), + set_max_if_greater(file_max, max_value), + set_min_if_lesser(file_min, min_value), + ) + }, + ), + ); // If the number of rows exceeds the limit, we can stop processing // files. This only applies when we know the number of rows. It also @@ -189,63 +209,47 @@ pub(crate) fn get_col_stats( } /// If the given value is numerically greater than the original maximum value, -/// set the new maximum value with appropriate exactness information. +/// return the new maximum value with appropriate exactness information. fn set_max_if_greater( - max_values: &mut Precision, max_nominee: Precision, -) { + max_values: Precision, +) -> Precision { match (&max_values, &max_nominee) { - (Precision::Exact(val1), Precision::Exact(val2)) => { - if val1 < val2 { - *max_values = max_nominee; - } - } + (Precision::Exact(val1), Precision::Exact(val2)) if val1 < val2 => max_nominee, (Precision::Exact(val1), Precision::Inexact(val2)) | (Precision::Inexact(val1), Precision::Inexact(val2)) - | (Precision::Inexact(val1), Precision::Exact(val2)) => { - if val1 < val2 { - *max_values = max_nominee.to_inexact() - } - } - (Precision::Inexact(_), Precision::Absent) - | (Precision::Exact(_), Precision::Absent) => { - *max_values = max_values.clone().to_inexact() + | (Precision::Inexact(val1), Precision::Exact(val2)) + if val1 < val2 => + { + max_nominee.to_inexact() } - (Precision::Absent, Precision::Exact(_)) - | (Precision::Absent, Precision::Inexact(_)) => { - *max_values = max_nominee.to_inexact() - } - (Precision::Absent, Precision::Absent) => *max_values = Precision::Absent, + (Precision::Exact(_), Precision::Absent) => max_values.to_inexact(), + (Precision::Absent, Precision::Exact(_)) => max_nominee.to_inexact(), + (Precision::Absent, Precision::Inexact(_)) => max_nominee, + (Precision::Absent, Precision::Absent) => Precision::Absent, + _ => max_values, } } /// If the given value is numerically lesser than the original minimum value, -/// set the new minimum value with appropriate exactness information. +/// return the new minimum value with appropriate exactness information. fn set_min_if_lesser( - min_values: &mut Precision, min_nominee: Precision, -) { + min_values: Precision, +) -> Precision { match (&min_values, &min_nominee) { - (Precision::Exact(val1), Precision::Exact(val2)) => { - if val1 > val2 { - *min_values = min_nominee; - } - } + (Precision::Exact(val1), Precision::Exact(val2)) if val1 > val2 => min_nominee, (Precision::Exact(val1), Precision::Inexact(val2)) | (Precision::Inexact(val1), Precision::Inexact(val2)) - | (Precision::Inexact(val1), Precision::Exact(val2)) => { - if val1 > val2 { - *min_values = min_nominee.to_inexact() - } - } - (Precision::Inexact(_), Precision::Absent) - | (Precision::Exact(_), Precision::Absent) => { - *min_values = min_values.clone().to_inexact() - } - (Precision::Absent, Precision::Exact(_)) - | (Precision::Absent, Precision::Inexact(_)) => { - *min_values = min_nominee.to_inexact() + | (Precision::Inexact(val1), Precision::Exact(val2)) + if val1 > val2 => + { + min_nominee.to_inexact() } - (Precision::Absent, Precision::Absent) => *min_values = Precision::Absent, + (Precision::Exact(_), Precision::Absent) => min_values.to_inexact(), + (Precision::Absent, Precision::Exact(_)) => min_nominee.to_inexact(), + (Precision::Absent, Precision::Inexact(_)) => min_nominee, + (Precision::Absent, Precision::Absent) => Precision::Absent, + _ => min_values, } }