From 9cfb9cd013f33bcdae25360790da7101ee33266f Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 6 Aug 2025 16:32:21 -0400 Subject: [PATCH 01/41] remove warning from every file open (#16968) (#17059) this is too noisy and not helpful yet, we don't have a fully implemented alternative (cherry picked from commit 0183244a426e5e1fda28c7b6e6961f70f04c0744) Co-authored-by: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> --- datafusion/datasource-parquet/src/source.rs | 6 ------ 1 file changed, 6 deletions(-) diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 2251c5c20fe51..aeeece1e47f35 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -469,12 +469,6 @@ impl FileSource for ParquetSource { .file_column_projection_indices() .unwrap_or_else(|| (0..base_config.file_schema.fields().len()).collect()); - if self.schema_adapter_factory.is_some() { - log::warn!("The SchemaAdapter API will be removed from ParquetSource in a future release. \ - Use PhysicalExprAdapterFactory API instead. \ - See https://github.com/apache/datafusion/issues/16800 for discussion and https://datafusion.apache.org/library-user-guide/upgrading.html#datafusion-49-0-0 for upgrade instructions."); - } - let (expr_adapter_factory, schema_adapter_factory) = match ( base_config.expr_adapter_factory.as_ref(), self.schema_adapter_factory.as_ref(), From f6ec4c38d884d0ad5bfb66221fd4ce179af4e84b Mon Sep 17 00:00:00 2001 From: Pepijn Van Eeckhoudt Date: Thu, 7 Aug 2025 16:45:45 +0200 Subject: [PATCH 02/41] #16994 Ensure CooperativeExec#maintains_input_order returns a Vec of the correct size (#16995) (#17068) * #16994 Ensure CooperativeExec#maintains_input_order returns a Vec of the correct size * #16994 Extend default ExecutionPlan invariant checks Add checks that verify the length of the vectors returned by methods that need to return a value per child. (cherry picked from commit 2968331e4c4a8e3596afb2e56a3f0e9e4a864674) --- datafusion/physical-plan/src/coop.rs | 2 +- .../physical-plan/src/execution_plan.rs | 40 +++++++++++++++++-- datafusion/physical-plan/src/union.rs | 7 +++- datafusion/physical-plan/src/work_table.rs | 8 ---- 4 files changed, 42 insertions(+), 15 deletions(-) diff --git a/datafusion/physical-plan/src/coop.rs b/datafusion/physical-plan/src/coop.rs index be0afa07eac2c..89d5ba6f4da9b 100644 --- a/datafusion/physical-plan/src/coop.rs +++ b/datafusion/physical-plan/src/coop.rs @@ -254,7 +254,7 @@ impl ExecutionPlan for CooperativeExec { } fn maintains_input_order(&self) -> Vec { - self.input.maintains_input_order() + vec![true; self.children().len()] } fn children(&self) -> Vec<&Arc> { diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 6d51bf195dc6f..3ed85b9267e31 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -48,7 +48,7 @@ use crate::stream::RecordBatchStreamAdapter; use arrow::array::{Array, RecordBatch}; use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; -use datafusion_common::{exec_err, Constraints, Result}; +use datafusion_common::{exec_err, Constraints, DataFusionError, Result}; use datafusion_common_runtime::JoinSet; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; @@ -118,10 +118,11 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Returns an error if this individual node does not conform to its invariants. /// These invariants are typically only checked in debug mode. /// - /// A default set of invariants is provided in the default implementation. + /// A default set of invariants is provided in the [check_default_invariants] function. + /// The default implementation of `check_invariants` calls this function. /// Extension nodes can provide their own invariants. - fn check_invariants(&self, _check: InvariantLevel) -> Result<()> { - Ok(()) + fn check_invariants(&self, check: InvariantLevel) -> Result<()> { + check_default_invariants(self, check) } /// Specifies the data distribution requirements for all the @@ -1045,6 +1046,37 @@ impl PlanProperties { } } +macro_rules! check_len { + ($target:expr, $func_name:ident, $expected_len:expr) => { + let actual_len = $target.$func_name().len(); + if actual_len != $expected_len { + return internal_err!( + "{}::{} returned Vec with incorrect size: {} != {}", + $target.name(), + stringify!($func_name), + actual_len, + $expected_len + ); + } + }; +} + +/// Checks a set of invariants that apply to all ExecutionPlan implementations. +/// Returns an error if the given node does not conform. +pub fn check_default_invariants( + plan: &P, + _check: InvariantLevel, +) -> Result<(), DataFusionError> { + let children_len = plan.children().len(); + + check_len!(plan, maintains_input_order, children_len); + check_len!(plan, required_input_ordering, children_len); + check_len!(plan, required_input_distribution, children_len); + check_len!(plan, benefits_from_input_partitioning, children_len); + + Ok(()) +} + /// Indicate whether a data exchange is needed for the input of `plan`, which will be very helpful /// especially for the distributed engine to judge whether need to deal with shuffling. /// Currently, there are 3 kinds of execution plan which needs data exchange diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 73d7933e7c053..aca03c57b1b48 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -33,7 +33,8 @@ use super::{ SendableRecordBatchStream, Statistics, }; use crate::execution_plan::{ - boundedness_from_children, emission_type_from_children, InvariantLevel, + boundedness_from_children, check_default_invariants, emission_type_from_children, + InvariantLevel, }; use crate::metrics::BaselineMetrics; use crate::projection::{make_with_child, ProjectionExec}; @@ -176,7 +177,9 @@ impl ExecutionPlan for UnionExec { &self.cache } - fn check_invariants(&self, _check: InvariantLevel) -> Result<()> { + fn check_invariants(&self, check: InvariantLevel) -> Result<()> { + check_default_invariants(self, check)?; + (self.inputs().len() >= 2) .then_some(()) .ok_or(DataFusionError::Internal( diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index 076e30ab902d4..40a22f94b81f6 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -174,14 +174,6 @@ impl ExecutionPlan for WorkTableExec { &self.cache } - fn maintains_input_order(&self) -> Vec { - vec![false] - } - - fn benefits_from_input_partitioning(&self) -> Vec { - vec![false] - } - fn children(&self) -> Vec<&Arc> { vec![] } From c7fbb3fe8a6dd7f06c14a9be00b4a8c9b2d946e9 Mon Sep 17 00:00:00 2001 From: Adrian Garcia Badaracco <1755071+adriangb@users.noreply.github.com> Date: Fri, 8 Aug 2025 13:38:44 -0500 Subject: [PATCH 03/41] Add ExecutionPlan::reset_state (#17028) (#17096) * Add ExecutionPlan::reset_state * Update datafusion/sqllogictest/test_files/cte.slt * Add reference * fmt * add to upgrade guide * add explain plan, implement in more plans * fmt * only explain --------- Co-authored-by: Robert Ream --- .../src/expressions/dynamic_filters.rs | 8 ++ .../physical-plan/src/execution_plan.rs | 25 ++++++ .../physical-plan/src/joins/cross_join.rs | 12 +++ .../physical-plan/src/joins/hash_join.rs | 20 +++++ .../physical-plan/src/recursive_query.rs | 5 +- datafusion/physical-plan/src/sorts/sort.rs | 80 ++++++++++++++----- datafusion/sqllogictest/test_files/cte.slt | 55 +++++++++++++ 7 files changed, 180 insertions(+), 25 deletions(-) diff --git a/datafusion/physical-expr/src/expressions/dynamic_filters.rs b/datafusion/physical-expr/src/expressions/dynamic_filters.rs index ba30b916b9f87..ea10b1197b1d7 100644 --- a/datafusion/physical-expr/src/expressions/dynamic_filters.rs +++ b/datafusion/physical-expr/src/expressions/dynamic_filters.rs @@ -32,6 +32,10 @@ use datafusion_expr::ColumnarValue; use datafusion_physical_expr_common::physical_expr::{DynEq, DynHash}; /// A dynamic [`PhysicalExpr`] that can be updated by anyone with a reference to it. +/// +/// Any `ExecutionPlan` that uses this expression and holds a reference to it internally should probably also +/// implement `ExecutionPlan::reset_state` to remain compatible with recursive queries and other situations where +/// the same `ExecutionPlan` is reused with different data. #[derive(Debug)] pub struct DynamicFilterPhysicalExpr { /// The original children of this PhysicalExpr, if any. @@ -121,6 +125,10 @@ impl DynamicFilterPhysicalExpr { /// do not change* since those will be used to determine what columns need to read or projected /// when evaluating the expression. /// + /// Any `ExecutionPlan` that uses this expression and holds a reference to it internally should probably also + /// implement `ExecutionPlan::reset_state` to remain compatible with recursive queries and other situations where + /// the same `ExecutionPlan` is reused with different data. + /// /// [`collect_columns`]: crate::utils::collect_columns #[allow(dead_code)] // Only used in tests for now pub fn new( diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 3ed85b9267e31..3f0facb24df4f 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -196,6 +196,31 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { children: Vec>, ) -> Result>; + /// Reset any internal state within this [`ExecutionPlan`]. + /// + /// This method is called when an [`ExecutionPlan`] needs to be re-executed, + /// such as in recursive queries. Unlike [`ExecutionPlan::with_new_children`], this method + /// ensures that any stateful components (e.g., [`DynamicFilterPhysicalExpr`]) + /// are reset to their initial state. + /// + /// The default implementation simply calls [`ExecutionPlan::with_new_children`] with the existing children, + /// effectively creating a new instance of the [`ExecutionPlan`] with the same children but without + /// necessarily resetting any internal state. Implementations that require resetting of some + /// internal state should override this method to provide the necessary logic. + /// + /// This method should *not* reset state recursively for children, as it is expected that + /// it will be called from within a walk of the execution plan tree so that it will be called on each child later + /// or was already called on each child. + /// + /// Note to implementers: unlike [`ExecutionPlan::with_new_children`] this method does not accept new children as an argument, + /// thus it is expected that any cached plan properties will remain valid after the reset. + /// + /// [`DynamicFilterPhysicalExpr`]: datafusion_physical_expr::expressions::DynamicFilterPhysicalExpr + fn reset_state(self: Arc) -> Result> { + let children = self.children().into_iter().cloned().collect(); + self.with_new_children(children) + } + /// If supported, attempt to increase the partitioning of this `ExecutionPlan` to /// produce `target_partitions` partitions. /// diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index a41e668ab4dab..b8ea6330a1e2e 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -270,6 +270,18 @@ impl ExecutionPlan for CrossJoinExec { ))) } + fn reset_state(self: Arc) -> Result> { + let new_exec = CrossJoinExec { + left: Arc::clone(&self.left), + right: Arc::clone(&self.right), + schema: Arc::clone(&self.schema), + left_fut: Default::default(), // reset the build side! + metrics: ExecutionPlanMetricsSet::default(), + cache: self.cache.clone(), + }; + Ok(Arc::new(new_exec)) + } + fn required_input_distribution(&self) -> Vec { vec![ Distribution::SinglePartition, diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index a7f28ede4408f..84ca7ce19f887 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -769,6 +769,26 @@ impl ExecutionPlan for HashJoinExec { )?)) } + fn reset_state(self: Arc) -> Result> { + // Reset the left_fut to allow re-execution + Ok(Arc::new(HashJoinExec { + left: Arc::clone(&self.left), + right: Arc::clone(&self.right), + on: self.on.clone(), + filter: self.filter.clone(), + join_type: self.join_type, + join_schema: Arc::clone(&self.join_schema), + left_fut: OnceAsync::default(), + random_state: self.random_state.clone(), + mode: self.mode, + metrics: ExecutionPlanMetricsSet::new(), + projection: self.projection.clone(), + column_indices: self.column_indices.clone(), + null_equality: self.null_equality, + cache: self.cache.clone(), + })) + } + fn execute( &self, partition: usize, diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index 99b460dfcfdcd..700a9076fecf0 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -372,7 +372,7 @@ fn assign_work_table( } /// Some plans will change their internal states after execution, making them unable to be executed again. -/// This function uses `ExecutionPlan::with_new_children` to fork a new plan with initial states. +/// This function uses [`ExecutionPlan::reset_state`] to reset any internal state within the plan. /// /// An example is `CrossJoinExec`, which loads the left table into memory and stores it in the plan. /// However, if the data of the left table is derived from the work table, it will become outdated @@ -383,8 +383,7 @@ fn reset_plan_states(plan: Arc) -> Result() { Ok(Transformed::no(plan)) } else { - let new_plan = Arc::clone(&plan) - .with_new_children(plan.children().into_iter().cloned().collect())?; + let new_plan = Arc::clone(&plan).reset_state()?; Ok(Transformed::yes(new_plan)) } }) diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index bb572c4315fb8..b82f1769d092f 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -899,6 +899,29 @@ impl SortExec { self } + /// Add or reset `self.filter` to a new `DynamicFilterPhysicalExpr`. + fn create_filter(&self) -> Arc { + let children = self + .expr + .iter() + .map(|sort_expr| Arc::clone(&sort_expr.expr)) + .collect::>(); + Arc::new(DynamicFilterPhysicalExpr::new(children, lit(true))) + } + + fn cloned(&self) -> Self { + SortExec { + input: Arc::clone(&self.input), + expr: self.expr.clone(), + metrics_set: self.metrics_set.clone(), + preserve_partitioning: self.preserve_partitioning, + common_sort_prefix: self.common_sort_prefix.clone(), + fetch: self.fetch, + cache: self.cache.clone(), + filter: self.filter.clone(), + } + } + /// Modify how many rows to include in the result /// /// If None, then all rows will be returned, in sorted order. @@ -920,25 +943,13 @@ impl SortExec { } let filter = fetch.is_some().then(|| { // If we already have a filter, keep it. Otherwise, create a new one. - self.filter.clone().unwrap_or_else(|| { - let children = self - .expr - .iter() - .map(|sort_expr| Arc::clone(&sort_expr.expr)) - .collect::>(); - Arc::new(DynamicFilterPhysicalExpr::new(children, lit(true))) - }) + self.filter.clone().unwrap_or_else(|| self.create_filter()) }); - SortExec { - input: Arc::clone(&self.input), - expr: self.expr.clone(), - metrics_set: self.metrics_set.clone(), - preserve_partitioning: self.preserve_partitioning, - common_sort_prefix: self.common_sort_prefix.clone(), - fetch, - cache, - filter, - } + let mut new_sort = self.cloned(); + new_sort.fetch = fetch; + new_sort.cache = cache; + new_sort.filter = filter; + new_sort } /// Input schema @@ -1110,10 +1121,35 @@ impl ExecutionPlan for SortExec { self: Arc, children: Vec>, ) -> Result> { - let mut new_sort = SortExec::new(self.expr.clone(), Arc::clone(&children[0])) - .with_fetch(self.fetch) - .with_preserve_partitioning(self.preserve_partitioning); - new_sort.filter = self.filter.clone(); + let mut new_sort = self.cloned(); + assert!( + children.len() == 1, + "SortExec should have exactly one child" + ); + new_sort.input = Arc::clone(&children[0]); + // Recompute the properties based on the new input since they may have changed + let (cache, sort_prefix) = Self::compute_properties( + &new_sort.input, + new_sort.expr.clone(), + new_sort.preserve_partitioning, + )?; + new_sort.cache = cache; + new_sort.common_sort_prefix = sort_prefix; + + Ok(Arc::new(new_sort)) + } + + fn reset_state(self: Arc) -> Result> { + let children = self.children().into_iter().cloned().collect(); + let new_sort = self.with_new_children(children)?; + let mut new_sort = new_sort + .as_any() + .downcast_ref::() + .expect("cloned 1 lines above this line, we know the type") + .clone(); + // Our dynamic filter and execution metrics are the state we need to reset. + new_sort.filter = Some(new_sort.create_filter()); + new_sort.metrics_set = ExecutionPlanMetricsSet::new(); Ok(Arc::new(new_sort)) } diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 32320a06f4fb0..5f8fd1a0b5efd 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -996,6 +996,61 @@ physical_plan 08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)------------WorkTableExec: name=numbers +# Test for issue #16998: SortExec shares DynamicFilterPhysicalExpr across multiple executions +query II +with recursive r as ( + select 0 as k, 0 as v + union all + ( + select * + from r + order by v + limit 1 + ) +) +select * +from r +limit 5; +---- +0 0 +0 0 +0 0 +0 0 +0 0 + +query TT +explain +with recursive r as ( + select 0 as k, 0 as v + union all + ( + select * + from r + order by v + limit 1 + ) +) +select * +from r +limit 5; +---- +logical_plan +01)SubqueryAlias: r +02)--Limit: skip=0, fetch=5 +03)----RecursiveQuery: is_distinct=false +04)------Projection: Int64(0) AS k, Int64(0) AS v +05)--------EmptyRelation +06)------Sort: r.v ASC NULLS LAST, fetch=1 +07)--------Projection: r.k, r.v +08)----------TableScan: r +physical_plan +01)GlobalLimitExec: skip=0, fetch=5 +02)--RecursiveQueryExec: name=r, is_distinct=false +03)----ProjectionExec: expr=[0 as k, 0 as v] +04)------PlaceholderRowExec +05)----SortExec: TopK(fetch=1), expr=[v@1 ASC NULLS LAST], preserve_partitioning=[false] +06)------WorkTableExec: name=r + statement count 0 set datafusion.execution.enable_recursive_ctes = false; From ee28aa7673db2234b87117d664559e5857ac8c38 Mon Sep 17 00:00:00 2001 From: Adam Gutglick Date: Tue, 12 Aug 2025 15:23:33 +0100 Subject: [PATCH 04/41] [branch-49] Backport #17129 to branch 49 (#17143) * Preserve equivalence properties during projection pushdown (#17129) * Adds parquet data diffs --------- Co-authored-by: Matthew Kim <38759997+friendlymatthew@users.noreply.github.com> --- datafusion/datasource/src/source.rs | 35 +++++++++++++++++- datafusion/sqllogictest/data/1.parquet | Bin 0 -> 1381 bytes datafusion/sqllogictest/data/2.parquet | Bin 0 -> 1403 bytes .../test_files/parquet_filter_pushdown.slt | 32 ++++++++++++++++ 4 files changed, 66 insertions(+), 1 deletion(-) create mode 100644 datafusion/sqllogictest/data/1.parquet create mode 100644 datafusion/sqllogictest/data/2.parquet diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index fde1944ae066a..3a7ff1ef09911 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -22,6 +22,7 @@ use std::fmt; use std::fmt::{Debug, Formatter}; use std::sync::Arc; +use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_plan::execution_plan::{ Boundedness, EmissionType, SchedulingType, }; @@ -324,7 +325,39 @@ impl ExecutionPlan for DataSourceExec { &self, projection: &ProjectionExec, ) -> Result>> { - self.data_source.try_swapping_with_projection(projection) + match self.data_source.try_swapping_with_projection(projection)? { + Some(new_plan) => { + if let Some(new_data_source_exec) = + new_plan.as_any().downcast_ref::() + { + let projection_mapping = ProjectionMapping::try_new( + projection.expr().iter().cloned(), + &self.schema(), + )?; + + // Project the equivalence properties to the new schema + let projected_eq_properties = self + .cache + .eq_properties + .project(&projection_mapping, new_data_source_exec.schema()); + + let preserved_exec = DataSourceExec { + data_source: Arc::clone(&new_data_source_exec.data_source), + cache: PlanProperties::new( + projected_eq_properties, + new_data_source_exec.cache.partitioning.clone(), + new_data_source_exec.cache.emission_type, + new_data_source_exec.cache.boundedness, + ) + .with_scheduling_type(new_data_source_exec.cache.scheduling_type), + }; + Ok(Some(Arc::new(preserved_exec))) + } else { + Ok(Some(new_plan)) + } + } + None => Ok(None), + } } fn handle_child_pushdown_result( diff --git a/datafusion/sqllogictest/data/1.parquet b/datafusion/sqllogictest/data/1.parquet new file mode 100644 index 0000000000000000000000000000000000000000..a04f669eaeaaeb9edd468f7c28c00ba526627007 GIT binary patch literal 1381 zcmb_c&ubGw6n?v1(;NaO;*7hnhf?TLq_j!3QYDC!);7kpHj6Z=2w|IS+dz^Hn>A67 zMFhc<#~%6zcoe*N_9A!?57Lu(^p8;Rz1jS*p|w3YWOv@Y@6GqV_hvR5!cH-bW!cp{ zQyE+Wn0`O^8%i!fasu#`Qeg56A{fMHFeJ_*EMne(>51gOM@m04B9XuhVLq}{%n@gk z$At(4K>76U>#Hj#h=}$vePErFR72X9?^RDA)yS{Q_b8c>P>r+eI!6do4Gtjb2Fi_L z5r4r_hY`x@xlO*#J}b1J-vPtqM=Go1ip+hDJ(fTeFgTx$Ilk|8%k9dZ+i+L}SZoUP zXy7{)w_K}ELEglDOhf0zcHsCyIjA*Uv>L7Y>x7v`5MkQGt8T0AJ!`nlpzJm~HQ#HJ z-DBXYVH#-*Ocpa1AQHz?`Z-vPtNc*qZ&YjDivGWwW6a=n!6E@)ah%fF3*bK^%;euS zcBlZU(Ryk|i<6>WD*UZt9jVrVY7SdJv^Y!;&SvPvD>0fH_=|DI`L7G?w#?e^!6`kH z$vgZ&vGz6V;~2H%_>~*wPjl4455}>y4-qztrp8q(%D-us>Cp9=r)^>68d3ALqK=Ojf6vSPGg;?yByc@11i7Zm0 z8Cy=FEht2`PWGdk&FyW?Z|*q7u$`bARVQ$Ep0sOTbE4z=a=w0ZHaVL#()dM%KiD4w I*uX#7KgL)qfdBvi literal 0 HcmV?d00001 diff --git a/datafusion/sqllogictest/data/2.parquet b/datafusion/sqllogictest/data/2.parquet new file mode 100644 index 0000000000000000000000000000000000000000..b5e29f81baf152a741dc310da3348bd072532029 GIT binary patch literal 1403 zcmb_c&rcIU6n;Bw+8!EgjWh0=JrEK$jS?tT43fq;1)F#_p-<$9Kn(YR+=BP@u^m?8W z3YP&AA7+0-$pJ`C0KQTROnycNjo37r#At|yjN6cyNdA4Kvgm{4;6N-n1`zQX-w24==Gr0Biy9fIA-`!WOx#yi@=UmM$8ENs$LPj*P`6A z9u*h+z1F^vSuuJ%!#OYDBgR9{dwi+Jb7Bi;E?L18aLxGx0r5aE<7y2bY3Yjysmy|q zUO^=E{IRkeAfD++Ug)n{jmqlNu2YvVtzEm=FDGbv)%NV&C!M<6n&>)hIp4Z7 VlblKFY5cJIKX^F?uz`QOzW^hSGfe;h literal 0 HcmV?d00001 diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index 24e76a570c009..61f4d6fc12a3b 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -528,3 +528,35 @@ query TT select val, part from t_pushdown where part = val AND part = 'a'; ---- a a + +statement ok +COPY ( + SELECT + '00000000000000000000000000000001' AS trace_id, + '2023-10-01 00:00:00'::timestamptz AS start_timestamp, + 'prod' as deployment_environment +) +TO 'data/1.parquet'; + +statement ok +COPY ( + SELECT + '00000000000000000000000000000002' AS trace_id, + '2024-10-01 00:00:00'::timestamptz AS start_timestamp, + 'staging' as deployment_environment +) +TO 'data/2.parquet'; + +statement ok +CREATE EXTERNAL TABLE t1 STORED AS PARQUET LOCATION 'data/'; + +statement ok +SET datafusion.execution.parquet.pushdown_filters = true; + +query T +SELECT deployment_environment +FROM t1 +WHERE trace_id = '00000000000000000000000000000002' +ORDER BY start_timestamp, trace_id; +---- +staging From 52e4ef8a1d68cb131e008c8ef6eef931628b33ec Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Wed, 13 Aug 2025 12:03:49 -0700 Subject: [PATCH 05/41] Pass the input schema to stats_projection for ProjectionExpr (#17123) (#17174) * Pass the input schema to stats_projection for ProjectionExpr * Adds a test * fmt * clippy --------- Co-authored-by: Haresh Khanna --- datafusion/physical-plan/src/projection.rs | 88 +++++++++++++++++++++- 1 file changed, 86 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index a29f4aeb4090b..4c0c103a37690 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -249,7 +249,7 @@ impl ExecutionPlan for ProjectionExec { Ok(stats_projection( input_stats, self.expr.iter().map(|(e, _)| Arc::clone(e)), - Arc::clone(&self.schema), + Arc::clone(&self.input.schema()), )) } @@ -1030,8 +1030,10 @@ mod tests { use crate::common::collect; use crate::test; + use crate::test::exec::StatisticsExec; - use arrow::datatypes::DataType; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::stats::{ColumnStatistics, Precision, Statistics}; use datafusion_common::ScalarValue; use datafusion_expr::Operator; @@ -1230,4 +1232,86 @@ mod tests { assert_eq!(result, expected); } + + #[test] + fn test_projection_statistics_uses_input_schema() { + let input_schema = Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, false), + Field::new("c", DataType::Int32, false), + Field::new("d", DataType::Int32, false), + Field::new("e", DataType::Int32, false), + Field::new("f", DataType::Int32, false), + ]); + + let input_statistics = Statistics { + num_rows: Precision::Exact(10), + column_statistics: vec![ + ColumnStatistics { + min_value: Precision::Exact(ScalarValue::Int32(Some(1))), + max_value: Precision::Exact(ScalarValue::Int32(Some(100))), + ..Default::default() + }, + ColumnStatistics { + min_value: Precision::Exact(ScalarValue::Int32(Some(5))), + max_value: Precision::Exact(ScalarValue::Int32(Some(50))), + ..Default::default() + }, + ColumnStatistics { + min_value: Precision::Exact(ScalarValue::Int32(Some(10))), + max_value: Precision::Exact(ScalarValue::Int32(Some(40))), + ..Default::default() + }, + ColumnStatistics { + min_value: Precision::Exact(ScalarValue::Int32(Some(20))), + max_value: Precision::Exact(ScalarValue::Int32(Some(30))), + ..Default::default() + }, + ColumnStatistics { + min_value: Precision::Exact(ScalarValue::Int32(Some(21))), + max_value: Precision::Exact(ScalarValue::Int32(Some(29))), + ..Default::default() + }, + ColumnStatistics { + min_value: Precision::Exact(ScalarValue::Int32(Some(24))), + max_value: Precision::Exact(ScalarValue::Int32(Some(26))), + ..Default::default() + }, + ], + ..Default::default() + }; + + let input = Arc::new(StatisticsExec::new(input_statistics, input_schema)); + + // Create projection expressions that reference columns from the input schema and the length + // of output schema columns < input schema columns and hence if we use the last few columns + // from the input schema in the expressions here, bounds_check would fail on them if output + // schema is supplied to the partitions_statistics method. + let exprs: Vec<(Arc, String)> = vec![ + ( + Arc::new(Column::new("c", 2)) as Arc, + "c_renamed".to_string(), + ), + ( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 4)), + Operator::Plus, + Arc::new(Column::new("f", 5)), + )) as Arc, + "e_plus_f".to_string(), + ), + ]; + + let projection = ProjectionExec::try_new(exprs, input).unwrap(); + + let stats = projection.partition_statistics(None).unwrap(); + + assert_eq!(stats.num_rows, Precision::Exact(10)); + assert_eq!( + stats.column_statistics.len(), + 2, + "Expected 2 columns in projection statistics" + ); + assert!(stats.total_byte_size.is_exact().unwrap_or(false)); + } } From f05b1285e90d5fd16b4c832cb229996beda138be Mon Sep 17 00:00:00 2001 From: Nuno Faria Date: Thu, 14 Aug 2025 13:16:20 +0100 Subject: [PATCH 06/41] [branch-49] fix: string_agg not respecting ORDER BY (#17058) * fix: string_agg not respecting ORDER BY * Fix equality of parametrizable ArrayAgg function (#17065) The `ArrayAgg` struct is stateful, therefore it must implement `AggregateUDFImpl::equals` and `hash_value` functions. * Implement AggregateUDFImpl::equals and AggregateUDFImpl::hash_value for ArrayAgg * Implement alternative fix * Remove 'use std::any::Any' * Add sqllogictest for string_agg plan * Revert as_any to their original implementations --------- Co-authored-by: Piotr Findeisen Co-authored-by: Andrew Lamb --- .../functions-aggregate/src/string_agg.rs | 4 + .../sqllogictest/test_files/aggregate.slt | 100 ++++++++++++++++++ 2 files changed, 104 insertions(+) diff --git a/datafusion/functions-aggregate/src/string_agg.rs b/datafusion/functions-aggregate/src/string_agg.rs index 56c5ee1aaa676..5bf9020cd16ad 100644 --- a/datafusion/functions-aggregate/src/string_agg.rs +++ b/datafusion/functions-aggregate/src/string_agg.rs @@ -178,6 +178,10 @@ impl AggregateUDFImpl for StringAgg { ))) } + fn reverse_expr(&self) -> datafusion_expr::ReversedUDAF { + datafusion_expr::ReversedUDAF::Reversed(string_agg_udaf()) + } + fn documentation(&self) -> Option<&Documentation> { self.doc() } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index bdf327c98248a..753820b6b6193 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -6028,6 +6028,106 @@ GROUP BY dummy ---- text1 + +# Test string_agg with ORDER BY clasuses (issue #17011) +statement ok +create table t (k varchar, v int); + +statement ok +insert into t values ('a', 2), ('b', 3), ('c', 1), ('d', null); + +query T +select string_agg(k, ',' order by k) from t; +---- +a,b,c,d + +query T +select string_agg(k, ',' order by k desc) from t; +---- +d,c,b,a + +query T +select string_agg(k, ',' order by v) from t; +---- +c,a,b,d + +query T +select string_agg(k, ',' order by v nulls first) from t; +---- +d,c,a,b + +query T +select string_agg(k, ',' order by v desc) from t; +---- +d,b,a,c + +query T +select string_agg(k, ',' order by v desc nulls last) from t; +---- +b,a,c,d + +query T +-- odd indexes should appear first, ties solved by v +select string_agg(k, ',' order by v % 2 == 0, v) from t; +---- +c,b,a,d + +query T +-- odd indexes should appear first, ties solved by v desc +select string_agg(k, ',' order by v % 2 == 0, v desc) from t; +---- +b,c,a,d + +query T +select string_agg(k, ',' order by + case + when k = 'a' then 3 + when k = 'b' then 0 + when k = 'c' then 2 + when k = 'd' then 1 + end) +from t; +---- +b,d,c,a + +query T +select string_agg(k, ',' order by + case + when k = 'a' then 3 + when k = 'b' then 0 + when k = 'c' then 2 + when k = 'd' then 1 + end desc) +from t; +---- +a,c,d,b + +query TT +explain select string_agg(k, ',' order by v) from t; +---- +logical_plan +01)Aggregate: groupBy=[[]], aggr=[[string_agg(t.k, Utf8(",")) ORDER BY [t.v ASC NULLS LAST]]] +02)--TableScan: t projection=[k, v] +physical_plan +01)AggregateExec: mode=Single, gby=[], aggr=[string_agg(t.k,Utf8(",")) ORDER BY [t.v ASC NULLS LAST]] +02)--SortExec: expr=[v@1 ASC NULLS LAST], preserve_partitioning=[false] +03)----DataSourceExec: partitions=1, partition_sizes=[1] + +query TT +explain select string_agg(k, ',' order by v desc) from t; +---- +logical_plan +01)Aggregate: groupBy=[[]], aggr=[[string_agg(t.k, Utf8(",")) ORDER BY [t.v DESC NULLS FIRST]]] +02)--TableScan: t projection=[k, v] +physical_plan +01)AggregateExec: mode=Single, gby=[], aggr=[string_agg(t.k,Utf8(",")) ORDER BY [t.v DESC NULLS FIRST]] +02)--SortExec: expr=[v@1 DESC], preserve_partitioning=[false] +03)----DataSourceExec: partitions=1, partition_sizes=[1] + +statement ok +drop table t; + + # Tests for aggregating with NaN values statement ok CREATE TABLE float_table ( From d1a6e9a6558300115d913a54bbea27e141156606 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 14 Aug 2025 07:07:36 -0700 Subject: [PATCH 07/41] [branch-49] Update version to 49.0.1 and add changelog (#17175) * Update to version 49.0.1 * Add changelog for 49.0.1 * Fix sqllogictests * update configs * Update with PR * prettier * Fix slt race condition * Tweak release notes --- Cargo.lock | 80 +++++++++--------- Cargo.toml | 2 +- datafusion/sqllogictest/data/1.parquet | Bin 1381 -> 0 bytes datafusion/sqllogictest/data/2.parquet | Bin 1403 -> 0 bytes datafusion/sqllogictest/test_files/joins.slt | 20 ++--- .../test_files/parquet_filter_pushdown.slt | 6 +- dev/changelog/49.0.1.md | 48 +++++++++++ docs/source/user-guide/configs.md | 2 +- 8 files changed, 103 insertions(+), 55 deletions(-) delete mode 100644 datafusion/sqllogictest/data/1.parquet delete mode 100644 datafusion/sqllogictest/data/2.parquet create mode 100644 dev/changelog/49.0.1.md diff --git a/Cargo.lock b/Cargo.lock index a088005a0f197..6706ed46c02dd 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1818,7 +1818,7 @@ dependencies = [ [[package]] name = "datafusion" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "arrow-ipc", @@ -1890,7 +1890,7 @@ dependencies = [ [[package]] name = "datafusion-benchmarks" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "datafusion", @@ -1914,7 +1914,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "async-trait", @@ -1938,7 +1938,7 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "async-trait", @@ -1959,7 +1959,7 @@ dependencies = [ [[package]] name = "datafusion-cli" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "assert_cmd", @@ -1991,7 +1991,7 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "49.0.0" +version = "49.0.1" dependencies = [ "ahash 0.8.12", "apache-avro", @@ -2019,7 +2019,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "49.0.0" +version = "49.0.1" dependencies = [ "futures", "log", @@ -2028,7 +2028,7 @@ dependencies = [ [[package]] name = "datafusion-datasource" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "async-compression", @@ -2063,7 +2063,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-avro" -version = "49.0.0" +version = "49.0.1" dependencies = [ "apache-avro", "arrow", @@ -2088,7 +2088,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "async-trait", @@ -2111,7 +2111,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "async-trait", @@ -2134,7 +2134,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "async-trait", @@ -2166,11 +2166,11 @@ dependencies = [ [[package]] name = "datafusion-doc" -version = "49.0.0" +version = "49.0.1" [[package]] name = "datafusion-examples" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "arrow-flight", @@ -2201,7 +2201,7 @@ dependencies = [ [[package]] name = "datafusion-execution" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "chrono", @@ -2220,7 +2220,7 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "async-trait", @@ -2243,7 +2243,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "datafusion-common", @@ -2254,7 +2254,7 @@ dependencies = [ [[package]] name = "datafusion-ffi" -version = "49.0.0" +version = "49.0.1" dependencies = [ "abi_stable", "arrow", @@ -2275,7 +2275,7 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "arrow-buffer", @@ -2304,7 +2304,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "49.0.0" +version = "49.0.1" dependencies = [ "ahash 0.8.12", "arrow", @@ -2325,7 +2325,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "49.0.0" +version = "49.0.1" dependencies = [ "ahash 0.8.12", "arrow", @@ -2338,7 +2338,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "arrow-ord", @@ -2360,7 +2360,7 @@ dependencies = [ [[package]] name = "datafusion-functions-table" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "async-trait", @@ -2374,7 +2374,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "datafusion-common", @@ -2390,7 +2390,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "49.0.0" +version = "49.0.1" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -2398,7 +2398,7 @@ dependencies = [ [[package]] name = "datafusion-macros" -version = "49.0.0" +version = "49.0.1" dependencies = [ "datafusion-expr", "quote", @@ -2407,7 +2407,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "async-trait", @@ -2434,7 +2434,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "49.0.0" +version = "49.0.1" dependencies = [ "ahash 0.8.12", "arrow", @@ -2459,7 +2459,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "49.0.0" +version = "49.0.1" dependencies = [ "ahash 0.8.12", "arrow", @@ -2471,7 +2471,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "datafusion-common", @@ -2492,7 +2492,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "49.0.0" +version = "49.0.1" dependencies = [ "ahash 0.8.12", "arrow", @@ -2528,7 +2528,7 @@ dependencies = [ [[package]] name = "datafusion-proto" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "chrono", @@ -2550,7 +2550,7 @@ dependencies = [ [[package]] name = "datafusion-proto-common" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "datafusion-common", @@ -2563,7 +2563,7 @@ dependencies = [ [[package]] name = "datafusion-pruning" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "arrow-schema", @@ -2582,7 +2582,7 @@ dependencies = [ [[package]] name = "datafusion-session" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "async-trait", @@ -2604,7 +2604,7 @@ dependencies = [ [[package]] name = "datafusion-spark" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "criterion", @@ -2620,7 +2620,7 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "bigdecimal", @@ -2644,7 +2644,7 @@ dependencies = [ [[package]] name = "datafusion-sqllogictest" -version = "49.0.0" +version = "49.0.1" dependencies = [ "arrow", "async-trait", @@ -2677,7 +2677,7 @@ dependencies = [ [[package]] name = "datafusion-substrait" -version = "49.0.0" +version = "49.0.1" dependencies = [ "async-recursion", "async-trait", @@ -2697,7 +2697,7 @@ dependencies = [ [[package]] name = "datafusion-wasmtest" -version = "49.0.0" +version = "49.0.1" dependencies = [ "chrono", "console_error_panic_hook", diff --git a/Cargo.toml b/Cargo.toml index 11cd3c637a971..742e2b8a19f9b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -77,7 +77,7 @@ repository = "https://github.com/apache/datafusion" # Define Minimum Supported Rust Version (MSRV) rust-version = "1.85.1" # Define DataFusion version -version = "49.0.0" +version = "49.0.1" [workspace.dependencies] # We turn off default-features for some dependencies here so the workspaces which inherit them can diff --git a/datafusion/sqllogictest/data/1.parquet b/datafusion/sqllogictest/data/1.parquet deleted file mode 100644 index a04f669eaeaaeb9edd468f7c28c00ba526627007..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1381 zcmb_c&ubGw6n?v1(;NaO;*7hnhf?TLq_j!3QYDC!);7kpHj6Z=2w|IS+dz^Hn>A67 zMFhc<#~%6zcoe*N_9A!?57Lu(^p8;Rz1jS*p|w3YWOv@Y@6GqV_hvR5!cH-bW!cp{ zQyE+Wn0`O^8%i!fasu#`Qeg56A{fMHFeJ_*EMne(>51gOM@m04B9XuhVLq}{%n@gk z$At(4K>76U>#Hj#h=}$vePErFR72X9?^RDA)yS{Q_b8c>P>r+eI!6do4Gtjb2Fi_L z5r4r_hY`x@xlO*#J}b1J-vPtqM=Go1ip+hDJ(fTeFgTx$Ilk|8%k9dZ+i+L}SZoUP zXy7{)w_K}ELEglDOhf0zcHsCyIjA*Uv>L7Y>x7v`5MkQGt8T0AJ!`nlpzJm~HQ#HJ z-DBXYVH#-*Ocpa1AQHz?`Z-vPtNc*qZ&YjDivGWwW6a=n!6E@)ah%fF3*bK^%;euS zcBlZU(Ryk|i<6>WD*UZt9jVrVY7SdJv^Y!;&SvPvD>0fH_=|DI`L7G?w#?e^!6`kH z$vgZ&vGz6V;~2H%_>~*wPjl4455}>y4-qztrp8q(%D-us>Cp9=r)^>68d3ALqK=Ojf6vSPGg;?yByc@11i7Zm0 z8Cy=FEht2`PWGdk&FyW?Z|*q7u$`bARVQ$Ep0sOTbE4z=a=w0ZHaVL#()dM%KiD4w I*uX#7KgL)qfdBvi diff --git a/datafusion/sqllogictest/data/2.parquet b/datafusion/sqllogictest/data/2.parquet deleted file mode 100644 index b5e29f81baf152a741dc310da3348bd072532029..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1403 zcmb_c&rcIU6n;Bw+8!EgjWh0=JrEK$jS?tT43fq;1)F#_p-<$9Kn(YR+=BP@u^m?8W z3YP&AA7+0-$pJ`C0KQTROnycNjo37r#At|yjN6cyNdA4Kvgm{4;6N-n1`zQX-w24==Gr0Biy9fIA-`!WOx#yi@=UmM$8ENs$LPj*P`6A z9u*h+z1F^vSuuJ%!#OYDBgR9{dwi+Jb7Bi;E?L18aLxGx0r5aE<7y2bY3Yjysmy|q zUO^=E{IRkeAfD++Ug)n{jmqlNu2YvVtzEm=FDGbv)%NV&C!M<6n&>)hIp4Z7 VlblKFY5cJIKX^F?uz`QOzW^hSGfe;h diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 6c40a71fa6ef5..71251c8b1625b 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -4240,23 +4240,23 @@ set datafusion.execution.target_partitions = 1; # Note we use csv as MemoryExec does not support limit push down (so doesn't manifest # bugs if limits are improperly pushed down) query I -COPY (values (1), (2), (3), (4), (5)) TO 'test_files/scratch/limit/t1.csv' +COPY (values (1), (2), (3), (4), (5)) TO 'test_files/scratch/joins/t1.csv' STORED AS CSV ---- 5 # store t2 in different order so the top N rows are not the same as the top N rows of t1 query I -COPY (values (5), (4), (3), (2), (1)) TO 'test_files/scratch/limit/t2.csv' +COPY (values (5), (4), (3), (2), (1)) TO 'test_files/scratch/joins/t2.csv' STORED AS CSV ---- 5 statement ok -create external table t1(a int) stored as CSV location 'test_files/scratch/limit/t1.csv'; +create external table t1(a int) stored as CSV location 'test_files/scratch/joins/t1.csv'; statement ok -create external table t2(b int) stored as CSV location 'test_files/scratch/limit/t2.csv'; +create external table t2(b int) stored as CSV location 'test_files/scratch/joins/t2.csv'; ###### ## LEFT JOIN w/ LIMIT @@ -4288,8 +4288,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Left, on=[(a@0, b@0)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], limit=2, file_type=csv, has_header=true -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t1.csv]]}, projection=[a], limit=2, file_type=csv, has_header=true +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], file_type=csv, has_header=true ###### ## RIGHT JOIN w/ LIMIT @@ -4322,8 +4322,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(a@0, b@0)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], file_type=csv, has_header=true -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], limit=2, file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t1.csv]]}, projection=[a], file_type=csv, has_header=true +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], limit=2, file_type=csv, has_header=true ###### ## FULL JOIN w/ LIMIT @@ -4359,8 +4359,8 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=3, fetch=2 02)--HashJoinExec: mode=CollectLeft, join_type=Full, on=[(a@0, b@0)] -03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t1.csv]]}, projection=[a], file_type=csv, has_header=true -04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/limit/t2.csv]]}, projection=[b], file_type=csv, has_header=true +03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t1.csv]]}, projection=[a], file_type=csv, has_header=true +04)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/joins/t2.csv]]}, projection=[b], file_type=csv, has_header=true statement ok drop table t1; diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index 61f4d6fc12a3b..9f6f81789e894 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -536,7 +536,7 @@ COPY ( '2023-10-01 00:00:00'::timestamptz AS start_timestamp, 'prod' as deployment_environment ) -TO 'data/1.parquet'; +TO 'test_files/scratch/parquet_filter_pushdown/data/1.parquet'; statement ok COPY ( @@ -545,10 +545,10 @@ COPY ( '2024-10-01 00:00:00'::timestamptz AS start_timestamp, 'staging' as deployment_environment ) -TO 'data/2.parquet'; +TO 'test_files/scratch/parquet_filter_pushdown/data/2.parquet'; statement ok -CREATE EXTERNAL TABLE t1 STORED AS PARQUET LOCATION 'data/'; +CREATE EXTERNAL TABLE t1 STORED AS PARQUET LOCATION 'test_files/scratch/parquet_filter_pushdown/data/'; statement ok SET datafusion.execution.parquet.pushdown_filters = true; diff --git a/dev/changelog/49.0.1.md b/dev/changelog/49.0.1.md new file mode 100644 index 0000000000000..06d7c1e2c77a6 --- /dev/null +++ b/dev/changelog/49.0.1.md @@ -0,0 +1,48 @@ + + +# Apache DataFusion 49.0.1 Changelog + +This release consists of 5 commits from 5 contributors. See credits at the end of this changelog for more information. + +See the [upgrade guide](https://datafusion.apache.org/library-user-guide/upgrading.html) for information on how to upgrade from previous versions. + +**Other:** + +- [branch-49] Final Changelog Tweaks [#16852](https://github.com/apache/datafusion/pull/16852) (alamb) +- [branch-49] remove warning from every file open [#17059](https://github.com/apache/datafusion/pull/17059) (mbutrovich) +- [branch-49] Backport PR #16995 to branch-49 [#17068](https://github.com/apache/datafusion/pull/17068) (pepijnve) +- [branch-49] Backport "Add ExecutionPlan::reset_state (apache#17028)" to v49 [#17096](https://github.com/apache/datafusion/pull/17096) (adriangb) +- [branch-49] Backport #17129 to branch 49 [#17143](https://github.com/apache/datafusion/pull/17143) (AdamGS) +- [branch-49] Backport Pass the input schema to stats_projection for ProjectionExpr (#17123) [#17174](https://github.com/apache/datafusion/pull/17174) (alamb) +- [branch-49] fix: string_agg not respecting ORDER BY [#17058](https://github.com/apache/datafusion/pull/17058) (nuno-faria) + +## Credits + +Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. + +``` + 1 Adam Gutglick + 1 Adrian Garcia Badaracco + 1 Andrew Lamb + 1 Matt Butrovich + 1 Pepijn Van Eeckhoudt +``` + +Thank you also to everyone who contributed in other ways such as filing issues, reviewing PRs, and providing feedback on this release. diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 9ac1c59caa800..dc950de01f1b1 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -70,7 +70,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_statistics_size | 4096 | (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting max_statistics_size is deprecated, currently it is not being used | | datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | -| datafusion.execution.parquet.created_by | datafusion version 49.0.0 | (writing) Sets "created by" property | +| datafusion.execution.parquet.created_by | datafusion version 49.0.1 | (writing) Sets "created by" property | | datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | | datafusion.execution.parquet.statistics_truncate_length | NULL | (writing) Sets statictics truncate length. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | From 374fcecca38136b29c8e1c145875e9e4464efb31 Mon Sep 17 00:00:00 2001 From: Huaijin Date: Wed, 20 Aug 2025 18:55:36 +0800 Subject: [PATCH 08/41] cherry-pick inlist fix (#17254) --- datafusion/proto/src/physical_plan/mod.rs | 36 +++++++------------ .../tests/cases/roundtrip_physical_plan.rs | 33 +++++++++++++++++ 2 files changed, 45 insertions(+), 24 deletions(-) diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 52e0b20db2c2e..6e7546737d72c 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -611,29 +611,6 @@ impl protobuf::PhysicalPlanNode { ) -> Result> { let input: Arc = into_physical_plan(&filter.input, registry, runtime, extension_codec)?; - let projection = if !filter.projection.is_empty() { - Some( - filter - .projection - .iter() - .map(|i| *i as usize) - .collect::>(), - ) - } else { - None - }; - - // Use the projected schema if projection is present, otherwise use the full schema - let predicate_schema = if let Some(ref proj_indices) = projection { - // Create projected schema for parsing the predicate - let projected_fields: Vec<_> = proj_indices - .iter() - .map(|&i| input.schema().field(i).clone()) - .collect(); - Arc::new(Schema::new(projected_fields)) - } else { - input.schema() - }; let predicate = filter .expr @@ -642,7 +619,7 @@ impl protobuf::PhysicalPlanNode { parse_physical_expr( expr, registry, - predicate_schema.as_ref(), + input.schema().as_ref(), extension_codec, ) }) @@ -653,6 +630,17 @@ impl protobuf::PhysicalPlanNode { ) })?; let filter_selectivity = filter.default_filter_selectivity.try_into(); + let projection = if !filter.projection.is_empty() { + Some( + filter + .projection + .iter() + .map(|i| *i as usize) + .collect::>(), + ) + } else { + None + }; let filter = FilterExec::try_new(predicate, input)?.with_projection(projection)?; match filter_selectivity { diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 2d27a21447b22..24816c24a5afb 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -1839,3 +1839,36 @@ async fn test_round_trip_tpch_queries() -> Result<()> { Ok(()) } + +#[tokio::test] +async fn test_tpch_part_in_list_query_with_real_parquet_data() -> Result<()> { + use datafusion_common::test_util::datafusion_test_data; + + let ctx = SessionContext::new(); + + // Register the TPC-H part table using the local test data + let test_data = datafusion_test_data(); + let table_sql = format!( + "CREATE EXTERNAL TABLE part STORED AS PARQUET LOCATION '{test_data}/tpch_part_small.parquet'" +); + ctx.sql(&table_sql).await.map_err(|e| { + DataFusionError::External(format!("Failed to create part table: {e}").into()) + })?; + + // Test the exact problematic query + let sql = + "SELECT p_size FROM part WHERE p_size IN (14, 6, 5, 31) and p_partkey > 1000"; + + let logical_plan = ctx.sql(sql).await?.into_unoptimized_plan(); + let optimized_plan = ctx.state().optimize(&logical_plan)?; + let physical_plan = ctx.state().create_physical_plan(&optimized_plan).await?; + + // Serialize the physical plan - bug may happen here already but not necessarily manifests + let codec = DefaultPhysicalExtensionCodec {}; + let proto = PhysicalPlanNode::try_from_physical_plan(physical_plan.clone(), &codec)?; + + // This will fail with the bug, but should succeed when fixed + let _deserialized_plan = + proto.try_into_physical_plan(&ctx, ctx.runtime_env().as_ref(), &codec)?; + Ok(()) +} From 930608a7161f89163727ec98684c4ecaf2d4bf04 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Thu, 21 Aug 2025 16:37:08 +0800 Subject: [PATCH 09/41] fix check license header --- datafusion/physical-plan/src/node_id.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/physical-plan/src/node_id.rs b/datafusion/physical-plan/src/node_id.rs index 2a246db0a77b5..7b8d0281eb73b 100644 --- a/datafusion/physical-plan/src/node_id.rs +++ b/datafusion/physical-plan/src/node_id.rs @@ -14,6 +14,7 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. + use std::sync::Arc; use crate::ExecutionPlan; From 66ae5885b255e5409a214be2e14441d66f062942 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Thu, 21 Aug 2025 17:16:07 +0800 Subject: [PATCH 10/41] fix cargo check: cargo check --profile ci --workspace --all-targets --features integration-tests --locked --- datafusion-examples/examples/planner_api.rs | 2 +- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 2 +- .../core/tests/physical_optimizer/enforce_distribution.rs | 5 +++-- datafusion/physical-optimizer/src/enforce_distribution.rs | 2 +- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/datafusion-examples/examples/planner_api.rs b/datafusion-examples/examples/planner_api.rs index 3e718d71f1fbb..690066a78d7c2 100644 --- a/datafusion-examples/examples/planner_api.rs +++ b/datafusion-examples/examples/planner_api.rs @@ -17,7 +17,7 @@ use datafusion::error::Result; use datafusion::logical_expr::LogicalPlan; -use datafusion::physical_plan::displayable; +use datafusion::physical_plan::{displayable, ExecutionPlan}; use datafusion::physical_planner::DefaultPhysicalPlanner; use datafusion::prelude::*; diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index 72aab1acd1f2a..703b8715821a8 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -224,7 +224,7 @@ impl SortTest { /// Sort the input using SortExec and ensure the results are /// correct according to `Vec::sort` both with and without spilling async fn run(&self) -> (Vec>, Vec) { - let input = Arc::clone(self.input()); + let input = self.input.clone(); let first_batch = input .iter() .flat_map(|p| p.iter()) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 4034800c30cba..d98f6f59ca34f 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -3603,10 +3603,11 @@ fn test_replace_order_preserving_variants_with_fetch() -> Result<()> { ); // Apply the function - let result = replace_order_preserving_variants(dist_context)?; + let result = replace_order_preserving_variants(dist_context, false)?; // Verify the plan was transformed to CoalescePartitionsExec result + .0 .plan .as_any() .downcast_ref::() @@ -3614,7 +3615,7 @@ fn test_replace_order_preserving_variants_with_fetch() -> Result<()> { // Verify fetch was preserved assert_eq!( - result.plan.fetch(), + result.0.plan.fetch(), Some(5), "Fetch value was not preserved after transformation" ); diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 0650bfbcf14ec..77c7af76ccbef 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1034,7 +1034,7 @@ fn remove_dist_changing_operators( /// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", /// " DataSourceExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC], file_type=parquet", /// ``` -fn replace_order_preserving_variants( +pub fn replace_order_preserving_variants( mut context: DistributionContext, ordering_satisfied: bool, ) -> Result<(DistributionContext, Option)> { From 292641c851047443463dd20e9b4d1c62bb200329 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Thu, 21 Aug 2025 17:53:30 +0800 Subject: [PATCH 11/41] fix cargo example --- datafusion-examples/examples/planner_api.rs | 28 +-------------------- 1 file changed, 1 insertion(+), 27 deletions(-) diff --git a/datafusion-examples/examples/planner_api.rs b/datafusion-examples/examples/planner_api.rs index 690066a78d7c2..55aec7b0108a4 100644 --- a/datafusion-examples/examples/planner_api.rs +++ b/datafusion-examples/examples/planner_api.rs @@ -17,7 +17,7 @@ use datafusion::error::Result; use datafusion::logical_expr::LogicalPlan; -use datafusion::physical_plan::{displayable, ExecutionPlan}; +use datafusion::physical_plan::displayable; use datafusion::physical_planner::DefaultPhysicalPlanner; use datafusion::prelude::*; @@ -80,35 +80,9 @@ async fn to_physical_plan_in_one_api_demo( displayable(physical_plan.as_ref()).indent(false) ); - let traversal = extract_node_ids_from_execution_plan_tree(physical_plan.as_ref()); - let expected_traversal = vec![ - Some(0), - Some(1), - Some(2), - Some(3), - Some(4), - Some(5), - Some(6), - Some(7), - Some(8), - Some(9), - ]; - assert_eq!(expected_traversal, traversal); Ok(()) } -fn extract_node_ids_from_execution_plan_tree( - physical_plan: &dyn ExecutionPlan, -) -> Vec> { - let mut traversed_nodes: Vec> = vec![]; - for child in physical_plan.children() { - let node_ids = extract_node_ids_from_execution_plan_tree(child.as_ref()); - traversed_nodes.extend(node_ids); - } - traversed_nodes.push(physical_plan.properties().node_id()); - traversed_nodes -} - /// Converts a logical plan into a physical plan by utilizing the analyzer, /// optimizer, and query planner APIs separately. This flavor gives more /// control over the planning process. From a6068c27ca2bf91a3d289e4e5d42beb6600f0154 Mon Sep 17 00:00:00 2001 From: Tim Saucer Date: Thu, 21 Aug 2025 12:18:59 -0400 Subject: [PATCH 12/41] FFI_RecordBatchStream was causing a memory leak (#17190) (#17270) Co-authored-by: Andrew Lamb --- datafusion/ffi/src/record_batch_stream.rs | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/datafusion/ffi/src/record_batch_stream.rs b/datafusion/ffi/src/record_batch_stream.rs index 78d65a816fcc2..6c2282df88dd0 100644 --- a/datafusion/ffi/src/record_batch_stream.rs +++ b/datafusion/ffi/src/record_batch_stream.rs @@ -57,6 +57,9 @@ pub struct FFI_RecordBatchStream { /// Return the schema of the record batch pub schema: unsafe extern "C" fn(stream: &Self) -> WrappedSchema, + /// Release the memory of the private data when it is no longer being used. + pub release: unsafe extern "C" fn(arg: &mut Self), + /// Internal data. This is only to be accessed by the provider of the plan. /// The foreign library should never attempt to access this data. pub private_data: *mut c_void, @@ -82,6 +85,7 @@ impl FFI_RecordBatchStream { FFI_RecordBatchStream { poll_next: poll_next_fn_wrapper, schema: schema_fn_wrapper, + release: release_fn_wrapper, private_data, } } @@ -96,6 +100,12 @@ unsafe extern "C" fn schema_fn_wrapper(stream: &FFI_RecordBatchStream) -> Wrappe (*stream).schema().into() } +unsafe extern "C" fn release_fn_wrapper(provider: &mut FFI_RecordBatchStream) { + let private_data = + Box::from_raw(provider.private_data as *mut RecordBatchStreamPrivateData); + drop(private_data); +} + fn record_batch_to_wrapped_array( record_batch: RecordBatch, ) -> RResult { @@ -197,6 +207,12 @@ impl Stream for FFI_RecordBatchStream { } } +impl Drop for FFI_RecordBatchStream { + fn drop(&mut self) { + unsafe { (self.release)(self) } + } +} + #[cfg(test)] mod tests { use std::sync::Arc; From 0d04475ea87a56d327ab772b2b35a94c0faf5bf6 Mon Sep 17 00:00:00 2001 From: Oleks V Date: Thu, 21 Aug 2025 12:54:09 -0700 Subject: [PATCH 13/41] fix: align `array_has` null buffer for scalar (#17272) (#17274) * fix: align `array_has` null buffer for scalar (#17272) * fix: align `array_has` null buffer for scalar * merge --- datafusion/physical-expr-common/src/datum.rs | 21 ++++++++++++++++++-- datafusion/sqllogictest/test_files/array.slt | 15 ++++++++++++++ 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr-common/src/datum.rs b/datafusion/physical-expr-common/src/datum.rs index 233deff758c7b..7084bc440e86b 100644 --- a/datafusion/physical-expr-common/src/datum.rs +++ b/datafusion/physical-expr-common/src/datum.rs @@ -154,9 +154,26 @@ pub fn compare_op_for_nested( if matches!(op, Operator::IsDistinctFrom | Operator::IsNotDistinctFrom) { Ok(BooleanArray::new(values, None)) } else { - // If one of the side is NULL, we returns NULL + // If one of the side is NULL, we return NULL // i.e. NULL eq NULL -> NULL - let nulls = NullBuffer::union(l.nulls(), r.nulls()); + // For nested comparisons, we need to ensure the null buffer matches the result length + let nulls = match (is_l_scalar, is_r_scalar) { + (false, false) | (true, true) => NullBuffer::union(l.nulls(), r.nulls()), + (true, false) => { + // When left is null-scalar and right is array, expand left nulls to match result length + match l.nulls().filter(|nulls| !nulls.is_valid(0)) { + Some(_) => Some(NullBuffer::new_null(len)), // Left scalar is null + None => r.nulls().cloned(), // Left scalar is non-null + } + } + (false, true) => { + // When right is null-scalar and left is array, expand right nulls to match result length + match r.nulls().filter(|nulls| !nulls.is_valid(0)) { + Some(_) => Some(NullBuffer::new_null(len)), // Right scalar is null + None => l.nulls().cloned(), // Right scalar is non-null + } + } + }; Ok(BooleanArray::new(values, nulls)) } } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index a3d9c3e1d9c1f..14261a6e627c9 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -8065,6 +8065,18 @@ FixedSizeList(Field { name: "item", data_type: Int32, nullable: true, dict_id: 0 statement error create table varying_fixed_size_col_table (a int[3]) as values ([1,2,3]), ([4,5]); +statement ok +COPY (select [[true, false], [false, true]] a, [false, true] b union select [[null, null]], null) to 'test_files/scratch/array/array_has/single_file.parquet' stored as parquet; + +statement ok +CREATE EXTERNAL TABLE array_has STORED AS PARQUET location 'test_files/scratch/array/array_has/single_file.parquet'; + +query B +select array_contains(a, b) from array_has order by 1 nulls last; +---- +true +NULL + ### Delete tables statement ok @@ -8243,3 +8255,6 @@ drop table values_all_empty; statement ok drop table fixed_size_col_table; + +statement ok +drop table array_has; From f43df3f2ae3aafb347996c58e852cc378807095b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Thu, 21 Aug 2025 14:49:49 -0700 Subject: [PATCH 14/41] [branch-49] Prepare `49.0.2` version and changelog (#17277) * Update versio to 49.0.2 * Add changelog * update configuration docs --- Cargo.lock | 80 +++++++++++++++---------------- Cargo.toml | 2 +- dev/changelog/49.0.2.md | 45 +++++++++++++++++ docs/source/user-guide/configs.md | 2 +- 4 files changed, 87 insertions(+), 42 deletions(-) create mode 100644 dev/changelog/49.0.2.md diff --git a/Cargo.lock b/Cargo.lock index 6706ed46c02dd..8ffdb8c6403c1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1818,7 +1818,7 @@ dependencies = [ [[package]] name = "datafusion" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "arrow-ipc", @@ -1890,7 +1890,7 @@ dependencies = [ [[package]] name = "datafusion-benchmarks" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "datafusion", @@ -1914,7 +1914,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "async-trait", @@ -1938,7 +1938,7 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "async-trait", @@ -1959,7 +1959,7 @@ dependencies = [ [[package]] name = "datafusion-cli" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "assert_cmd", @@ -1991,7 +1991,7 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "49.0.1" +version = "49.0.2" dependencies = [ "ahash 0.8.12", "apache-avro", @@ -2019,7 +2019,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "49.0.1" +version = "49.0.2" dependencies = [ "futures", "log", @@ -2028,7 +2028,7 @@ dependencies = [ [[package]] name = "datafusion-datasource" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "async-compression", @@ -2063,7 +2063,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-avro" -version = "49.0.1" +version = "49.0.2" dependencies = [ "apache-avro", "arrow", @@ -2088,7 +2088,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "async-trait", @@ -2111,7 +2111,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "async-trait", @@ -2134,7 +2134,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "async-trait", @@ -2166,11 +2166,11 @@ dependencies = [ [[package]] name = "datafusion-doc" -version = "49.0.1" +version = "49.0.2" [[package]] name = "datafusion-examples" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "arrow-flight", @@ -2201,7 +2201,7 @@ dependencies = [ [[package]] name = "datafusion-execution" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "chrono", @@ -2220,7 +2220,7 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "async-trait", @@ -2243,7 +2243,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "datafusion-common", @@ -2254,7 +2254,7 @@ dependencies = [ [[package]] name = "datafusion-ffi" -version = "49.0.1" +version = "49.0.2" dependencies = [ "abi_stable", "arrow", @@ -2275,7 +2275,7 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "arrow-buffer", @@ -2304,7 +2304,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "49.0.1" +version = "49.0.2" dependencies = [ "ahash 0.8.12", "arrow", @@ -2325,7 +2325,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "49.0.1" +version = "49.0.2" dependencies = [ "ahash 0.8.12", "arrow", @@ -2338,7 +2338,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "arrow-ord", @@ -2360,7 +2360,7 @@ dependencies = [ [[package]] name = "datafusion-functions-table" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "async-trait", @@ -2374,7 +2374,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "datafusion-common", @@ -2390,7 +2390,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "49.0.1" +version = "49.0.2" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -2398,7 +2398,7 @@ dependencies = [ [[package]] name = "datafusion-macros" -version = "49.0.1" +version = "49.0.2" dependencies = [ "datafusion-expr", "quote", @@ -2407,7 +2407,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "async-trait", @@ -2434,7 +2434,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "49.0.1" +version = "49.0.2" dependencies = [ "ahash 0.8.12", "arrow", @@ -2459,7 +2459,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "49.0.1" +version = "49.0.2" dependencies = [ "ahash 0.8.12", "arrow", @@ -2471,7 +2471,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "datafusion-common", @@ -2492,7 +2492,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "49.0.1" +version = "49.0.2" dependencies = [ "ahash 0.8.12", "arrow", @@ -2528,7 +2528,7 @@ dependencies = [ [[package]] name = "datafusion-proto" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "chrono", @@ -2550,7 +2550,7 @@ dependencies = [ [[package]] name = "datafusion-proto-common" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "datafusion-common", @@ -2563,7 +2563,7 @@ dependencies = [ [[package]] name = "datafusion-pruning" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "arrow-schema", @@ -2582,7 +2582,7 @@ dependencies = [ [[package]] name = "datafusion-session" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "async-trait", @@ -2604,7 +2604,7 @@ dependencies = [ [[package]] name = "datafusion-spark" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "criterion", @@ -2620,7 +2620,7 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "bigdecimal", @@ -2644,7 +2644,7 @@ dependencies = [ [[package]] name = "datafusion-sqllogictest" -version = "49.0.1" +version = "49.0.2" dependencies = [ "arrow", "async-trait", @@ -2677,7 +2677,7 @@ dependencies = [ [[package]] name = "datafusion-substrait" -version = "49.0.1" +version = "49.0.2" dependencies = [ "async-recursion", "async-trait", @@ -2697,7 +2697,7 @@ dependencies = [ [[package]] name = "datafusion-wasmtest" -version = "49.0.1" +version = "49.0.2" dependencies = [ "chrono", "console_error_panic_hook", diff --git a/Cargo.toml b/Cargo.toml index 742e2b8a19f9b..601d11f12dd81 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -77,7 +77,7 @@ repository = "https://github.com/apache/datafusion" # Define Minimum Supported Rust Version (MSRV) rust-version = "1.85.1" # Define DataFusion version -version = "49.0.1" +version = "49.0.2" [workspace.dependencies] # We turn off default-features for some dependencies here so the workspaces which inherit them can diff --git a/dev/changelog/49.0.2.md b/dev/changelog/49.0.2.md new file mode 100644 index 0000000000000..7e6fc3e7eb487 --- /dev/null +++ b/dev/changelog/49.0.2.md @@ -0,0 +1,45 @@ + + +# Apache DataFusion 49.0.2 Changelog + +This release consists of 3 commits from 3 contributors. See credits at the end of this changelog for more information. + +See the [upgrade guide](https://datafusion.apache.org/library-user-guide/upgrading.html) for information on how to upgrade from previous versions. + +**Fixed bugs:** + +- fix: align `array_has` null buffer for scalar (#17272) [#17274](https://github.com/apache/datafusion/pull/17274) (comphead) + +**Other:** + +- [branch-49] Backport fix: deserialization error for FilterExec (predicates with inlist) [#17254](https://github.com/apache/datafusion/pull/17254) (haohuaijin) +- [branch-49] FFI_RecordBatchStream was causing a memory leak (#17190) [#17270](https://github.com/apache/datafusion/pull/17270) (timsaucer) + +## Credits + +Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. + +``` + 1 Huaijin + 1 Oleks V + 1 Tim Saucer +``` + +Thank you also to everyone who contributed in other ways such as filing issues, reviewing PRs, and providing feedback on this release. diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index dc950de01f1b1..d453cb0684daf 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -70,7 +70,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_statistics_size | 4096 | (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting max_statistics_size is deprecated, currently it is not being used | | datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | -| datafusion.execution.parquet.created_by | datafusion version 49.0.1 | (writing) Sets "created by" property | +| datafusion.execution.parquet.created_by | datafusion version 49.0.2 | (writing) Sets "created by" property | | datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | | datafusion.execution.parquet.statistics_truncate_length | NULL | (writing) Sets statictics truncate length. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | From 25058de27fe70bafa61514e445ea5cda2d9b3661 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Fri, 22 Aug 2025 15:25:19 +0800 Subject: [PATCH 15/41] fix cargo check --profile ci --no-default-features -p datafusion-proto --- datafusion/proto/Cargo.toml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index e9a8b83dc4f91..a1eeabdf87f4a 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -46,7 +46,7 @@ avro = ["datafusion/avro", "datafusion-common/avro"] [dependencies] arrow = { workspace = true } chrono = { workspace = true } -datafusion = { workspace = true, default-features = false } +datafusion = { workspace = true, default-features = true } datafusion-common = { workspace = true, default-features = true } datafusion-expr = { workspace = true } datafusion-proto-common = { workspace = true } @@ -56,7 +56,6 @@ prost = { workspace = true } serde = { version = "1.0", optional = true } serde_json = { workspace = true, optional = true } [dev-dependencies] -datafusion = { workspace = true, default-features = true } datafusion-functions = { workspace = true, default-features = true } datafusion-functions-aggregate = { workspace = true } datafusion-functions-window-common = { workspace = true } From c46f7a9e4c5fd7c3bdf0d86141d2282af97896bf Mon Sep 17 00:00:00 2001 From: xudong963 Date: Fri, 22 Aug 2025 15:34:35 +0800 Subject: [PATCH 16/41] fix cargo doc --- datafusion/physical-plan/src/aggregates/group_values/mod.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index 6c0584bdf5c25..1e4c7558bda39 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! [`GroupValues`] trait for storing and interning group keys +//! `GroupValues` trait for storing and interning group keys use arrow::array::types::{ Date32Type, Date64Type, Decimal128Type, Time32MillisecondType, Time32SecondType, @@ -84,7 +84,7 @@ mod null_builder; /// Each distinct group in a hash aggregation is identified by a unique group id /// (usize) which is assigned by instances of this trait. Group ids are /// continuous without gaps, starting from 0. -pub trait GroupValues: Send { +pub(crate) trait GroupValues: Send { /// Calculates the group id for each input row of `cols`, assigning new /// group ids as necessary. /// @@ -127,7 +127,7 @@ pub trait GroupValues: Send { /// /// [`GroupColumn`]: crate::aggregates::group_values::multi_group_by::GroupColumn /// -pub fn new_group_values( +pub(crate) fn new_group_values( schema: SchemaRef, group_ordering: &GroupOrdering, ) -> Result> { From deaf2e2d6dc1ac8b0c293494a865546f1682c949 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Fri, 22 Aug 2025 17:18:21 +0800 Subject: [PATCH 17/41] fix ut:custom_sources_cases::statistics::sql_limit(with_node_id of CoalescePartitionsExec missed fetch) --- .github/workflows/audit.yml | 2 +- datafusion/physical-plan/src/coalesce_partitions.rs | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/audit.yml b/.github/workflows/audit.yml index 491fa27c2a56a..ce41442f3c167 100644 --- a/.github/workflows/audit.yml +++ b/.github/workflows/audit.yml @@ -44,4 +44,4 @@ jobs: - name: Run audit check # Ignored until https://github.com/apache/datafusion/issues/15571 # ignored py03 warning until arrow 55 upgrade - run: cargo audit --ignore RUSTSEC-2024-0370 --ignore RUSTSEC-2025-0020 + run: cargo audit --ignore RUSTSEC-2024-0370 --ignore RUSTSEC-2025-0020 --ignore RUSTSEC-2025-0047 diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index d7beffb19faac..7daf0d753d15f 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -217,6 +217,7 @@ impl ExecutionPlan for CoalescePartitionsExec { _node_id: usize, ) -> Result>> { let mut new_plan = CoalescePartitionsExec::new(Arc::clone(self.input())); + new_plan.fetch = self.fetch; let new_props = new_plan.cache.clone().with_node_id(_node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) From f1b1bd83193554174a0f6eadcce782e51d181d15 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Fri, 22 Aug 2025 17:54:24 +0800 Subject: [PATCH 18/41] fix ut: test_no_pushdown_through_aggregates & test_plan_with_order_preserving_variants_preserves_fetch --- .../core/tests/physical_optimizer/filter_pushdown/mod.rs | 2 +- .../replace_with_order_preserving_variants.rs | 5 ----- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs index a28933d97bcd1..040b99f5ffdc1 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/mod.rs @@ -276,7 +276,7 @@ fn test_no_pushdown_through_aggregates() { Ok: - FilterExec: b@1 = bar - CoalesceBatchesExec: target_batch_size=100 - - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt] + - AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b], aggr=[cnt], ordering_mode=PartiallySorted([0]) - CoalesceBatchesExec: target_batch_size=10 - DataSourceExec: file_groups={1 group: [[test.paqruet]]}, projection=[a, b, c], file_type=test, pushdown_supported=true, predicate=a@0 = foo " diff --git a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs index caa536107d8da..9769e2e0366f7 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -139,11 +139,6 @@ pub fn plan_with_order_preserving_variants( return Ok(sort_input); } else if is_coalesce_partitions(&sort_input.plan) && is_spm_better { let child = &sort_input.children[0].plan; - let mut fetch = fetch; - if let Some(coalesce_fetch) = sort_input.plan.fetch() { - // Get the min fetch between the `fetch` and the coalesce's fetch: - fetch = Some(coalesce_fetch.min(fetch.unwrap_or(usize::MAX))) - }; if let Some(ordering) = child.output_ordering() { let mut fetch = fetch; if let Some(coalesce_fetch) = sort_input.plan.fetch() { From 7dd5e6e7dff05047281e4a6cbdc31d093a4932d0 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Fri, 22 Aug 2025 18:27:52 +0800 Subject: [PATCH 19/41] fix format --- datafusion/optimizer/src/push_down_filter.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index cfa22c3b61426..f701fcf861e0d 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -2308,7 +2308,7 @@ mod tests { plan, @r" Projection: test.a, test1.d - Cross Join: + Cross Join: Projection: test.a, test.b, test.c TableScan: test, full_filters=[test.a = Int32(1)] Projection: test1.d, test1.e, test1.f @@ -2338,7 +2338,7 @@ mod tests { plan, @r" Projection: test.a, test1.a - Cross Join: + Cross Join: Projection: test.a, test.b, test.c TableScan: test, full_filters=[test.a = Int32(1)] Projection: test1.a, test1.b, test1.c From 2eca4c0d67ce39550d7004624c6d443f43bee25f Mon Sep 17 00:00:00 2001 From: xudong963 Date: Fri, 22 Aug 2025 19:03:23 +0800 Subject: [PATCH 20/41] fix roundtrip_test --- datafusion/physical-plan/src/empty.rs | 10 ++++++++++ .../proto/tests/cases/roundtrip_physical_plan.rs | 14 +++++++++++++- 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index 36634fbe6d7e9..eae1aaff1c492 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -173,6 +173,16 @@ impl ExecutionPlan for EmptyExec { None, )) } + + fn with_node_id( + self: Arc, + _node_id: usize, + ) -> Result>> { + let mut new_plan = EmptyExec::new(self.schema.clone()); + let new_props = new_plan.cache.clone().with_node_id(_node_id); + new_plan.cache = new_props; + Ok(Some(Arc::new(new_plan))) + } } #[cfg(test)] diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 7d56bb6c5db1b..b3827dae51b24 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -31,6 +31,9 @@ use arrow::csv::WriterBuilder; use arrow::datatypes::{Fields, TimeUnit}; use datafusion::physical_expr::aggregate::AggregateExprBuilder; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion::physical_plan::node_id::{ + annotate_node_id_for_execution_plan, NodeIdAnnotator, +}; use datafusion_expr::dml::InsertOp; use datafusion_functions_aggregate::approx_percentile_cont::approx_percentile_cont_udaf; use datafusion_functions_aggregate::array_agg::array_agg_udaf; @@ -133,13 +136,22 @@ fn roundtrip_test_and_return( ctx: &SessionContext, codec: &dyn PhysicalExtensionCodec, ) -> Result> { + let mut annotator = NodeIdAnnotator::new(); + let exec_plan = annotate_node_id_for_execution_plan(&exec_plan, &mut annotator)?; let proto: protobuf::PhysicalPlanNode = protobuf::PhysicalPlanNode::try_from_physical_plan(exec_plan.clone(), codec) .expect("to proto"); let runtime = ctx.runtime_env(); - let result_exec_plan: Arc = proto + let mut result_exec_plan: Arc = proto .try_into_physical_plan(ctx, runtime.deref(), codec) .expect("from proto"); + + // Re-annotate the deserialized plan with node IDs to match the original plan structure + // This ensures that the roundtrip preserves the node_id values for comparison + let mut annotator = NodeIdAnnotator::new(); + result_exec_plan = + annotate_node_id_for_execution_plan(&result_exec_plan, &mut annotator)?; + assert_eq!(format!("{exec_plan:?}"), format!("{result_exec_plan:?}")); Ok(result_exec_plan) } From 8baa05db257c9a9e07c148269faf3388d39d49df Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 25 Aug 2025 12:03:01 +0800 Subject: [PATCH 21/41] schema_force_view_types to true --- datafusion/common/src/config.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 0d34815a248f7..883d2b60a8976 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -457,7 +457,7 @@ config_namespace! { /// (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, /// and `Binary/BinaryLarge` with `BinaryView`. - pub schema_force_view_types: bool, default = false + pub schema_force_view_types: bool, default = true /// (reading) If true, parquet reader will read columns of /// `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. From 9b2fbbbcaca511a184dd331b679f62c9721e8bd8 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 25 Aug 2025 12:40:32 +0800 Subject: [PATCH 22/41] use utfview8 --- datafusion/core/tests/parquet/page_pruning.rs | 2 +- .../sqllogictest/test_files/describe.slt | 4 +- .../test_files/information_schema.slt | 4 +- datafusion/sqllogictest/test_files/map.slt | 2 +- .../sqllogictest/test_files/parquet.slt | 46 +++++++++---------- .../test_files/simplify_predicates.slt | 4 +- 6 files changed, 31 insertions(+), 31 deletions(-) diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 12e3436550c57..9da879a32f6b5 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -160,7 +160,7 @@ async fn page_index_filter_one_col() { // 5.create filter date_string_col == "01/01/09"`; // Note this test doesn't apply type coercion so the literal must match the actual view type - let filter = col("date_string_col").eq(lit(ScalarValue::new_utf8("01/01/09"))); + let filter = col("date_string_col").eq(lit(ScalarValue::new_utf8view("01/01/09"))); let parquet_exec = get_parquet_exec(&state, filter).await; let mut results = parquet_exec.execute(0, task_ctx.clone()).unwrap(); let batch = results.next().await.unwrap().unwrap(); diff --git a/datafusion/sqllogictest/test_files/describe.slt b/datafusion/sqllogictest/test_files/describe.slt index 077e8e6474d1f..e4cb30628eec5 100644 --- a/datafusion/sqllogictest/test_files/describe.slt +++ b/datafusion/sqllogictest/test_files/describe.slt @@ -81,8 +81,8 @@ int_col Int32 YES bigint_col Int64 YES float_col Float32 YES double_col Float64 YES -date_string_col Utf8 YES -string_col Utf8 YES +date_string_col Utf8View YES +string_col Utf8View YES timestamp_col Timestamp(Nanosecond, None) YES year Int32 YES month Int32 YES diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 9f39dbbd5ba25..2ce64ffc68365 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -249,7 +249,7 @@ datafusion.execution.parquet.metadata_size_hint NULL datafusion.execution.parquet.pruning true datafusion.execution.parquet.pushdown_filters false datafusion.execution.parquet.reorder_filters false -datafusion.execution.parquet.schema_force_view_types false +datafusion.execution.parquet.schema_force_view_types true datafusion.execution.parquet.skip_arrow_metadata false datafusion.execution.parquet.skip_metadata true datafusion.execution.parquet.statistics_enabled page @@ -359,7 +359,7 @@ datafusion.execution.parquet.metadata_size_hint NULL (reading) If specified, the datafusion.execution.parquet.pruning true (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file datafusion.execution.parquet.pushdown_filters false (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". datafusion.execution.parquet.reorder_filters false (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query -datafusion.execution.parquet.schema_force_view_types false (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. +datafusion.execution.parquet.schema_force_view_types true (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. datafusion.execution.parquet.skip_arrow_metadata false (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to datafusion.execution.parquet.skip_metadata true (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata datafusion.execution.parquet.statistics_enabled page (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 8546bdbdd0673..42a4ba6218016 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -45,7 +45,7 @@ describe data; ---- ints Map(Field { name: "entries", data_type: Struct([Field { name: "key", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: "value", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }]), nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, false) NO strings Map(Field { name: "entries", data_type: Struct([Field { name: "key", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: "value", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }]), nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, false) NO -timestamp Utf8 NO +timestamp Utf8View NO query ??T SELECT * FROM data ORDER by ints['bytes'] DESC LIMIT 10; diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index e5b3f740e2912..abc6fdab3c8a0 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -384,15 +384,15 @@ select arrow_typeof(binaryview_col), binaryview_col FROM binary_as_string_default; ---- -Binary 616161 Binary 616161 Binary 616161 -Binary 626262 Binary 626262 Binary 626262 -Binary 636363 Binary 636363 Binary 636363 -Binary 646464 Binary 646464 Binary 646464 -Binary 656565 Binary 656565 Binary 656565 -Binary 666666 Binary 666666 Binary 666666 -Binary 676767 Binary 676767 Binary 676767 -Binary 686868 Binary 686868 Binary 686868 -Binary 696969 Binary 696969 Binary 696969 +BinaryView 616161 BinaryView 616161 BinaryView 616161 +BinaryView 626262 BinaryView 626262 BinaryView 626262 +BinaryView 636363 BinaryView 636363 BinaryView 636363 +BinaryView 646464 BinaryView 646464 BinaryView 646464 +BinaryView 656565 BinaryView 656565 BinaryView 656565 +BinaryView 666666 BinaryView 666666 BinaryView 666666 +BinaryView 676767 BinaryView 676767 BinaryView 676767 +BinaryView 686868 BinaryView 686868 BinaryView 686868 +BinaryView 696969 BinaryView 696969 BinaryView 696969 # Run an explain plan to show the cast happens in the plan (a CAST is needed for the predicates) query TT @@ -405,11 +405,11 @@ EXPLAIN binaryview_col LIKE '%a%'; ---- logical_plan -01)Filter: CAST(binary_as_string_default.binary_col AS Utf8) LIKE Utf8("%a%") AND CAST(binary_as_string_default.largebinary_col AS Utf8) LIKE Utf8("%a%") AND CAST(binary_as_string_default.binaryview_col AS Utf8) LIKE Utf8("%a%") -02)--TableScan: binary_as_string_default projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[CAST(binary_as_string_default.binary_col AS Utf8) LIKE Utf8("%a%"), CAST(binary_as_string_default.largebinary_col AS Utf8) LIKE Utf8("%a%"), CAST(binary_as_string_default.binaryview_col AS Utf8) LIKE Utf8("%a%")] +01)Filter: CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%") AND CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%") AND CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%") +02)--TableScan: binary_as_string_default projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%")] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: CAST(binary_col@0 AS Utf8) LIKE %a% AND CAST(largebinary_col@1 AS Utf8) LIKE %a% AND CAST(binaryview_col@2 AS Utf8) LIKE %a% +02)--FilterExec: CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% @@ -432,15 +432,15 @@ select arrow_typeof(binaryview_col), binaryview_col FROM binary_as_string_option; ---- -Utf8 aaa Utf8 aaa Utf8 aaa -Utf8 bbb Utf8 bbb Utf8 bbb -Utf8 ccc Utf8 ccc Utf8 ccc -Utf8 ddd Utf8 ddd Utf8 ddd -Utf8 eee Utf8 eee Utf8 eee -Utf8 fff Utf8 fff Utf8 fff -Utf8 ggg Utf8 ggg Utf8 ggg -Utf8 hhh Utf8 hhh Utf8 hhh -Utf8 iii Utf8 iii Utf8 iii +Utf8View aaa Utf8View aaa Utf8View aaa +Utf8View bbb Utf8View bbb Utf8View bbb +Utf8View ccc Utf8View ccc Utf8View ccc +Utf8View ddd Utf8View ddd Utf8View ddd +Utf8View eee Utf8View eee Utf8View eee +Utf8View fff Utf8View fff Utf8View fff +Utf8View ggg Utf8View ggg Utf8View ggg +Utf8View hhh Utf8View hhh Utf8View hhh +Utf8View iii Utf8View iii Utf8View iii # Run an explain plan to show the cast happens in the plan (there should be no casts) query TT @@ -453,8 +453,8 @@ EXPLAIN binaryview_col LIKE '%a%'; ---- logical_plan -01)Filter: binary_as_string_option.binary_col LIKE Utf8("%a%") AND binary_as_string_option.largebinary_col LIKE Utf8("%a%") AND binary_as_string_option.binaryview_col LIKE Utf8("%a%") -02)--TableScan: binary_as_string_option projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_option.binary_col LIKE Utf8("%a%"), binary_as_string_option.largebinary_col LIKE Utf8("%a%"), binary_as_string_option.binaryview_col LIKE Utf8("%a%")] +01)Filter: binary_as_string_option.binary_col LIKE Utf8View("%a%") AND binary_as_string_option.largebinary_col LIKE Utf8View("%a%") AND binary_as_string_option.binaryview_col LIKE Utf8View("%a%") +02)--TableScan: binary_as_string_option projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_option.binary_col LIKE Utf8View("%a%"), binary_as_string_option.largebinary_col LIKE Utf8View("%a%"), binary_as_string_option.binaryview_col LIKE Utf8View("%a%")] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% diff --git a/datafusion/sqllogictest/test_files/simplify_predicates.slt b/datafusion/sqllogictest/test_files/simplify_predicates.slt index 6d3eefdfc8213..cef78d97bb46c 100644 --- a/datafusion/sqllogictest/test_files/simplify_predicates.slt +++ b/datafusion/sqllogictest/test_files/simplify_predicates.slt @@ -84,7 +84,7 @@ query TT EXPLAIN SELECT * FROM test_data WHERE str_col > 'apple' AND str_col > 'banana'; ---- logical_plan -01)Filter: test_data.str_col > Utf8("banana") +01)Filter: test_data.str_col > Utf8View("banana") 02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] # date_col > '2023-01-01' AND date_col > '2023-02-01' should simplify to date_col > '2023-02-01' @@ -120,7 +120,7 @@ WHERE int_col > 5 AND float_col BETWEEN 1 AND 100; ---- logical_plan -01)Filter: test_data.str_col LIKE Utf8("A%") AND test_data.float_col >= Float32(1) AND test_data.float_col <= Float32(100) AND test_data.int_col > Int32(10) +01)Filter: test_data.str_col LIKE Utf8View("A%") AND test_data.float_col >= Float32(1) AND test_data.float_col <= Float32(100) AND test_data.int_col > Int32(10) 02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] statement ok From 63c2ebc6667d5de4b5214496fd2bea421ddc8e69 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 25 Aug 2025 12:56:36 +0800 Subject: [PATCH 23/41] schema_force_view_types to false(try true after df49) --- datafusion/common/src/config.rs | 2 +- .../sqllogictest/test_files/describe.slt | 4 +- .../sqllogictest/test_files/explain.slt | 12 +- .../sqllogictest/test_files/explain_tree.slt | 108 ++++++------------ .../test_files/information_schema.slt | 4 +- datafusion/sqllogictest/test_files/limit.slt | 2 +- .../test_files/listing_table_statistics.slt | 2 +- datafusion/sqllogictest/test_files/map.slt | 2 +- .../sqllogictest/test_files/parquet.slt | 52 ++++----- 9 files changed, 76 insertions(+), 112 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 883d2b60a8976..0d34815a248f7 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -457,7 +457,7 @@ config_namespace! { /// (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, /// and `Binary/BinaryLarge` with `BinaryView`. - pub schema_force_view_types: bool, default = true + pub schema_force_view_types: bool, default = false /// (reading) If true, parquet reader will read columns of /// `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. diff --git a/datafusion/sqllogictest/test_files/describe.slt b/datafusion/sqllogictest/test_files/describe.slt index e4cb30628eec5..077e8e6474d1f 100644 --- a/datafusion/sqllogictest/test_files/describe.slt +++ b/datafusion/sqllogictest/test_files/describe.slt @@ -81,8 +81,8 @@ int_col Int32 YES bigint_col Int64 YES float_col Float32 YES double_col Float64 YES -date_string_col Utf8View YES -string_col Utf8View YES +date_string_col Utf8 YES +string_col Utf8 YES timestamp_col Timestamp(Nanosecond, None) YES year Int32 YES month Int32 YES diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index fff82ca1e5927..235f95eb4595d 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -297,8 +297,8 @@ initial_physical_plan 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] 02)--DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] initial_physical_plan_with_schema -01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] -02)--DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:Binary;N, string_col:Binary;N, timestamp_col:Timestamp(Nanosecond, None);N] +02)--DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:Binary;N, string_col:Binary;N, timestamp_col:Timestamp(Nanosecond, None);N] physical_plan after OutputRequirements 01)OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] 02)--GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] @@ -321,7 +321,7 @@ physical_plan after LimitPushdown DataSourceExec: file_groups={1 group: [[WORKSP physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE physical_plan DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan_with_schema DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +physical_plan_with_schema DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:Binary;N, string_col:Binary;N, timestamp_col:Timestamp(Nanosecond, None);N] statement ok @@ -338,8 +338,8 @@ initial_physical_plan_with_stats 01)GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] 02)--DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] initial_physical_plan_with_schema -01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] -02)--DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +01)GlobalLimitExec: skip=0, fetch=10, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:Binary;N, string_col:Binary;N, timestamp_col:Timestamp(Nanosecond, None);N] +02)--DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:Binary;N, string_col:Binary;N, timestamp_col:Timestamp(Nanosecond, None);N] physical_plan after OutputRequirements 01)OutputRequirementExec 02)--GlobalLimitExec: skip=0, fetch=10 @@ -363,7 +363,7 @@ physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SanityCheckPlan SAME TEXT AS ABOVE physical_plan DataSourceExec: 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, file_type=parquet physical_plan_with_stats DataSourceExec: 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, file_type=parquet, statistics=[Rows=Exact(8), Bytes=Exact(671), [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan_with_schema DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:BinaryView;N, string_col:BinaryView;N, timestamp_col:Timestamp(Nanosecond, None);N] +physical_plan_with_schema DataSourceExec: 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, file_type=parquet, schema=[id:Int32;N, bool_col:Boolean;N, tinyint_col:Int32;N, smallint_col:Int32;N, int_col:Int32;N, bigint_col:Int64;N, float_col:Float32;N, double_col:Float64;N, date_string_col:Binary;N, string_col:Binary;N, timestamp_col:Timestamp(Nanosecond, None);N] statement ok diff --git a/datafusion/sqllogictest/test_files/explain_tree.slt b/datafusion/sqllogictest/test_files/explain_tree.slt index 22183195c3df0..9d404fac603ca 100644 --- a/datafusion/sqllogictest/test_files/explain_tree.slt +++ b/datafusion/sqllogictest/test_files/explain_tree.slt @@ -1185,42 +1185,24 @@ physical_plan 21)│ -------------------- │ 22)│ on: │ 23)│ (int_col = int_col), ├──────────────┐ -24)│ (string_col = CAST │ │ -25)│ (table1.string_col AS │ │ -26)│ Utf8View)) │ │ -27)└─────────────┬─────────────┘ │ -28)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -29)│ DataSourceExec ││ ProjectionExec │ -30)│ -------------------- ││ -------------------- │ -31)│ files: 1 ││ CAST(table1.string_col AS │ -32)│ format: parquet ││ Utf8View): │ -33)│ ││ CAST(string_col AS │ -34)│ ││ Utf8View) │ -35)│ ││ │ -36)│ ││ bigint_col: │ -37)│ ││ bigint_col │ -38)│ ││ │ -39)│ ││ date_col: date_col │ -40)│ ││ int_col: int_col │ -41)│ ││ │ -42)│ ││ string_col: │ -43)│ ││ string_col │ -44)└───────────────────────────┘└─────────────┬─────────────┘ -45)-----------------------------┌─────────────┴─────────────┐ -46)-----------------------------│ RepartitionExec │ -47)-----------------------------│ -------------------- │ -48)-----------------------------│ partition_count(in->out): │ -49)-----------------------------│ 1 -> 4 │ -50)-----------------------------│ │ -51)-----------------------------│ partitioning_scheme: │ -52)-----------------------------│ RoundRobinBatch(4) │ -53)-----------------------------└─────────────┬─────────────┘ -54)-----------------------------┌─────────────┴─────────────┐ -55)-----------------------------│ DataSourceExec │ -56)-----------------------------│ -------------------- │ -57)-----------------------------│ files: 1 │ -58)-----------------------------│ format: csv │ -59)-----------------------------└───────────────────────────┘ +24)│ (string_col = │ │ +25)│ string_col) │ │ +26)└─────────────┬─────────────┘ │ +27)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +28)│ DataSourceExec ││ RepartitionExec │ +29)│ -------------------- ││ -------------------- │ +30)│ files: 1 ││ partition_count(in->out): │ +31)│ format: parquet ││ 1 -> 4 │ +32)│ ││ │ +33)│ ││ partitioning_scheme: │ +34)│ ││ RoundRobinBatch(4) │ +35)└───────────────────────────┘└─────────────┬─────────────┘ +36)-----------------------------┌─────────────┴─────────────┐ +37)-----------------------------│ DataSourceExec │ +38)-----------------------------│ -------------------- │ +39)-----------------------------│ files: 1 │ +40)-----------------------------│ format: csv │ +41)-----------------------------└───────────────────────────┘ # Query with outer hash join. query TT @@ -1252,42 +1234,24 @@ physical_plan 23)│ │ 24)│ on: ├──────────────┐ 25)│ (int_col = int_col), │ │ -26)│ (string_col = CAST │ │ -27)│ (table1.string_col AS │ │ -28)│ Utf8View)) │ │ -29)└─────────────┬─────────────┘ │ -30)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ -31)│ DataSourceExec ││ ProjectionExec │ -32)│ -------------------- ││ -------------------- │ -33)│ files: 1 ││ CAST(table1.string_col AS │ -34)│ format: parquet ││ Utf8View): │ -35)│ ││ CAST(string_col AS │ -36)│ ││ Utf8View) │ -37)│ ││ │ -38)│ ││ bigint_col: │ -39)│ ││ bigint_col │ -40)│ ││ │ -41)│ ││ date_col: date_col │ -42)│ ││ int_col: int_col │ -43)│ ││ │ -44)│ ││ string_col: │ -45)│ ││ string_col │ -46)└───────────────────────────┘└─────────────┬─────────────┘ -47)-----------------------------┌─────────────┴─────────────┐ -48)-----------------------------│ RepartitionExec │ -49)-----------------------------│ -------------------- │ -50)-----------------------------│ partition_count(in->out): │ -51)-----------------------------│ 1 -> 4 │ -52)-----------------------------│ │ -53)-----------------------------│ partitioning_scheme: │ -54)-----------------------------│ RoundRobinBatch(4) │ -55)-----------------------------└─────────────┬─────────────┘ -56)-----------------------------┌─────────────┴─────────────┐ -57)-----------------------------│ DataSourceExec │ -58)-----------------------------│ -------------------- │ -59)-----------------------------│ files: 1 │ -60)-----------------------------│ format: csv │ -61)-----------------------------└───────────────────────────┘ +26)│ (string_col = │ │ +27)│ string_col) │ │ +28)└─────────────┬─────────────┘ │ +29)┌─────────────┴─────────────┐┌─────────────┴─────────────┐ +30)│ DataSourceExec ││ RepartitionExec │ +31)│ -------------------- ││ -------------------- │ +32)│ files: 1 ││ partition_count(in->out): │ +33)│ format: parquet ││ 1 -> 4 │ +34)│ ││ │ +35)│ ││ partitioning_scheme: │ +36)│ ││ RoundRobinBatch(4) │ +37)└───────────────────────────┘└─────────────┬─────────────┘ +38)-----------------------------┌─────────────┴─────────────┐ +39)-----------------------------│ DataSourceExec │ +40)-----------------------------│ -------------------- │ +41)-----------------------------│ files: 1 │ +42)-----------------------------│ format: csv │ +43)-----------------------------└───────────────────────────┘ # Query with nested loop join. query TT diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 2ce64ffc68365..9f39dbbd5ba25 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -249,7 +249,7 @@ datafusion.execution.parquet.metadata_size_hint NULL datafusion.execution.parquet.pruning true datafusion.execution.parquet.pushdown_filters false datafusion.execution.parquet.reorder_filters false -datafusion.execution.parquet.schema_force_view_types true +datafusion.execution.parquet.schema_force_view_types false datafusion.execution.parquet.skip_arrow_metadata false datafusion.execution.parquet.skip_metadata true datafusion.execution.parquet.statistics_enabled page @@ -359,7 +359,7 @@ datafusion.execution.parquet.metadata_size_hint NULL (reading) If specified, the datafusion.execution.parquet.pruning true (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file datafusion.execution.parquet.pushdown_filters false (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". datafusion.execution.parquet.reorder_filters false (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query -datafusion.execution.parquet.schema_force_view_types true (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. +datafusion.execution.parquet.schema_force_view_types false (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. datafusion.execution.parquet.skip_arrow_metadata false (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to datafusion.execution.parquet.skip_metadata true (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata datafusion.execution.parquet.statistics_enabled page (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 1af14a52e2bc6..894ba424eddcc 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -854,7 +854,7 @@ physical_plan 02)--SortExec: TopK(fetch=1000), expr=[part_key@1 ASC NULLS LAST], preserve_partitioning=[false] 03)----ProjectionExec: expr=[1 as foo, part_key@0 as part_key] 04)------CoalescePartitionsExec: fetch=1 -05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:0..265], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:265..530], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:530..794]]}, projection=[part_key], limit=1, file_type=parquet +05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:0..265], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:265..530], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:530..794]]}, projection=[part_key], limit=1, file_type=parquet query I with selection as ( diff --git a/datafusion/sqllogictest/test_files/listing_table_statistics.slt b/datafusion/sqllogictest/test_files/listing_table_statistics.slt index 890d1f2e9250e..d3af6b321c7e6 100644 --- a/datafusion/sqllogictest/test_files/listing_table_statistics.slt +++ b/datafusion/sqllogictest/test_files/listing_table_statistics.slt @@ -35,7 +35,7 @@ query TT explain format indent select * from t; ---- logical_plan TableScan: t projection=[int_col, str_col] -physical_plan DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/table/2.parquet]]}, projection=[int_col, str_col], file_type=parquet, statistics=[Rows=Exact(4), Bytes=Exact(288), [(Col[0]: Min=Exact(Int64(-1)) Max=Exact(Int64(3)) Null=Exact(0)),(Col[1]: Min=Exact(Utf8View("a")) Max=Exact(Utf8View("d")) Null=Exact(0))]] +physical_plan DataSourceExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/table/2.parquet]]}, projection=[int_col, str_col], file_type=parquet, statistics=[Rows=Exact(4), Bytes=Exact(288), [(Col[0]: Min=Exact(Int64(-1)) Max=Exact(Int64(3)) Null=Exact(0)),(Col[1]: Min=Exact(Utf8("a")) Max=Exact(Utf8("d")) Null=Exact(0))]] statement ok drop table t; diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 42a4ba6218016..8546bdbdd0673 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -45,7 +45,7 @@ describe data; ---- ints Map(Field { name: "entries", data_type: Struct([Field { name: "key", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: "value", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }]), nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, false) NO strings Map(Field { name: "entries", data_type: Struct([Field { name: "key", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, Field { name: "value", data_type: Utf8, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }]), nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }, false) NO -timestamp Utf8View NO +timestamp Utf8 NO query ??T SELECT * FROM data ORDER by ints['bytes'] DESC LIMIT 10; diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index abc6fdab3c8a0..3e640d7d6b9a8 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -384,15 +384,15 @@ select arrow_typeof(binaryview_col), binaryview_col FROM binary_as_string_default; ---- -BinaryView 616161 BinaryView 616161 BinaryView 616161 -BinaryView 626262 BinaryView 626262 BinaryView 626262 -BinaryView 636363 BinaryView 636363 BinaryView 636363 -BinaryView 646464 BinaryView 646464 BinaryView 646464 -BinaryView 656565 BinaryView 656565 BinaryView 656565 -BinaryView 666666 BinaryView 666666 BinaryView 666666 -BinaryView 676767 BinaryView 676767 BinaryView 676767 -BinaryView 686868 BinaryView 686868 BinaryView 686868 -BinaryView 696969 BinaryView 696969 BinaryView 696969 +Binary 616161 LargeBinary 616161 BinaryView 616161 +Binary 626262 LargeBinary 626262 BinaryView 626262 +Binary 636363 LargeBinary 636363 BinaryView 636363 +Binary 646464 LargeBinary 646464 BinaryView 646464 +Binary 656565 LargeBinary 656565 BinaryView 656565 +Binary 666666 LargeBinary 666666 BinaryView 666666 +Binary 676767 LargeBinary 676767 BinaryView 676767 +Binary 686868 LargeBinary 686868 BinaryView 686868 +Binary 696969 LargeBinary 696969 BinaryView 696969 # Run an explain plan to show the cast happens in the plan (a CAST is needed for the predicates) query TT @@ -405,13 +405,13 @@ EXPLAIN binaryview_col LIKE '%a%'; ---- logical_plan -01)Filter: CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%") AND CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%") AND CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%") -02)--TableScan: binary_as_string_default projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[CAST(binary_as_string_default.binary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.largebinary_col AS Utf8View) LIKE Utf8View("%a%"), CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%")] +01)Filter: CAST(binary_as_string_default.binary_col AS Utf8) LIKE Utf8("%a%") AND CAST(binary_as_string_default.largebinary_col AS LargeUtf8) LIKE LargeUtf8("%a%") AND CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%") +02)--TableScan: binary_as_string_default projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[CAST(binary_as_string_default.binary_col AS Utf8) LIKE Utf8("%a%"), CAST(binary_as_string_default.largebinary_col AS LargeUtf8) LIKE LargeUtf8("%a%"), CAST(binary_as_string_default.binaryview_col AS Utf8View) LIKE Utf8View("%a%")] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% +02)--FilterExec: CAST(binary_col@0 AS Utf8) LIKE %a% AND CAST(largebinary_col@1 AS LargeUtf8) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% 03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% +04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], file_type=parquet, predicate=CAST(binary_col@0 AS Utf8) LIKE %a% AND CAST(largebinary_col@1 AS LargeUtf8) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% statement ok @@ -432,15 +432,15 @@ select arrow_typeof(binaryview_col), binaryview_col FROM binary_as_string_option; ---- -Utf8View aaa Utf8View aaa Utf8View aaa -Utf8View bbb Utf8View bbb Utf8View bbb -Utf8View ccc Utf8View ccc Utf8View ccc -Utf8View ddd Utf8View ddd Utf8View ddd -Utf8View eee Utf8View eee Utf8View eee -Utf8View fff Utf8View fff Utf8View fff -Utf8View ggg Utf8View ggg Utf8View ggg -Utf8View hhh Utf8View hhh Utf8View hhh -Utf8View iii Utf8View iii Utf8View iii +Utf8 aaa LargeUtf8 aaa Utf8View aaa +Utf8 bbb LargeUtf8 bbb Utf8View bbb +Utf8 ccc LargeUtf8 ccc Utf8View ccc +Utf8 ddd LargeUtf8 ddd Utf8View ddd +Utf8 eee LargeUtf8 eee Utf8View eee +Utf8 fff LargeUtf8 fff Utf8View fff +Utf8 ggg LargeUtf8 ggg Utf8View ggg +Utf8 hhh LargeUtf8 hhh Utf8View hhh +Utf8 iii LargeUtf8 iii Utf8View iii # Run an explain plan to show the cast happens in the plan (there should be no casts) query TT @@ -453,8 +453,8 @@ EXPLAIN binaryview_col LIKE '%a%'; ---- logical_plan -01)Filter: binary_as_string_option.binary_col LIKE Utf8View("%a%") AND binary_as_string_option.largebinary_col LIKE Utf8View("%a%") AND binary_as_string_option.binaryview_col LIKE Utf8View("%a%") -02)--TableScan: binary_as_string_option projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_option.binary_col LIKE Utf8View("%a%"), binary_as_string_option.largebinary_col LIKE Utf8View("%a%"), binary_as_string_option.binaryview_col LIKE Utf8View("%a%")] +01)Filter: binary_as_string_option.binary_col LIKE Utf8("%a%") AND binary_as_string_option.largebinary_col LIKE LargeUtf8("%a%") AND binary_as_string_option.binaryview_col LIKE Utf8View("%a%") +02)--TableScan: binary_as_string_option projection=[binary_col, largebinary_col, binaryview_col], partial_filters=[binary_as_string_option.binary_col LIKE Utf8("%a%"), binary_as_string_option.largebinary_col LIKE LargeUtf8("%a%"), binary_as_string_option.binaryview_col LIKE Utf8View("%a%")] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% @@ -619,8 +619,8 @@ query TT explain select * from foo where starts_with(column1, 'f'); ---- logical_plan -01)Filter: foo.column1 LIKE Utf8View("f%") -02)--TableScan: foo projection=[column1], partial_filters=[foo.column1 LIKE Utf8View("f%")] +01)Filter: foo.column1 LIKE Utf8("f%") +02)--TableScan: foo projection=[column1], partial_filters=[foo.column1 LIKE Utf8("f%")] physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: column1@0 LIKE f% From ed718c015eb4fdf1a06724622adf3d441d42bc67 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 25 Aug 2025 13:07:12 +0800 Subject: [PATCH 24/41] fix page_index_filter_one_col and remove an example of proto --- datafusion/core/tests/parquet/page_pruning.rs | 3 ++- datafusion/proto/src/lib.rs | 23 ------------------- datafusion/sqllogictest/test_files/limit.slt | 2 +- 3 files changed, 3 insertions(+), 25 deletions(-) diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 9da879a32f6b5..fef8fd1d401e5 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -160,7 +160,8 @@ async fn page_index_filter_one_col() { // 5.create filter date_string_col == "01/01/09"`; // Note this test doesn't apply type coercion so the literal must match the actual view type - let filter = col("date_string_col").eq(lit(ScalarValue::new_utf8view("01/01/09"))); + // xudong: use new_utf8, because schema_force_view_types was changed to false now. + let filter = col("date_string_col").eq(lit(ScalarValue::new_utf8("01/01/09"))); let parquet_exec = get_parquet_exec(&state, filter).await; let mut results = parquet_exec.execute(0, task_ctx.clone()).unwrap(); let batch = results.next().await.unwrap().unwrap(); diff --git a/datafusion/proto/src/lib.rs b/datafusion/proto/src/lib.rs index 2df162f21e3a3..f7da0f1f2f741 100644 --- a/datafusion/proto/src/lib.rs +++ b/datafusion/proto/src/lib.rs @@ -97,29 +97,6 @@ //! assert_eq!(format!("{:?}", plan), format!("{:?}", logical_round_trip)); //! # Ok(()) //! # } -//! ``` -//! # Example: Serializing [`ExecutionPlan`]s -//! -//! ``` -//! # use datafusion::prelude::*; -//! # use datafusion_common::Result; -//! # use datafusion_proto::bytes::{physical_plan_from_bytes,physical_plan_to_bytes}; -//! # #[tokio::main] -//! # async fn main() -> Result<()>{ -//! // Create a plan that scans table 't' -//! let ctx = SessionContext::new(); -//! ctx.register_csv("t1", "tests/testdata/test.csv", CsvReadOptions::default()).await?; -//! let physical_plan = ctx.table("t1").await?.create_physical_plan().await?; -//! -//! // Convert the plan into bytes (for sending over the network, etc.) -//! let bytes = physical_plan_to_bytes(physical_plan.clone())?; -//! -//! // Decode bytes from somewhere (over network, etc.) back to ExecutionPlan -//! let physical_round_trip = physical_plan_from_bytes(&bytes, &ctx)?; -//! assert_eq!(format!("{:?}", physical_plan), format!("{:?}", physical_round_trip)); -//! # Ok(()) -//! # } -//! ``` pub mod bytes; pub mod common; pub mod generated; diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 894ba424eddcc..1af14a52e2bc6 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -854,7 +854,7 @@ physical_plan 02)--SortExec: TopK(fetch=1000), expr=[part_key@1 ASC NULLS LAST], preserve_partitioning=[false] 03)----ProjectionExec: expr=[1 as foo, part_key@0 as part_key] 04)------CoalescePartitionsExec: fetch=1 -05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:0..265], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:265..530], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:530..794]]}, projection=[part_key], limit=1, file_type=parquet +05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:0..265], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:265..530], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:530..794]]}, projection=[part_key], limit=1, file_type=parquet query I with selection as ( From 0bb16fa06e20f2552d19b41738d37b8241e148e3 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 25 Aug 2025 13:23:20 +0800 Subject: [PATCH 25/41] fix configs.md --- docs/source/user-guide/configs.md | 109 +++++++++++++++++++++++++++++- 1 file changed, 108 insertions(+), 1 deletion(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 05cc36651a1a8..75184fcd00e75 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -56,7 +56,114 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.metadata_size_hint | NULL | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | | datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | | datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.execution.parquet.schema_force_view_types | true | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| datafusion.execution.parquet.schema_force_view_types | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | +| datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | +| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | +| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | +| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | +| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | +| datafusion.execution.parquet.skip_arrow_metadata | false | (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to | +| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | +| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | +| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.max_statistics_size | 4096 | (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting max_statistics_size is deprecated, currently it is not being used | +| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | +| datafusion.execution.parquet.created_by | datafusion version 48.0.1 | (writing) Sets "created by" property | +| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | +| datafusion.execution.parquet.statistics_truncate_length | NULL | (writing) Sets statictics truncate length. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | +| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | +| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | +| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | +| datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | +| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | +| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | +| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | +| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | +| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | +| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | +| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | +| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | +| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | +| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | +| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | +| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | +| datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for this decision. Setting this flag to `true` will likely produce better plans. if the source of statistics is accurate. We plan to make this the default in the future. | +| datafusion.execution.enforce_batch_size_in_joins | false | Should DataFusion enforce batch size in joins or not. By default, DataFusion will not enforce batch size in joins. Enforcing batch size in joins can reduce memory usage when joining large tables with a highly-selective join filter, but is also slightly slower. | +| datafusion.execution.objectstore_writer_buffer_size | 10485760 | Size (bytes) of data buffer DataFusion uses when writing output files. This affects the size of the data chunks that are uploaded to remote object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being written, it may be necessary to increase this size to avoid errors from the remote end point. | +| datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | +| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | +| datafusion.optimizer.repartition_file_scans | true | When set to `true`, datasource partitions will be repartitioned to achieve maximum parallelism. This applies to both in-memory partitions and FileSource's file groups (1 group is 1 partition). For FileSources, only Parquet and CSV formats are currently supported. If set to `true` for a FileSource, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false` for a FileSource, different files will be read in parallel, but repartitioning won't happen within a single file. If set to `true` for an in-memory source, all memtable's partitions will have their batches repartitioned evenly to the desired number of `target_partitions`. Repartitioning can change the total number of partitions and batches per partition, but does not slice the initial record tables provided to the MemTable on creation. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | +| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | +| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | +| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | +| datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | +| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | +| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | +| datafusion.explain.format | indent | Display format of explain. Default is "indent". When set to "tree", it will print the plan in a tree-rendered format. | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | +| datafusion.sql_parser.enable_options_value_normalization | false | When set to true, SQL parser will normalize options value (convert value to lowercase). Note that this option is ignored and will be removed in the future. All case-insensitive values are normalized automatically. | +| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. | +| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | +| datafusion.sql_parser.map_varchar_to_utf8view | true | If true, `VARCHAR` is mapped to `Utf8View` during SQL planning. If false, `VARCHAR` is mapped to `Utf8` during SQL planning. Default is false. | +| datafusion.sql_parser.collect_spans | false | When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. | +| datafusion.sql_parser.recursion_limit | 50 | Specifies the recursion depth limit when parsing complex SQL Queries | +| datafusion.format.safe | true | If set to `true` any formatting errors will be written to the output instead of being converted into a [`std::fmt::Error`] | +| datafusion.format.null | | Format string for nulls | +| datafusion.format.date_format | %Y-%m-%d | Date format for date arrays | +| datafusion.format.datetime_format | %Y-%m-%dT%H:%M:%S%.f | Format for DateTime arrays | +| datafusion.format.timestamp_format | %Y-%m-%dT%H:%M:%S%.f | Timestamp format for timestamp arrays | +| datafusion.format.timestamp_tz_format | NULL | Timestamp format for timestamp with timezone arrays. When `None`, ISO 8601 format is used. | +| datafusion.format.time_format | %H:%M:%S%.f | Time format for time arrays | +| datafusion.format.duration_format | pretty | Duration format. Can be either `"pretty"` or `"ISO8601"` | +| datafusion.format.types_info | false | Show types in visual representation batches | +| key | default | description | +| ----- | --------- | ------------- | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | +| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | +| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | +| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | +| datafusion.execution.collect_statistics | true | Should DataFusion collect statistics when first creating a table. Has no effect after the table is created. Applies to the default `ListingTableProvider` in DataFusion. Defaults to true. | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | +| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | +| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | +| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | NULL | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | +| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | +| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.execution.parquet.schema_force_view_types | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | | datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | | datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | | datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | From 09ff8f7b91136b55684f302c7b67692fb530e0a9 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 25 Aug 2025 13:41:48 +0800 Subject: [PATCH 26/41] fix clippy --- datafusion/datasource/src/sink.rs | 4 +-- datafusion/datasource/src/source.rs | 6 ++-- datafusion/expr/src/expr.rs | 5 +--- .../optimizer/src/simplify_predicates.rs | 18 ++++++------ .../src/enforce_distribution.rs | 29 +++++++++---------- .../physical-plan/src/aggregates/mod.rs | 4 +-- datafusion/physical-plan/src/analyze.rs | 4 +-- .../physical-plan/src/coalesce_batches.rs | 4 +-- .../physical-plan/src/coalesce_partitions.rs | 4 +-- datafusion/physical-plan/src/empty.rs | 6 ++-- datafusion/physical-plan/src/filter.rs | 4 +-- .../physical-plan/src/joins/cross_join.rs | 4 +-- .../physical-plan/src/joins/hash_join.rs | 4 +-- .../src/joins/sort_merge_join.rs | 4 +-- .../src/joins/symmetric_hash_join.rs | 4 +-- datafusion/physical-plan/src/limit.rs | 4 +-- .../physical-plan/src/placeholder_row.rs | 4 +-- datafusion/physical-plan/src/projection.rs | 4 +-- .../physical-plan/src/recursive_query.rs | 4 +-- .../physical-plan/src/repartition/mod.rs | 4 +-- .../physical-plan/src/sorts/partial_sort.rs | 4 +-- datafusion/physical-plan/src/sorts/sort.rs | 4 +-- .../src/sorts/sort_preserving_merge.rs | 4 +-- datafusion/physical-plan/src/streaming.rs | 4 +-- datafusion/physical-plan/src/union.rs | 4 +-- datafusion/physical-plan/src/unnest.rs | 4 +-- datafusion/physical-plan/src/values.rs | 4 +-- .../src/windows/window_agg_exec.rs | 4 +-- datafusion/physical-plan/src/work_table.rs | 4 +-- 29 files changed, 77 insertions(+), 83 deletions(-) diff --git a/datafusion/datasource/src/sink.rs b/datafusion/datasource/src/sink.rs index faec74b98c5d7..7b84ffe1ff1b8 100644 --- a/datafusion/datasource/src/sink.rs +++ b/datafusion/datasource/src/sink.rs @@ -251,14 +251,14 @@ impl ExecutionPlan for DataSinkExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = DataSinkExec::new( Arc::clone(self.input()), Arc::clone(&self.sink), self.sort_order.clone(), ); - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 9c32f2170071a..521ad9d91573c 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -308,10 +308,10 @@ impl ExecutionPlan for DataSourceExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { - let mut new_plan = DataSourceExec::new(self.data_source.clone()); - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let mut new_plan = DataSourceExec::new(Arc::clone(&self.data_source)); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index ffa2417fc5b66..bbfdf7d870555 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -1885,10 +1885,7 @@ impl Expr { /// Check if the Expr is literal pub fn is_literal(&self) -> bool { - match self { - Expr::Literal(_, _) => true, - _ => false, - } + matches!(self, Expr::Literal(_, _)) } } diff --git a/datafusion/optimizer/src/simplify_predicates.rs b/datafusion/optimizer/src/simplify_predicates.rs index 5ff7e42d95310..198e630203977 100644 --- a/datafusion/optimizer/src/simplify_predicates.rs +++ b/datafusion/optimizer/src/simplify_predicates.rs @@ -31,16 +31,16 @@ pub(crate) fn simplify_predicates(predicates: Vec) -> Result> { for pred in predicates { match &pred { - Expr::BinaryExpr(BinaryExpr { left, op, right }) - if matches!( - op, + Expr::BinaryExpr(BinaryExpr { + left, + op: Operator::Gt - | Operator::GtEq - | Operator::Lt - | Operator::LtEq - | Operator::Eq - ) => - { + | Operator::GtEq + | Operator::Lt + | Operator::LtEq + | Operator::Eq, + right, + }) => { let left_col = extract_column_from_expr(left); let right_col = extract_column_from_expr(right); let left_lit = left.is_literal(); diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 77c7af76ccbef..8f8c7274cf78e 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -951,17 +951,16 @@ fn add_spm_on_top( // (determined by flag `config.optimizer.bounded_order_preserving_variants`) let should_preserve_ordering = input.plan.output_ordering().is_some(); + let ordering = input + .plan + .output_ordering() + .cloned() + .unwrap_or_else(LexOrdering::default); + let new_plan = if should_preserve_ordering { Arc::new( - SortPreservingMergeExec::new( - input - .plan - .output_ordering() - .unwrap_or(&LexOrdering::default()) - .clone(), - Arc::clone(&input.plan), - ) - .with_fetch(fetch.take()), + SortPreservingMergeExec::new(ordering, Arc::clone(&input.plan)) + .with_fetch(fetch.take()), ) as _ } else { Arc::new(CoalescePartitionsExec::new(Arc::clone(&input.plan))) as _ @@ -1405,14 +1404,12 @@ pub fn ensure_distribution( // It was removed by `remove_dist_changing_operators` // and we need to add it back. if fetch.is_some() { + let ordering = plan + .output_ordering() + .cloned() + .unwrap_or_else(LexOrdering::default); let plan = Arc::new( - SortPreservingMergeExec::new( - plan.output_ordering() - .unwrap_or(&LexOrdering::default()) - .clone(), - plan, - ) - .with_fetch(fetch.take()), + SortPreservingMergeExec::new(ordering, plan).with_fetch(fetch.take()), ); optimized_distribution_ctx = DistributionContext::new(plan, data, vec![optimized_distribution_ctx]); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 98901be23b90c..8082527bfd8d3 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1023,7 +1023,7 @@ impl ExecutionPlan for AggregateExec { } fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = AggregateExec { mode: self.mode, @@ -1040,7 +1040,7 @@ impl ExecutionPlan for AggregateExec { metrics: self.metrics.clone(), }; - let new_props: PlanProperties = new_plan.cache.clone().with_node_id(_node_id); + let new_props: PlanProperties = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index 791a5f8cb2d9c..c67430b467b69 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -213,7 +213,7 @@ impl ExecutionPlan for AnalyzeExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = AnalyzeExec::new( self.verbose, @@ -221,7 +221,7 @@ impl ExecutionPlan for AnalyzeExec { Arc::clone(self.input()), Arc::clone(&self.schema), ); - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 088d3a2850bbb..7d57ece88e867 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -228,12 +228,12 @@ impl ExecutionPlan for CoalesceBatchesExec { } fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = CoalesceBatchesExec::new(Arc::clone(self.input()), self.target_batch_size) .with_fetch(self.fetch()); - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 7daf0d753d15f..fcff76d54b1a2 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -214,11 +214,11 @@ impl ExecutionPlan for CoalescePartitionsExec { } fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = CoalescePartitionsExec::new(Arc::clone(self.input())); new_plan.fetch = self.fetch; - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index eae1aaff1c492..b005e5e1ef9bc 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -176,10 +176,10 @@ impl ExecutionPlan for EmptyExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { - let mut new_plan = EmptyExec::new(self.schema.clone()); - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let mut new_plan = EmptyExec::new(Arc::clone(&self.schema)); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 07aea16d06283..60b48d542efd1 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -431,12 +431,12 @@ impl ExecutionPlan for FilterExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = FilterExec::try_new(Arc::clone(&self.predicate), Arc::clone(self.input()))? .with_projection(self.projection.clone())?; - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 3ceea2e8833b3..3159ef1d3699f 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -350,11 +350,11 @@ impl ExecutionPlan for CrossJoinExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = CrossJoinExec::new(Arc::clone(&self.left), Arc::clone(&self.right)); - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index f018de2e0a6d8..c868bbb6ef1bf 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -906,7 +906,7 @@ impl ExecutionPlan for HashJoinExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = HashJoinExec::try_new( Arc::clone(&self.left), @@ -918,7 +918,7 @@ impl ExecutionPlan for HashJoinExec { *self.partition_mode(), self.null_equals_null, )?; - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 34915a96ba213..d1f22ad7542bf 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -528,7 +528,7 @@ impl ExecutionPlan for SortMergeJoinExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = SortMergeJoinExec::try_new( Arc::clone(&self.left), @@ -539,7 +539,7 @@ impl ExecutionPlan for SortMergeJoinExec { self.sort_options.clone(), self.null_equals_null, )?; - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 82b099b22999f..21926d099996a 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -478,7 +478,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = SymmetricHashJoinExec::try_new( Arc::clone(&self.left), @@ -491,7 +491,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { self.right_sort_exprs.clone(), self.mode, )?; - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 34aca9a74a951..c1410677330a5 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -212,11 +212,11 @@ impl ExecutionPlan for GlobalLimitExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = GlobalLimitExec::new(Arc::clone(self.input()), self.skip, self.fetch); - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index 98acadb0bce60..45f4d5e1437c4 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -185,10 +185,10 @@ impl ExecutionPlan for PlaceholderRowExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = PlaceholderRowExec::new(Arc::clone(&self.schema)); - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 7149bf031683d..a917d5547cc8f 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -265,11 +265,11 @@ impl ExecutionPlan for ProjectionExec { } fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = ProjectionExec::try_new(self.expr.clone(), Arc::clone(self.input()))?; - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index 9dde7dc91b6e6..516863dbe78d5 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -209,7 +209,7 @@ impl ExecutionPlan for RecursiveQueryExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = RecursiveQueryExec::try_new( self.name.clone(), @@ -217,7 +217,7 @@ impl ExecutionPlan for RecursiveQueryExec { Arc::clone(&self.recursive_term), self.is_distinct, )?; - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 1102c9897e193..84a11839d0286 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -766,7 +766,7 @@ impl ExecutionPlan for RepartitionExec { } fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = RepartitionExec { input: Arc::clone(&self.input), @@ -775,7 +775,7 @@ impl ExecutionPlan for RepartitionExec { preserve_order: self.preserve_order, cache: self.cache.clone(), }; - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 0cb6180667c0f..60b27fac3fa37 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -327,7 +327,7 @@ impl ExecutionPlan for PartialSortExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = PartialSortExec { expr: self.expr.clone(), @@ -338,7 +338,7 @@ impl ExecutionPlan for PartialSortExec { fetch: self.fetch, cache: self.cache.clone(), }; - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 5df18c1526ae1..9eef517aa6d6a 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1208,7 +1208,7 @@ impl ExecutionPlan for SortExec { } fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let new_plan = SortExec { input: Arc::clone(self.input()), @@ -1216,7 +1216,7 @@ impl ExecutionPlan for SortExec { fetch: self.fetch, metrics_set: self.metrics_set.clone(), preserve_partitioning: self.preserve_partitioning, - cache: self.cache.clone().with_node_id(_node_id), + cache: self.cache.clone().with_node_id(node_id), common_sort_prefix: self.common_sort_prefix.clone(), }; Ok(Some(Arc::new(new_plan))) diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 7b2af16641796..519acbafe9846 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -352,12 +352,12 @@ impl ExecutionPlan for SortPreservingMergeExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = SortPreservingMergeExec::new(self.expr.clone(), Arc::clone(self.input())) .with_fetch(self.fetch()); - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 0dead076d5eec..7a614ccb34984 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -352,7 +352,7 @@ impl ExecutionPlan for StreamingTableExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = StreamingTableExec { partitions: self.partitions.clone(), @@ -364,7 +364,7 @@ impl ExecutionPlan for StreamingTableExec { cache: self.cache.clone(), metrics: self.metrics.clone(), }; - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 35a06c8d25381..591e28e4c3220 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -300,10 +300,10 @@ impl ExecutionPlan for UnionExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = UnionExec::new(self.inputs.clone()); - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 548ab4f10f874..460b52d925d56 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -202,7 +202,7 @@ impl ExecutionPlan for UnnestExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = UnnestExec::new( Arc::clone(self.input()), @@ -211,7 +211,7 @@ impl ExecutionPlan for UnnestExec { Arc::clone(&self.schema), self.options.clone(), ); - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index 388b6110e4d8c..e5dc3b1162338 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -231,13 +231,13 @@ impl ExecutionPlan for ValuesExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = ValuesExec::try_new_from_batches( Arc::clone(&self.schema), self.data.clone(), )?; - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index dd6c510b968d9..f5a14e1344deb 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -302,14 +302,14 @@ impl ExecutionPlan for WindowAggExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = WindowAggExec::try_new( self.window_expr.clone(), Arc::clone(self.input()), self.can_repartition, )?; - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index f674ee0b2064f..ffec2f82a2a89 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -230,11 +230,11 @@ impl ExecutionPlan for WorkTableExec { fn with_node_id( self: Arc, - _node_id: usize, + node_id: usize, ) -> Result>> { let mut new_plan = WorkTableExec::new(self.name.clone(), Arc::clone(&self.schema)); - let new_props = new_plan.cache.clone().with_node_id(_node_id); + let new_props = new_plan.cache.clone().with_node_id(node_id); new_plan.cache = new_props; Ok(Some(Arc::new(new_plan))) } From 1545f2dd47b2d05bdc83a67dab132314e8a9bdd0 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 25 Aug 2025 13:45:39 +0800 Subject: [PATCH 27/41] update configs.md --- docs/source/user-guide/configs.md | 107 ------------------------------ 1 file changed, 107 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 75184fcd00e75..2727d52679018 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -142,110 +142,3 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.format.time_format | %H:%M:%S%.f | Time format for time arrays | | datafusion.format.duration_format | pretty | Duration format. Can be either `"pretty"` or `"ISO8601"` | | datafusion.format.types_info | false | Show types in visual representation batches | -| key | default | description | -| ----- | --------- | ------------- | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | -| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | -| datafusion.execution.batch_size | 8192 | Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption | -| datafusion.execution.coalesce_batches | true | When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting | -| datafusion.execution.collect_statistics | true | Should DataFusion collect statistics when first creating a table. Has no effect after the table is created. Applies to the default `ListingTableProvider` in DataFusion. Defaults to true. | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | -| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | -| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | -| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | NULL | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | -| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | -| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.execution.parquet.schema_force_view_types | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | -| datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | -| datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | -| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | -| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | -| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | -| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | -| datafusion.execution.parquet.skip_arrow_metadata | false | (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to | -| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | -| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | -| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.max_statistics_size | 4096 | (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting max_statistics_size is deprecated, currently it is not being used | -| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | -| datafusion.execution.parquet.created_by | datafusion version 48.0.1 | (writing) Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | -| datafusion.execution.parquet.statistics_truncate_length | NULL | (writing) Sets statictics truncate length. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | -| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | -| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | -| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | -| datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | -| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | -| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | -| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | -| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | -| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | -| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | -| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | -| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | -| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | -| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | -| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | -| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | -| datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for this decision. Setting this flag to `true` will likely produce better plans. if the source of statistics is accurate. We plan to make this the default in the future. | -| datafusion.execution.enforce_batch_size_in_joins | false | Should DataFusion enforce batch size in joins or not. By default, DataFusion will not enforce batch size in joins. Enforcing batch size in joins can reduce memory usage when joining large tables with a highly-selective join filter, but is also slightly slower. | -| datafusion.execution.objectstore_writer_buffer_size | 10485760 | Size (bytes) of data buffer DataFusion uses when writing output files. This affects the size of the data chunks that are uploaded to remote object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being written, it may be necessary to increase this size to avoid errors from the remote end point. | -| datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | -| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | -| datafusion.optimizer.repartition_file_scans | true | When set to `true`, datasource partitions will be repartitioned to achieve maximum parallelism. This applies to both in-memory partitions and FileSource's file groups (1 group is 1 partition). For FileSources, only Parquet and CSV formats are currently supported. If set to `true` for a FileSource, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false` for a FileSource, different files will be read in parallel, but repartitioning won't happen within a single file. If set to `true` for an in-memory source, all memtable's partitions will have their batches repartitioned evenly to the desired number of `target_partitions`. Repartitioning can change the total number of partitions and batches per partition, but does not slice the initial record tables provided to the MemTable on creation. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | -| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | -| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | -| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | -| datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | -| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | -| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | -| datafusion.explain.format | indent | Display format of explain. Default is "indent". When set to "tree", it will print the plan in a tree-rendered format. | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | -| datafusion.sql_parser.enable_options_value_normalization | false | When set to true, SQL parser will normalize options value (convert value to lowercase). Note that this option is ignored and will be removed in the future. All case-insensitive values are normalized automatically. | -| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. | -| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | -| datafusion.sql_parser.map_varchar_to_utf8view | true | If true, `VARCHAR` is mapped to `Utf8View` during SQL planning. If false, `VARCHAR` is mapped to `Utf8` during SQL planning. Default is false. | -| datafusion.sql_parser.collect_spans | false | When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. | -| datafusion.sql_parser.recursion_limit | 50 | Specifies the recursion depth limit when parsing complex SQL Queries | -| datafusion.format.safe | true | If set to `true` any formatting errors will be written to the output instead of being converted into a [`std::fmt::Error`] | -| datafusion.format.null | | Format string for nulls | -| datafusion.format.date_format | %Y-%m-%d | Date format for date arrays | -| datafusion.format.datetime_format | %Y-%m-%dT%H:%M:%S%.f | Format for DateTime arrays | -| datafusion.format.timestamp_format | %Y-%m-%dT%H:%M:%S%.f | Timestamp format for timestamp arrays | -| datafusion.format.timestamp_tz_format | NULL | Timestamp format for timestamp with timezone arrays. When `None`, ISO 8601 format is used. | -| datafusion.format.time_format | %H:%M:%S%.f | Time format for time arrays | -| datafusion.format.duration_format | pretty | Duration format. Can be either `"pretty"` or `"ISO8601"` | -| datafusion.format.types_info | false | Show types in visual representation batches | From ca5b0fb74a66e71f358572c7cdf77e5e931f4f0b Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 25 Aug 2025 14:20:40 +0800 Subject: [PATCH 28/41] fix flaky test limit.test --- datafusion/sqllogictest/test_files/limit.slt | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 1af14a52e2bc6..2e09af4a20eea 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -734,7 +734,7 @@ explain select * from testSubQueryLimit as t1 join (select * from testSubQueryLi ---- logical_plan 01)Limit: skip=0, fetch=10 -02)--Cross Join: +02)--Cross Join: 03)----SubqueryAlias: t1 04)------Limit: skip=0, fetch=10 05)--------TableScan: testsubquerylimit projection=[a, b], fetch=10 @@ -759,7 +759,7 @@ explain select * from testSubQueryLimit as t1 join (select * from testSubQueryLi ---- logical_plan 01)Limit: skip=0, fetch=2 -02)--Cross Join: +02)--Cross Join: 03)----SubqueryAlias: t1 04)------Limit: skip=0, fetch=2 05)--------TableScan: testsubquerylimit projection=[a, b], fetch=2 @@ -830,6 +830,9 @@ CREATE EXTERNAL TABLE test_limit_with_partitions STORED AS PARQUET LOCATION 'test_files/scratch/parquet/test_limit_with_partitions/'; +statement ok +set datafusion.explain.logical_plan_only = true; + query TT explain with selection as ( @@ -849,12 +852,9 @@ logical_plan 04)------SubqueryAlias: selection 05)--------Limit: skip=0, fetch=1 06)----------TableScan: test_limit_with_partitions projection=[part_key], fetch=1 -physical_plan -01)ProjectionExec: expr=[foo@0 as foo] -02)--SortExec: TopK(fetch=1000), expr=[part_key@1 ASC NULLS LAST], preserve_partitioning=[false] -03)----ProjectionExec: expr=[1 as foo, part_key@0 as part_key] -04)------CoalescePartitionsExec: fetch=1 -05)--------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:0..265], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:265..530], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:530..794]]}, projection=[part_key], limit=1, file_type=parquet + +statement ok +set datafusion.explain.logical_plan_only = false; query I with selection as ( From d8c3e03f58dbe20d2b0c35523840dcec3ab29719 Mon Sep 17 00:00:00 2001 From: "xudong.w" Date: Wed, 25 Jun 2025 09:51:04 +0800 Subject: [PATCH 29/41] Simplify predicates in `PushDownFilter` optimizer rule (#16362) * Simplify predicates in filter * add slt test * Use BtreeMap to make tests stable * process edge coner * add doc for simplify_predicates.rs * add as_literal to make code neat * reorgnize file * reduce clone call --- Cargo.lock | 12 +- datafusion/expr/src/expr.rs | 10 +- datafusion/optimizer/src/lib.rs | 1 - datafusion/optimizer/src/push_down_filter.rs | 6 +- .../optimizer/src/simplify_expressions/mod.rs | 2 + .../simplify_predicates.rs | 247 ++++++++++++++++++ .../optimizer/src/simplify_predicates.rs | 194 -------------- .../test_files/simplify_predicates.slt | 30 ++- 8 files changed, 292 insertions(+), 210 deletions(-) create mode 100644 datafusion/optimizer/src/simplify_expressions/simplify_predicates.rs delete mode 100644 datafusion/optimizer/src/simplify_predicates.rs diff --git a/Cargo.lock b/Cargo.lock index e7fb4ef136c7a..41d372f71723b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3959,9 +3959,9 @@ checksum = "f9fbbcab51052fe104eb5e5d351cf728d30a5be1fe14d9be8a3b097481fb97de" [[package]] name = "libmimalloc-sys" -version = "0.1.43" +version = "0.1.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf88cd67e9de251c1781dbe2f641a1a3ad66eaae831b8a2c38fbdc5ddae16d4d" +checksum = "ec9d6fac27761dabcd4ee73571cdb06b7022dc99089acbe5435691edffaac0f4" dependencies = [ "cc", "libc", @@ -4092,9 +4092,9 @@ dependencies = [ [[package]] name = "mimalloc" -version = "0.1.47" +version = "0.1.46" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1791cbe101e95af5764f06f20f6760521f7158f69dbf9d6baf941ee1bf6bc40" +checksum = "995942f432bbb4822a7e9c3faa87a695185b0d09273ba85f097b54f4e458f2af" dependencies = [ "libmimalloc-sys", ] @@ -4320,9 +4320,9 @@ dependencies = [ [[package]] name = "object_store" -version = "0.12.2" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7781f96d79ed0f961a7021424ab01840efbda64ae7a505aaea195efc91eaaec4" +checksum = "d94ac16b433c0ccf75326388c893d2835ab7457ea35ab8ba5d745c053ef5fa16" dependencies = [ "async-trait", "base64 0.22.1", diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index bbfdf7d870555..25b2ddfc00474 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -1883,9 +1883,13 @@ impl Expr { } } - /// Check if the Expr is literal - pub fn is_literal(&self) -> bool { - matches!(self, Expr::Literal(_, _)) + /// Check if the Expr is literal and get the literal value if it is. + pub fn as_literal(&self) -> Option<&ScalarValue> { + if let Expr::Literal(lit, _) = self { + Some(lit) + } else { + None + } } } diff --git a/datafusion/optimizer/src/lib.rs b/datafusion/optimizer/src/lib.rs index ea14cf114030a..280010e3d92c0 100644 --- a/datafusion/optimizer/src/lib.rs +++ b/datafusion/optimizer/src/lib.rs @@ -61,7 +61,6 @@ pub mod push_down_limit; pub mod replace_distinct_aggregate; pub mod scalar_subquery_to_join; pub mod simplify_expressions; -mod simplify_predicates; pub mod single_distinct_to_groupby; pub mod utils; diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index f701fcf861e0d..b1aa0c09249ff 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -41,7 +41,7 @@ use datafusion_expr::{ use crate::optimizer::ApplyOrder; use crate::utils::{has_all_column_refs, is_restrict_null_predicate}; -use crate::{simplify_predicates::simplify_predicates, OptimizerConfig, OptimizerRule}; +use crate::{simplify_expressions::simplify_predicates, OptimizerConfig, OptimizerRule}; /// Optimizer rule for pushing (moving) filter expressions down in a plan so /// they are applied as early as possible. @@ -783,7 +783,9 @@ impl OptimizerRule for PushDownFilter { let new_predicates = simplify_predicates(predicate)?; if old_predicate_len != new_predicates.len() { let Some(new_predicate) = conjunction(new_predicates) else { - return plan_err!("at least one expression exists"); + // new_predicates is empty - remove the filter entirely + // Return the child plan without the filter + return Ok(Transformed::yes(Arc::unwrap_or_clone(filter.input))); }; filter.predicate = new_predicate; } diff --git a/datafusion/optimizer/src/simplify_expressions/mod.rs b/datafusion/optimizer/src/simplify_expressions/mod.rs index 5fbee02e3909e..7ae38eec9a3ad 100644 --- a/datafusion/optimizer/src/simplify_expressions/mod.rs +++ b/datafusion/optimizer/src/simplify_expressions/mod.rs @@ -23,6 +23,7 @@ mod guarantees; mod inlist_simplifier; mod regex; pub mod simplify_exprs; +mod simplify_predicates; mod unwrap_cast; mod utils; @@ -31,6 +32,7 @@ pub use datafusion_expr::simplify::{SimplifyContext, SimplifyInfo}; pub use expr_simplifier::*; pub use simplify_exprs::*; +pub use simplify_predicates::simplify_predicates; // Export for test in datafusion/core/tests/optimizer_integration.rs pub use guarantees::GuaranteeRewriter; diff --git a/datafusion/optimizer/src/simplify_expressions/simplify_predicates.rs b/datafusion/optimizer/src/simplify_expressions/simplify_predicates.rs new file mode 100644 index 0000000000000..32b2315e15d58 --- /dev/null +++ b/datafusion/optimizer/src/simplify_expressions/simplify_predicates.rs @@ -0,0 +1,247 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Simplifies predicates by reducing redundant or overlapping conditions. +//! +//! This module provides functionality to optimize logical predicates used in query planning +//! by eliminating redundant conditions, thus reducing the number of predicates to evaluate. +//! Unlike the simplifier in `simplify_expressions/simplify_exprs.rs`, which focuses on +//! general expression simplification (e.g., constant folding and algebraic simplifications), +//! this module specifically targets predicate optimization by handling containment relationships. +//! For example, it can simplify `x > 5 AND x > 6` to just `x > 6`, as the latter condition +//! encompasses the former, resulting in fewer checks during query execution. + +use datafusion_common::{Column, Result, ScalarValue}; +use datafusion_expr::{BinaryExpr, Cast, Expr, Operator}; +use std::collections::BTreeMap; + +/// Simplifies a list of predicates by removing redundancies. +/// +/// This function takes a vector of predicate expressions and groups them by the column they reference. +/// Predicates that reference a single column and are comparison operations (e.g., >, >=, <, <=, =) +/// are analyzed to remove redundant conditions. For instance, `x > 5 AND x > 6` is simplified to +/// `x > 6`. Other predicates that do not fit this pattern are retained as-is. +/// +/// # Arguments +/// * `predicates` - A vector of `Expr` representing the predicates to simplify. +/// +/// # Returns +/// A `Result` containing a vector of simplified `Expr` predicates. +pub fn simplify_predicates(predicates: Vec) -> Result> { + // Early return for simple cases + if predicates.len() <= 1 { + return Ok(predicates); + } + + // Group predicates by their column reference + let mut column_predicates: BTreeMap> = BTreeMap::new(); + let mut other_predicates = Vec::new(); + + for pred in predicates { + match &pred { + Expr::BinaryExpr(BinaryExpr { + left, + op: + Operator::Gt + | Operator::GtEq + | Operator::Lt + | Operator::LtEq + | Operator::Eq, + right, + }) => { + let left_col = extract_column_from_expr(left); + let right_col = extract_column_from_expr(right); + if let (Some(col), Some(_)) = (&left_col, right.as_literal()) { + column_predicates.entry(col.clone()).or_default().push(pred); + } else if let (Some(_), Some(col)) = (left.as_literal(), &right_col) { + column_predicates.entry(col.clone()).or_default().push(pred); + } else { + other_predicates.push(pred); + } + } + _ => other_predicates.push(pred), + } + } + + // Process each column's predicates to remove redundancies + let mut result = other_predicates; + for (_, preds) in column_predicates { + let simplified = simplify_column_predicates(preds)?; + result.extend(simplified); + } + + Ok(result) +} + +/// Simplifies predicates related to a single column. +/// +/// This function processes a list of predicates that all reference the same column and +/// simplifies them based on their operators. It groups predicates into greater-than (>, >=), +/// less-than (<, <=), and equality (=) categories, then selects the most restrictive condition +/// in each category to reduce redundancy. For example, among `x > 5` and `x > 6`, only `x > 6` +/// is retained as it is more restrictive. +/// +/// # Arguments +/// * `predicates` - A vector of `Expr` representing predicates for a single column. +/// +/// # Returns +/// A `Result` containing a vector of simplified `Expr` predicates for the column. +fn simplify_column_predicates(predicates: Vec) -> Result> { + if predicates.len() <= 1 { + return Ok(predicates); + } + + // Group by operator type, but combining similar operators + let mut greater_predicates = Vec::new(); // Combines > and >= + let mut less_predicates = Vec::new(); // Combines < and <= + let mut eq_predicates = Vec::new(); + + for pred in predicates { + match &pred { + Expr::BinaryExpr(BinaryExpr { left: _, op, right }) => { + match (op, right.as_literal().is_some()) { + (Operator::Gt, true) + | (Operator::Lt, false) + | (Operator::GtEq, true) + | (Operator::LtEq, false) => greater_predicates.push(pred), + (Operator::Lt, true) + | (Operator::Gt, false) + | (Operator::LtEq, true) + | (Operator::GtEq, false) => less_predicates.push(pred), + (Operator::Eq, _) => eq_predicates.push(pred), + _ => unreachable!("Unexpected operator: {}", op), + } + } + _ => unreachable!("Unexpected predicate {}", pred.to_string()), + } + } + + let mut result = Vec::new(); + + if !eq_predicates.is_empty() { + // If there are many equality predicates, we can only keep one if they are all the same + if eq_predicates.len() == 1 + || eq_predicates.iter().all(|e| e == &eq_predicates[0]) + { + result.push(eq_predicates.pop().unwrap()); + } else { + // If they are not the same, add a false predicate + result.push(Expr::Literal(ScalarValue::Boolean(Some(false)), None)); + } + } + + // Handle all greater-than-style predicates (keep the most restrictive - highest value) + if !greater_predicates.is_empty() { + if let Some(most_restrictive) = + find_most_restrictive_predicate(&greater_predicates, true)? + { + result.push(most_restrictive); + } else { + result.extend(greater_predicates); + } + } + + // Handle all less-than-style predicates (keep the most restrictive - lowest value) + if !less_predicates.is_empty() { + if let Some(most_restrictive) = + find_most_restrictive_predicate(&less_predicates, false)? + { + result.push(most_restrictive); + } else { + result.extend(less_predicates); + } + } + + Ok(result) +} + +/// Finds the most restrictive predicate from a list based on literal values. +/// +/// This function iterates through a list of predicates to identify the most restrictive one +/// by comparing their literal values. For greater-than predicates, the highest value is most +/// restrictive, while for less-than predicates, the lowest value is most restrictive. +/// +/// # Arguments +/// * `predicates` - A slice of `Expr` representing predicates to compare. +/// * `find_greater` - A boolean indicating whether to find the highest value (true for >, >=) +/// or the lowest value (false for <, <=). +/// +/// # Returns +/// A `Result` containing an `Option` with the most restrictive predicate, if any. +fn find_most_restrictive_predicate( + predicates: &[Expr], + find_greater: bool, +) -> Result> { + if predicates.is_empty() { + return Ok(None); + } + + let mut most_restrictive_idx = 0; + let mut best_value: Option<&ScalarValue> = None; + + for (idx, pred) in predicates.iter().enumerate() { + if let Expr::BinaryExpr(BinaryExpr { left, op: _, right }) = pred { + // Extract the literal value based on which side has it + let scalar_value = match (right.as_literal(), left.as_literal()) { + (Some(scalar), _) => Some(scalar), + (_, Some(scalar)) => Some(scalar), + _ => None, + }; + + if let Some(scalar) = scalar_value { + if let Some(current_best) = best_value { + if let Some(comparison) = scalar.partial_cmp(current_best) { + let is_better = if find_greater { + comparison == std::cmp::Ordering::Greater + } else { + comparison == std::cmp::Ordering::Less + }; + + if is_better { + best_value = Some(scalar); + most_restrictive_idx = idx; + } + } + } else { + best_value = Some(scalar); + most_restrictive_idx = idx; + } + } + } + } + + Ok(Some(predicates[most_restrictive_idx].clone())) +} + +/// Extracts a column reference from an expression, if present. +/// +/// This function checks if the given expression is a column reference or contains one, +/// such as within a cast operation. It returns the `Column` if found. +/// +/// # Arguments +/// * `expr` - A reference to an `Expr` to inspect for a column reference. +/// +/// # Returns +/// An `Option` containing the column reference if found, otherwise `None`. +fn extract_column_from_expr(expr: &Expr) -> Option { + match expr { + Expr::Column(col) => Some(col.clone()), + // Handle cases where the column might be wrapped in a cast or other operation + Expr::Cast(Cast { expr, .. }) => extract_column_from_expr(expr), + _ => None, + } +} diff --git a/datafusion/optimizer/src/simplify_predicates.rs b/datafusion/optimizer/src/simplify_predicates.rs deleted file mode 100644 index 198e630203977..0000000000000 --- a/datafusion/optimizer/src/simplify_predicates.rs +++ /dev/null @@ -1,194 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use datafusion_common::{Column, Result, ScalarValue}; -use datafusion_expr::{BinaryExpr, Cast, Expr, Operator}; -use std::collections::BTreeMap; - -pub(crate) fn simplify_predicates(predicates: Vec) -> Result> { - // Early return for simple cases - if predicates.len() <= 1 { - return Ok(predicates); - } - - // Group predicates by their column reference - let mut column_predicates: BTreeMap> = BTreeMap::new(); - let mut other_predicates = Vec::new(); - - for pred in predicates { - match &pred { - Expr::BinaryExpr(BinaryExpr { - left, - op: - Operator::Gt - | Operator::GtEq - | Operator::Lt - | Operator::LtEq - | Operator::Eq, - right, - }) => { - let left_col = extract_column_from_expr(left); - let right_col = extract_column_from_expr(right); - let left_lit = left.is_literal(); - let right_lit = right.is_literal(); - if let (Some(col), true) = (&left_col, right_lit) { - column_predicates.entry(col.clone()).or_default().push(pred); - } else if let (true, Some(col)) = (left_lit, &right_col) { - column_predicates.entry(col.clone()).or_default().push(pred); - } else { - other_predicates.push(pred); - } - } - _ => other_predicates.push(pred), - } - } - - // Process each column's predicates to remove redundancies - let mut result = other_predicates; - for (_, preds) in column_predicates { - let simplified = simplify_column_predicates(preds)?; - result.extend(simplified); - } - - Ok(result) -} - -fn simplify_column_predicates(predicates: Vec) -> Result> { - if predicates.len() <= 1 { - return Ok(predicates); - } - - // Group by operator type, but combining similar operators - let mut greater_predicates = Vec::new(); // Combines > and >= - let mut less_predicates = Vec::new(); // Combines < and <= - let mut eq_predicates = Vec::new(); - - for pred in predicates { - match &pred { - Expr::BinaryExpr(BinaryExpr { left: _, op, right }) => { - let right_is_literal = right.is_literal(); - match (op, right_is_literal) { - (Operator::Gt, true) - | (Operator::Lt, false) - | (Operator::GtEq, true) - | (Operator::LtEq, false) => greater_predicates.push(pred), - (Operator::Lt, true) - | (Operator::Gt, false) - | (Operator::LtEq, true) - | (Operator::GtEq, false) => less_predicates.push(pred), - (Operator::Eq, _) => eq_predicates.push(pred), - _ => unreachable!("Unexpected operator: {}", op), - } - } - _ => unreachable!("Unexpected predicate {}", pred.to_string()), - } - } - - let mut result = Vec::new(); - - // If we have equality predicates, they're the most restrictive - if !eq_predicates.is_empty() { - if eq_predicates.len() > 1 { - result.push(Expr::Literal(ScalarValue::Boolean(Some(false)), None)); - } else { - result.push(eq_predicates[0].clone()); - } - } else { - // Handle all greater-than-style predicates (keep the most restrictive - highest value) - if !greater_predicates.is_empty() { - if let Some(most_restrictive) = - find_most_restrictive_predicate(&greater_predicates, true)? - { - result.push(most_restrictive); - } else { - result.extend(greater_predicates); - } - } - - // Handle all less-than-style predicates (keep the most restrictive - lowest value) - if !less_predicates.is_empty() { - if let Some(most_restrictive) = - find_most_restrictive_predicate(&less_predicates, false)? - { - result.push(most_restrictive); - } else { - result.extend(less_predicates); - } - } - } - - Ok(result) -} - -fn find_most_restrictive_predicate( - predicates: &[Expr], - find_greater: bool, -) -> Result> { - if predicates.is_empty() { - return Ok(None); - } - - let mut most_restrictive = predicates[0].clone(); - let mut best_value: Option = None; - - for pred in predicates { - if let Expr::BinaryExpr(BinaryExpr { left, op: _, right }) = pred { - // Extract the literal value based on which side has it - let mut scalar_value = None; - if right.is_literal() { - if let Expr::Literal(scalar, _) = right.as_ref() { - scalar_value = Some(scalar.clone()); - } - } else if left.is_literal() { - if let Expr::Literal(scalar, _) = left.as_ref() { - scalar_value = Some(scalar.clone()); - } - } - - if let Some(scalar) = scalar_value { - if let Some(current_best) = &best_value { - if let Some(comparison) = scalar.partial_cmp(current_best) { - let is_better = if find_greater { - comparison == std::cmp::Ordering::Greater - } else { - comparison == std::cmp::Ordering::Less - }; - - if is_better { - best_value = Some(scalar); - most_restrictive = pred.clone(); - } - } - } else { - best_value = Some(scalar); - most_restrictive = pred.clone(); - } - } - } - } - - Ok(Some(most_restrictive)) -} - -fn extract_column_from_expr(expr: &Expr) -> Option { - match expr { - Expr::Column(col) => Some(col.clone()), - // Handle cases where the column might be wrapped in a cast or other operation - Expr::Cast(Cast { expr, .. }) => extract_column_from_expr(expr), - _ => None, - } -} diff --git a/datafusion/sqllogictest/test_files/simplify_predicates.slt b/datafusion/sqllogictest/test_files/simplify_predicates.slt index cef78d97bb46c..0dd551d96d0ce 100644 --- a/datafusion/sqllogictest/test_files/simplify_predicates.slt +++ b/datafusion/sqllogictest/test_files/simplify_predicates.slt @@ -70,13 +70,35 @@ logical_plan 01)Filter: test_data.float_col < Float32(8) AND test_data.int_col > Int32(6) 02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] +# x = 7 AND x = 7 should simplify to x = 7 +query TT +EXPLAIN SELECT * FROM test_data WHERE int_col = 7 AND int_col = 7; +---- +logical_plan +01)Filter: test_data.int_col = Int32(7) +02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] + +# x = 7 AND x = 6 should simplify to false +query TT +EXPLAIN SELECT * FROM test_data WHERE int_col = 7 AND int_col = 6; +---- +logical_plan EmptyRelation + +# TODO: x = 7 AND x < 2 should simplify to false +query TT +EXPLAIN SELECT * FROM test_data WHERE int_col = 7 AND int_col < 2; +---- +logical_plan +01)Filter: test_data.int_col = Int32(7) AND test_data.int_col < Int32(2) +02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] -# x = 7 AND x > 5 should simplify to x = 7 + +# TODO: x = 7 AND x > 5 should simplify to x = 7 query TT EXPLAIN SELECT * FROM test_data WHERE int_col = 7 AND int_col > 5; ---- logical_plan -01)Filter: test_data.int_col = Int32(7) +01)Filter: test_data.int_col = Int32(7) AND test_data.int_col > Int32(5) 02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] # str_col > 'apple' AND str_col > 'banana' should simplify to str_col > 'banana' @@ -148,7 +170,7 @@ logical_plan 07)------Filter: test_data2.value < Int32(50) AND test_data2.id > Int32(10) 08)--------TableScan: test_data2 projection=[id, value] -# Case 13: Handling negated predicates +# Handling negated predicates # NOT (x < 10) AND NOT (x < 5) should simplify to NOT (x < 10) query TT EXPLAIN SELECT * FROM test_data WHERE NOT (int_col < 10) AND NOT (int_col < 5); @@ -198,7 +220,7 @@ logical_plan 01)Filter: (test_data.int_col > Int32(5) OR test_data.float_col < Float32(10)) AND (test_data.int_col > Int32(6) OR test_data.float_col < Float32(8)) 02)--TableScan: test_data projection=[int_col, float_col, str_col, date_col, bool_col] -# Case 20: Combination of AND and OR with simplifiable predicates +# Combination of AND and OR with simplifiable predicates query TT EXPLAIN SELECT * FROM test_data WHERE (int_col > 5 AND int_col > 6) From 209988285b9def1040e20d61e74cb04e070ffaa1 Mon Sep 17 00:00:00 2001 From: kosiew Date: Sat, 7 Jun 2025 02:03:09 +0800 Subject: [PATCH 30/41] Fix intermittent SQL logic test failure in limit.slt by adding ORDER BY clause (#16257) * Add order by clause to limit query for consistent results * test: update explain plan --- datafusion/sqllogictest/test_files/limit.slt | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 2e09af4a20eea..6f13570773555 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -830,14 +830,12 @@ CREATE EXTERNAL TABLE test_limit_with_partitions STORED AS PARQUET LOCATION 'test_files/scratch/parquet/test_limit_with_partitions/'; -statement ok -set datafusion.explain.logical_plan_only = true; - query TT explain with selection as ( select * from test_limit_with_partitions + order by part_key limit 1 ) select 1 as foo @@ -850,16 +848,19 @@ logical_plan 02)--Sort: selection.part_key ASC NULLS LAST, fetch=1000 03)----Projection: Int64(1) AS foo, selection.part_key 04)------SubqueryAlias: selection -05)--------Limit: skip=0, fetch=1 -06)----------TableScan: test_limit_with_partitions projection=[part_key], fetch=1 - -statement ok -set datafusion.explain.logical_plan_only = false; +05)--------Sort: test_limit_with_partitions.part_key ASC NULLS LAST, fetch=1 +06)----------TableScan: test_limit_with_partitions projection=[part_key] +physical_plan +01)ProjectionExec: expr=[1 as foo] +02)--SortPreservingMergeExec: [part_key@0 ASC NULLS LAST], fetch=1 +03)----SortExec: TopK(fetch=1), expr=[part_key@0 ASC NULLS LAST], preserve_partitioning=[true] +04)------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-0.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:0..794]]}, projection=[part_key], file_type=parquet query I with selection as ( select * from test_limit_with_partitions + order by part_key limit 1 ) select 1 as foo From ff8418c8ecbf74a02a709c8ca2ca23b72cb9a843 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 25 Aug 2025 15:11:54 +0800 Subject: [PATCH 31/41] fix limit.rs --- datafusion/sqllogictest/test_files/limit.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 6f13570773555..a6bafd703de85 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -854,7 +854,7 @@ physical_plan 01)ProjectionExec: expr=[1 as foo] 02)--SortPreservingMergeExec: [part_key@0 ASC NULLS LAST], fetch=1 03)----SortExec: TopK(fetch=1), expr=[part_key@0 ASC NULLS LAST], preserve_partitioning=[true] -04)------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-0.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet:0..794], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet:0..794]]}, projection=[part_key], file_type=parquet +04)------DataSourceExec: file_groups={3 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_limit_with_partitions/part-2.parquet]]}, projection=[part_key], file_type=parquet query I with selection as ( From 2c7836ad071996dc3fbd6b60040a36b9ea439edd Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 25 Aug 2025 15:26:19 +0800 Subject: [PATCH 32/41] fix tpch q19 --- datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part index ace2081eb18fe..3000165b4c181 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -59,7 +59,7 @@ logical_plan 03)----Projection: lineitem.l_extendedprice, lineitem.l_discount 04)------Inner Join: lineitem.l_partkey = part.p_partkey Filter: part.p_brand = Utf8View("Brand#12") AND part.p_container IN ([Utf8View("SM CASE"), Utf8View("SM BOX"), Utf8View("SM PACK"), Utf8View("SM PKG")]) AND lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8View("Brand#23") AND part.p_container IN ([Utf8View("MED BAG"), Utf8View("MED BOX"), Utf8View("MED PKG"), Utf8View("MED PACK")]) AND lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8View("Brand#34") AND part.p_container IN ([Utf8View("LG CASE"), Utf8View("LG BOX"), Utf8View("LG PACK"), Utf8View("LG PKG")]) AND lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND part.p_size <= Int32(15) 05)--------Projection: lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount -06)----------Filter: (lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)) AND (lineitem.l_shipmode = Utf8View("AIR") OR lineitem.l_shipmode = Utf8View("AIR REG")) AND lineitem.l_shipinstruct = Utf8View("DELIVER IN PERSON") +06)----------Filter: lineitem.l_shipinstruct = Utf8View("DELIVER IN PERSON") AND (lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)) AND (lineitem.l_shipmode = Utf8View("AIR") OR lineitem.l_shipmode = Utf8View("AIR REG")) 07)------------TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8View("AIR") OR lineitem.l_shipmode = Utf8View("AIR REG"), lineitem.l_shipinstruct = Utf8View("DELIVER IN PERSON"), lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)] 08)--------Filter: (part.p_brand = Utf8View("Brand#12") AND part.p_container IN ([Utf8View("SM CASE"), Utf8View("SM BOX"), Utf8View("SM PACK"), Utf8View("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8View("Brand#23") AND part.p_container IN ([Utf8View("MED BAG"), Utf8View("MED BOX"), Utf8View("MED PKG"), Utf8View("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8View("Brand#34") AND part.p_container IN ([Utf8View("LG CASE"), Utf8View("LG BOX"), Utf8View("LG PACK"), Utf8View("LG PKG")]) AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) 09)----------TableScan: part projection=[p_partkey, p_brand, p_size, p_container], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8View("Brand#12") AND part.p_container IN ([Utf8View("SM CASE"), Utf8View("SM BOX"), Utf8View("SM PACK"), Utf8View("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8View("Brand#23") AND part.p_container IN ([Utf8View("MED BAG"), Utf8View("MED BOX"), Utf8View("MED PKG"), Utf8View("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8View("Brand#34") AND part.p_container IN ([Utf8View("LG CASE"), Utf8View("LG BOX"), Utf8View("LG PACK"), Utf8View("LG PKG")]) AND part.p_size <= Int32(15)] @@ -73,7 +73,7 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------FilterExec: (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG) AND l_shipinstruct@4 = DELIVER IN PERSON, projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] +10)------------------FilterExec: l_shipinstruct@4 = DELIVER IN PERSON AND (l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2) AND (l_shipmode@5 = AIR OR l_shipmode@5 = AIR REG), projection=[l_partkey@0, l_quantity@1, l_extendedprice@2, l_discount@3] 11)--------------------DataSourceExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], file_type=csv, has_header=false 12)------------CoalesceBatchesExec: target_batch_size=8192 13)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 From 9191f3922175d2945da60b1c63a2e68ab36a896f Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 25 Aug 2025 16:15:09 +0800 Subject: [PATCH 33/41] public GroupValues & new_group_values --- .../src/aggregates/group_values/mod.rs | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index 1e4c7558bda39..f2f489b7223c3 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! `GroupValues` trait for storing and interning group keys +//! [`GroupValues`] trait for storing and interning group keys use arrow::array::types::{ Date32Type, Date64Type, Decimal128Type, Time32MillisecondType, Time32SecondType, @@ -28,7 +28,7 @@ use datafusion_common::Result; use datafusion_expr::EmitTo; -pub(crate) mod multi_group_by; +pub mod multi_group_by; mod row; mod single_group_by; @@ -84,7 +84,7 @@ mod null_builder; /// Each distinct group in a hash aggregation is identified by a unique group id /// (usize) which is assigned by instances of this trait. Group ids are /// continuous without gaps, starting from 0. -pub(crate) trait GroupValues: Send { +pub trait GroupValues: Send { /// Calculates the group id for each input row of `cols`, assigning new /// group ids as necessary. /// @@ -119,15 +119,17 @@ pub(crate) trait GroupValues: Send { /// - If group by single column, and type of this column has /// the specific [`GroupValues`] implementation, such implementation /// will be chosen. -/// +/// /// - If group by multiple columns, and all column types have the specific -/// [`GroupColumn`] implementations, [`GroupValuesColumn`] will be chosen. +/// `GroupColumn` implementations, `GroupValuesColumn` will be chosen. /// -/// - Otherwise, the general implementation [`GroupValuesRows`] will be chosen. +/// - Otherwise, the general implementation `GroupValuesRows` will be chosen. /// -/// [`GroupColumn`]: crate::aggregates::group_values::multi_group_by::GroupColumn +/// `GroupColumn`: crate::aggregates::group_values::multi_group_by::GroupColumn +/// `GroupValuesColumn`: crate::aggregates::group_values::multi_group_by::GroupValuesColumn +/// `GroupValuesRows`: crate::aggregates::group_values::row::GroupValuesRows /// -pub(crate) fn new_group_values( +pub fn new_group_values( schema: SchemaRef, group_ordering: &GroupOrdering, ) -> Result> { From d358db4587210671bc1a7d0241d884327e7be0d4 Mon Sep 17 00:00:00 2001 From: xudong963 Date: Mon, 25 Aug 2025 16:35:50 +0800 Subject: [PATCH 34/41] fix clippy --- datafusion/physical-plan/src/aggregates/group_values/mod.rs | 2 +- .../src/aggregates/group_values/multi_group_by/mod.rs | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index f2f489b7223c3..c64be0de1e83f 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -119,7 +119,7 @@ pub trait GroupValues: Send { /// - If group by single column, and type of this column has /// the specific [`GroupValues`] implementation, such implementation /// will be chosen. -/// +/// /// - If group by multiple columns, and all column types have the specific /// `GroupColumn` implementations, `GroupValuesColumn` will be chosen. /// diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs index 2ac0389454dec..9b547a45e8b08 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs @@ -91,6 +91,11 @@ pub trait GroupColumn: Send + Sync { /// Returns the number of rows stored in this builder fn len(&self) -> usize; + /// true if len == 0 + fn is_empty(&self) -> bool { + self.len() == 0 + } + /// Returns the number of bytes used by this [`GroupColumn`] fn size(&self) -> usize; From cefa63a42e616819cc0dd60ee39cdb6aa8792a9a Mon Sep 17 00:00:00 2001 From: Qi Zhu Date: Wed, 3 Sep 2025 11:16:28 +0800 Subject: [PATCH 35/41] fix fetch with new order lex --- .../src/enforce_distribution.rs | 23 +++++++------------ 1 file changed, 8 insertions(+), 15 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 86bdec312a24d..1a1bea02c7c16 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -940,28 +940,21 @@ fn add_merge_on_top( input: DistributionContext, fetch: &mut Option, ) -> DistributionContext { - // Add SortPreservingMerge only when partition count is larger than 1. + // Apply only when the partition count is larger than one. if input.plan.output_partitioning().partition_count() > 1 { // When there is an existing ordering, we preserve ordering // when decreasing partitions. This will be un-done in the future // if any of the following conditions is true // - Preserving ordering is not helpful in terms of satisfying ordering requirements // - Usage of order preserving variants is not desirable - // (determined by flag `config.optimizer.bounded_order_preserving_variants`) - let should_preserve_ordering = input.plan.output_ordering().is_some(); - - let ordering = input - .plan - .output_ordering() - .cloned() - .unwrap_or_else(LexOrdering::default); - - let new_plan = if should_preserve_ordering { - Arc::new( - SortPreservingMergeExec::new(ordering, Arc::clone(&input.plan)) - .with_fetch(fetch.take()), - ) as _ + // (determined by flag `config.optimizer.prefer_existing_sort`) + let new_plan = if let Some(req) = input.plan.output_ordering() { + Arc::new(SortPreservingMergeExec::new( + req.clone(), + Arc::clone(&input.plan), + ).with_fetch(*fetch)) as _ } else { + // If there is no input order, we can simply coalesce partitions: Arc::new(CoalescePartitionsExec::new(Arc::clone(&input.plan))) as _ }; From 1f47d469005cf56a6ae7b6c74ce12a2d8fb40552 Mon Sep 17 00:00:00 2001 From: Qi Zhu Date: Wed, 3 Sep 2025 11:19:02 +0800 Subject: [PATCH 36/41] fix fetch add back with new lex order --- .../src/enforce_distribution.rs | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 1a1bea02c7c16..c43f98c999c8f 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -949,10 +949,10 @@ fn add_merge_on_top( // - Usage of order preserving variants is not desirable // (determined by flag `config.optimizer.prefer_existing_sort`) let new_plan = if let Some(req) = input.plan.output_ordering() { - Arc::new(SortPreservingMergeExec::new( - req.clone(), - Arc::clone(&input.plan), - ).with_fetch(*fetch)) as _ + Arc::new( + SortPreservingMergeExec::new(req.clone(), Arc::clone(&input.plan)) + .with_fetch(*fetch), + ) as _ } else { // If there is no input order, we can simply coalesce partitions: Arc::new(CoalescePartitionsExec::new(Arc::clone(&input.plan))) as _ @@ -1406,13 +1406,8 @@ pub fn ensure_distribution( // It was removed by `remove_dist_changing_operators` // and we need to add it back. if fetch.is_some() { - let ordering = plan - .output_ordering() - .cloned() - .unwrap_or_else(LexOrdering::default); - let plan = Arc::new( - SortPreservingMergeExec::new(ordering, plan).with_fetch(fetch.take()), - ); + // It's safe to unwrap because `spm` is set only if `fetch` is set. + let plan = spm.unwrap().with_fetch(fetch.take()).unwrap(); optimized_distribution_ctx = DistributionContext::new(plan, data, vec![optimized_distribution_ctx]); } From 70a3c94742a594cb7490b8d9ba4248990a24ec1d Mon Sep 17 00:00:00 2001 From: Qi Zhu Date: Wed, 3 Sep 2025 11:42:28 +0800 Subject: [PATCH 37/41] fix clippy --- datafusion/physical-optimizer/src/enforce_distribution.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index c43f98c999c8f..0d458add31305 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1031,6 +1031,7 @@ fn remove_dist_changing_operators( /// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", /// " DataSourceExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC], file_type=parquet", /// ``` +#[allow(clippy::type_complexity)] pub fn replace_order_preserving_variants( mut context: DistributionContext, ordering_satisfied: bool, From a93e81e3713841b744811e360619d120052b5da2 Mon Sep 17 00:00:00 2001 From: Qi Zhu Date: Wed, 3 Sep 2025 11:51:21 +0800 Subject: [PATCH 38/41] add order needed --- .../physical-optimizer/src/enforce_distribution.rs | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 0d458add31305..2f7c4e75038d3 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1407,8 +1407,13 @@ pub fn ensure_distribution( // It was removed by `remove_dist_changing_operators` // and we need to add it back. if fetch.is_some() { - // It's safe to unwrap because `spm` is set only if `fetch` is set. - let plan = spm.unwrap().with_fetch(fetch.take()).unwrap(); + // We can make sure that `plan` has an ordering because + // `SortPreservingMergeExec` requires ordering to be constructed. + // If there is no ordering, `SortPreservingMergeExec::new` will panic + let ordering = plan.output_ordering().cloned().unwrap(); + let plan = Arc::new( + SortPreservingMergeExec::new(ordering, plan).with_fetch(fetch.take()), + ); optimized_distribution_ctx = DistributionContext::new(plan, data, vec![optimized_distribution_ctx]); } From 6a3d4f8cc2d0f7250eafe4539d39a3b1ff0944e7 Mon Sep 17 00:00:00 2001 From: Qi Zhu Date: Wed, 3 Sep 2025 12:17:13 +0800 Subject: [PATCH 39/41] fix --- datafusion/physical-optimizer/src/enforce_distribution.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 2f7c4e75038d3..740235b68ee25 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -981,6 +981,7 @@ fn add_merge_on_top( /// ```text /// "DataSourceExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC], file_type=parquet", /// ``` +#[allow(clippy::type_complexity)] fn remove_dist_changing_operators( mut distribution_context: DistributionContext, ) -> Result<( @@ -1222,7 +1223,7 @@ pub fn ensure_distribution( children, }, mut fetch, - spm, + _spm, ) = remove_dist_changing_operators(dist_context)?; if let Some(exec) = plan.as_any().downcast_ref::() { From 91e2904da4da233b3d56901abacdecd4287260f5 Mon Sep 17 00:00:00 2001 From: Qi Zhu Date: Wed, 3 Sep 2025 15:29:23 +0800 Subject: [PATCH 40/41] fix auth check and port upstream fix: https://github.com/apache/datafusion/pull/17355 --- Cargo.lock | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 9dc0a5b639888..9a6be2dc5b6af 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4248,12 +4248,11 @@ dependencies = [ [[package]] name = "nu-ansi-term" -version = "0.46.0" +version = "0.50.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77a8165726e8236064dbb45459242600304b42a5ea24ee2948e18e023bf7ba84" +checksum = "d4a28e057d01f97e61255210fcff094d74ed0466038633e95017f5beb68e4399" dependencies = [ - "overload", - "winapi", + "windows-sys 0.52.0", ] [[package]] @@ -4453,12 +4452,6 @@ version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1a80800c0488c3a21695ea981a54918fbb37abf04f4d0720c453632255e2ff0e" -[[package]] -name = "overload" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" - [[package]] name = "owo-colors" version = "4.2.1" @@ -6737,9 +6730,9 @@ dependencies = [ [[package]] name = "tracing-subscriber" -version = "0.3.19" +version = "0.3.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e8189decb5ac0fa7bc8b96b7cb9b2701d60d48805aca84a238004d665fcc4008" +checksum = "2054a14f5307d601f88daf0553e1cbf472acc4f2c51afab632431cdcd72124d5" dependencies = [ "nu-ansi-term", "sharded-slab", From 1a2f8dc98d9791ee24476f3fef83d9ce5f68aa82 Mon Sep 17 00:00:00 2001 From: Qi Zhu Date: Thu, 4 Sep 2025 13:31:45 +0800 Subject: [PATCH 41/41] Addressed in latest PR --- .../physical-optimizer/src/enforce_distribution.rs | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 740235b68ee25..42c08ce8f437e 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -951,7 +951,7 @@ fn add_merge_on_top( let new_plan = if let Some(req) = input.plan.output_ordering() { Arc::new( SortPreservingMergeExec::new(req.clone(), Arc::clone(&input.plan)) - .with_fetch(*fetch), + .with_fetch(fetch.take()), ) as _ } else { // If there is no input order, we can simply coalesce partitions: @@ -1223,7 +1223,7 @@ pub fn ensure_distribution( children, }, mut fetch, - _spm, + spm, ) = remove_dist_changing_operators(dist_context)?; if let Some(exec) = plan.as_any().downcast_ref::() { @@ -1408,13 +1408,8 @@ pub fn ensure_distribution( // It was removed by `remove_dist_changing_operators` // and we need to add it back. if fetch.is_some() { - // We can make sure that `plan` has an ordering because - // `SortPreservingMergeExec` requires ordering to be constructed. - // If there is no ordering, `SortPreservingMergeExec::new` will panic - let ordering = plan.output_ordering().cloned().unwrap(); - let plan = Arc::new( - SortPreservingMergeExec::new(ordering, plan).with_fetch(fetch.take()), - ); + // It's safe to unwrap because `spm` is set only if `fetch` is set. + let plan = spm.unwrap().with_fetch(fetch.take()).unwrap(); optimized_distribution_ctx = DistributionContext::new(plan, data, vec![optimized_distribution_ctx]); }