From db92106d391cdde1529c373add81fcef21094018 Mon Sep 17 00:00:00 2001 From: buraksenn Date: Wed, 22 Jan 2025 16:47:32 +0300 Subject: [PATCH 01/12] first iteration --- .../core/src/datasource/physical_plan/csv.rs | 34 ++ .../physical_optimizer/projection_pushdown.rs | 410 ++---------------- .../src/output_requirements.rs | 53 ++- .../physical-plan/src/execution_plan.rs | 12 + datafusion/physical-plan/src/filter.rs | 27 ++ datafusion/physical-plan/src/lib.rs | 1 + datafusion/physical-plan/src/memory.rs | 26 ++ .../physical-plan/src/projection_utils.rs | 150 +++++++ .../physical-plan/src/repartition/mod.rs | 42 ++ datafusion/physical-plan/src/sorts/sort.rs | 33 ++ .../src/sorts/sort_preserving_merge.rs | 36 ++ datafusion/physical-plan/src/streaming.rs | 53 ++- datafusion/physical-plan/src/union.rs | 23 + 13 files changed, 517 insertions(+), 383 deletions(-) create mode 100644 datafusion/physical-plan/src/projection_utils.rs diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index dd5736806eeb..0635d5bfa023 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -45,6 +45,10 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::projection_utils::{ + all_alias_free_columns, new_projections_for_columns, +}; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; use object_store::{GetOptions, GetResultPayload, ObjectStore}; @@ -479,6 +483,36 @@ impl ExecutionPlan for CsvExec { cache: self.cache.clone(), })) } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + // If there is any non-column or alias-carrier expression, Projection should not be removed. + // This process can be moved into CsvExec, but it would be an overlap of their responsibility. + Ok(all_alias_free_columns(projection.expr()).then(|| { + let mut file_scan = self.base_config().clone(); + let new_projections = new_projections_for_columns( + projection, + &file_scan + .projection + .unwrap_or((0..self.schema().fields().len()).collect()), + ); + file_scan.projection = Some(new_projections); + + Arc::new( + CsvExec::builder(file_scan) + .with_has_header(self.has_header()) + .with_delimeter(self.delimiter()) + .with_quote(self.quote()) + .with_escape(self.escape()) + .with_comment(self.comment()) + .with_newlines_in_values(self.newlines_in_values()) + .with_file_compression_type(self.file_compression_type) + .build(), + ) as _ + })) + } } /// A Config for [`CsvOpener`] diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index cf8d5e352ef1..956c73dd069c 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -23,8 +23,6 @@ use std::collections::HashMap; use std::sync::Arc; -use super::output_requirements::OutputRequirementExec; -use crate::datasource::physical_plan::CsvExec; use crate::error::Result; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::filter::FilterExec; @@ -33,12 +31,10 @@ use crate::physical_plan::joins::{ CrossJoinExec, HashJoinExec, NestedLoopJoinExec, SortMergeJoinExec, SymmetricHashJoinExec, }; -use crate::physical_plan::memory::MemoryExec; + use crate::physical_plan::projection::ProjectionExec; -use crate::physical_plan::repartition::RepartitionExec; -use crate::physical_plan::sorts::sort::SortExec; -use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use crate::physical_plan::{Distribution, ExecutionPlan, ExecutionPlanProperties}; + +use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; @@ -47,15 +43,10 @@ use datafusion_common::tree_node::{ }; use datafusion_common::{internal_err, JoinSide}; use datafusion_physical_expr::expressions::{Column, Literal}; -use datafusion_physical_expr::{ - utils::collect_columns, Partitioning, PhysicalExpr, PhysicalExprRef, - PhysicalSortExpr, PhysicalSortRequirement, -}; +use datafusion_physical_expr::{utils::collect_columns, PhysicalExpr, PhysicalExprRef}; use datafusion_physical_plan::joins::utils::{JoinOn, JoinOnRef}; -use datafusion_physical_plan::streaming::StreamingTableExec; -use datafusion_physical_plan::union::UnionExec; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_optimizer::PhysicalOptimizerRule; use itertools::Itertools; @@ -106,11 +97,7 @@ pub fn remove_unnecessary_projections( } // If it does, check if we can push it under its child(ren): let input = projection.input().as_any(); - if let Some(csv) = input.downcast_ref::() { - try_swapping_with_csv(projection, csv) - } else if let Some(memory) = input.downcast_ref::() { - try_swapping_with_memory(projection, memory)? - } else if let Some(child_projection) = input.downcast_ref::() { + if let Some(child_projection) = input.downcast_ref::() { let maybe_unified = try_unifying_projections(projection, child_projection)?; return if let Some(new_plan) = maybe_unified { // To unify 3 or more sequential projections: @@ -120,23 +107,15 @@ pub fn remove_unnecessary_projections( } else { Ok(Transformed::no(plan)) }; - } else if let Some(output_req) = input.downcast_ref::() { - try_swapping_with_output_req(projection, output_req)? } else if input.is::() { try_swapping_with_coalesce_partitions(projection)? } else if let Some(filter) = input.downcast_ref::() { - try_swapping_with_filter(projection, filter)?.map_or_else( - || try_embed_projection(projection, filter), - |e| Ok(Some(e)), - )? - } else if let Some(repartition) = input.downcast_ref::() { - try_swapping_with_repartition(projection, repartition)? - } else if let Some(sort) = input.downcast_ref::() { - try_swapping_with_sort(projection, sort)? - } else if let Some(spm) = input.downcast_ref::() { - try_swapping_with_sort_preserving_merge(projection, spm)? - } else if let Some(union) = input.downcast_ref::() { - try_pushdown_through_union(projection, union)? + filter + .try_swapping_with_projection(projection)? + .map_or_else( + || try_embed_projection(projection, filter), + |e| Ok(Some(e)), + )? } else if let Some(hash_join) = input.downcast_ref::() { try_pushdown_through_hash_join(projection, hash_join)? } else if let Some(cross_join) = input.downcast_ref::() { @@ -147,14 +126,10 @@ pub fn remove_unnecessary_projections( try_swapping_with_sort_merge_join(projection, sm_join)? } else if let Some(sym_join) = input.downcast_ref::() { try_swapping_with_sym_hash_join(projection, sym_join)? - } else if let Some(ste) = input.downcast_ref::() { - try_swapping_with_streaming_table(projection, ste)? } else { - // If the input plan of the projection is not one of the above, we - // conservatively assume that pushing the projection down may hurt. - // When adding new operators, consider adding them here if you - // think pushing projections under them is beneficial. - None + projection + .input() + .try_swapping_with_projection(projection)? } } else { return Ok(Transformed::no(plan)); @@ -163,112 +138,6 @@ pub fn remove_unnecessary_projections( Ok(maybe_modified.map_or(Transformed::no(plan), Transformed::yes)) } -/// Tries to embed `projection` to its input (`csv`). If possible, returns -/// [`CsvExec`] as the top plan. Otherwise, returns `None`. -fn try_swapping_with_csv( - projection: &ProjectionExec, - csv: &CsvExec, -) -> Option> { - // If there is any non-column or alias-carrier expression, Projection should not be removed. - // This process can be moved into CsvExec, but it would be an overlap of their responsibility. - all_alias_free_columns(projection.expr()).then(|| { - let mut file_scan = csv.base_config().clone(); - let new_projections = new_projections_for_columns( - projection, - &file_scan - .projection - .unwrap_or((0..csv.schema().fields().len()).collect()), - ); - file_scan.projection = Some(new_projections); - - Arc::new( - CsvExec::builder(file_scan) - .with_has_header(csv.has_header()) - .with_delimeter(csv.delimiter()) - .with_quote(csv.quote()) - .with_escape(csv.escape()) - .with_comment(csv.comment()) - .with_newlines_in_values(csv.newlines_in_values()) - .with_file_compression_type(csv.file_compression_type) - .build(), - ) as _ - }) -} - -/// Tries to embed `projection` to its input (`memory`). If possible, returns -/// [`MemoryExec`] as the top plan. Otherwise, returns `None`. -fn try_swapping_with_memory( - projection: &ProjectionExec, - memory: &MemoryExec, -) -> Result>> { - // If there is any non-column or alias-carrier expression, Projection should not be removed. - // This process can be moved into MemoryExec, but it would be an overlap of their responsibility. - all_alias_free_columns(projection.expr()) - .then(|| { - let all_projections = (0..memory.schema().fields().len()).collect(); - let new_projections = new_projections_for_columns( - projection, - memory.projection().as_ref().unwrap_or(&all_projections), - ); - - MemoryExec::try_new( - memory.partitions(), - memory.original_schema(), - Some(new_projections), - ) - .map(|e| Arc::new(e) as _) - }) - .transpose() -} - -/// Tries to embed `projection` to its input (`streaming table`). -/// If possible, returns [`StreamingTableExec`] as the top plan. Otherwise, -/// returns `None`. -fn try_swapping_with_streaming_table( - projection: &ProjectionExec, - streaming_table: &StreamingTableExec, -) -> Result>> { - if !all_alias_free_columns(projection.expr()) { - return Ok(None); - } - - let streaming_table_projections = streaming_table - .projection() - .as_ref() - .map(|i| i.as_ref().to_vec()); - let new_projections = new_projections_for_columns( - projection, - &streaming_table_projections - .unwrap_or((0..streaming_table.schema().fields().len()).collect()), - ); - - let mut lex_orderings = vec![]; - for lex_ordering in streaming_table.projected_output_ordering().into_iter() { - let mut orderings = LexOrdering::default(); - for order in lex_ordering { - let Some(new_ordering) = update_expr(&order.expr, projection.expr(), false)? - else { - return Ok(None); - }; - orderings.push(PhysicalSortExpr { - expr: new_ordering, - options: order.options, - }); - } - lex_orderings.push(orderings); - } - - StreamingTableExec::try_new( - Arc::clone(streaming_table.partition_schema()), - streaming_table.partitions().clone(), - Some(new_projections.as_ref()), - lex_orderings, - streaming_table.is_infinite(), - streaming_table.limit(), - ) - .map(|e| Some(Arc::new(e) as _)) -} - /// Unifies `projection` with its input (which is also a [`ProjectionExec`]). fn try_unifying_projections( projection: &ProjectionExec, @@ -321,57 +190,6 @@ fn is_expr_trivial(expr: &Arc) -> bool { || expr.as_any().downcast_ref::().is_some() } -/// Tries to swap `projection` with its input (`output_req`). If possible, -/// performs the swap and returns [`OutputRequirementExec`] as the top plan. -/// Otherwise, returns `None`. -fn try_swapping_with_output_req( - projection: &ProjectionExec, - output_req: &OutputRequirementExec, -) -> Result>> { - // If the projection does not narrow the schema, we should not try to push it down: - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - let mut updated_sort_reqs = LexRequirement::new(vec![]); - // None or empty_vec can be treated in the same way. - if let Some(reqs) = &output_req.required_input_ordering()[0] { - for req in &reqs.inner { - let Some(new_expr) = update_expr(&req.expr, projection.expr(), false)? else { - return Ok(None); - }; - updated_sort_reqs.push(PhysicalSortRequirement { - expr: new_expr, - options: req.options, - }); - } - } - - let dist_req = match &output_req.required_input_distribution()[0] { - Distribution::HashPartitioned(exprs) => { - let mut updated_exprs = vec![]; - for expr in exprs { - let Some(new_expr) = update_expr(expr, projection.expr(), false)? else { - return Ok(None); - }; - updated_exprs.push(new_expr); - } - Distribution::HashPartitioned(updated_exprs) - } - dist => dist.clone(), - }; - - make_with_child(projection, &output_req.input()) - .map(|input| { - OutputRequirementExec::new( - input, - (!updated_sort_reqs.is_empty()).then_some(updated_sort_reqs), - dist_req, - ) - }) - .map(|e| Some(Arc::new(e) as _)) -} - /// Tries to swap `projection` with its input, which is known to be a /// [`CoalescePartitionsExec`]. If possible, performs the swap and returns /// [`CoalescePartitionsExec`] as the top plan. Otherwise, returns `None`. @@ -387,156 +205,6 @@ fn try_swapping_with_coalesce_partitions( .map(|e| Some(Arc::new(CoalescePartitionsExec::new(e)) as _)) } -/// Tries to swap `projection` with its input (`filter`). If possible, performs -/// the swap and returns [`FilterExec`] as the top plan. Otherwise, returns `None`. -fn try_swapping_with_filter( - projection: &ProjectionExec, - filter: &FilterExec, -) -> Result>> { - // If the projection does not narrow the schema, we should not try to push it down: - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - // Each column in the predicate expression must exist after the projection. - let Some(new_predicate) = update_expr(filter.predicate(), projection.expr(), false)? - else { - return Ok(None); - }; - - FilterExec::try_new(new_predicate, make_with_child(projection, filter.input())?) - .and_then(|e| { - let selectivity = filter.default_selectivity(); - e.with_default_selectivity(selectivity) - }) - .map(|e| Some(Arc::new(e) as _)) -} - -/// Tries to swap the projection with its input [`RepartitionExec`]. If it can be done, -/// it returns the new swapped version having the [`RepartitionExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_repartition( - projection: &ProjectionExec, - repartition: &RepartitionExec, -) -> Result>> { - // If the projection does not narrow the schema, we should not try to push it down. - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - // If pushdown is not beneficial or applicable, break it. - if projection.benefits_from_input_partitioning()[0] || !all_columns(projection.expr()) - { - return Ok(None); - } - - let new_projection = make_with_child(projection, repartition.input())?; - - let new_partitioning = match repartition.partitioning() { - Partitioning::Hash(partitions, size) => { - let mut new_partitions = vec![]; - for partition in partitions { - let Some(new_partition) = - update_expr(partition, projection.expr(), false)? - else { - return Ok(None); - }; - new_partitions.push(new_partition); - } - Partitioning::Hash(new_partitions, *size) - } - others => others.clone(), - }; - - Ok(Some(Arc::new(RepartitionExec::try_new( - new_projection, - new_partitioning, - )?))) -} - -/// Tries to swap the projection with its input [`SortExec`]. If it can be done, -/// it returns the new swapped version having the [`SortExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_sort( - projection: &ProjectionExec, - sort: &SortExec, -) -> Result>> { - // If the projection does not narrow the schema, we should not try to push it down. - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - let mut updated_exprs = LexOrdering::default(); - for sort in sort.expr() { - let Some(new_expr) = update_expr(&sort.expr, projection.expr(), false)? else { - return Ok(None); - }; - updated_exprs.push(PhysicalSortExpr { - expr: new_expr, - options: sort.options, - }); - } - - Ok(Some(Arc::new( - SortExec::new(updated_exprs, make_with_child(projection, sort.input())?) - .with_fetch(sort.fetch()) - .with_preserve_partitioning(sort.preserve_partitioning()), - ))) -} - -/// Tries to swap the projection with its input [`SortPreservingMergeExec`]. -/// If this is possible, it returns the new [`SortPreservingMergeExec`] whose -/// child is a projection. Otherwise, it returns None. -fn try_swapping_with_sort_preserving_merge( - projection: &ProjectionExec, - spm: &SortPreservingMergeExec, -) -> Result>> { - // If the projection does not narrow the schema, we should not try to push it down. - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - let mut updated_exprs = LexOrdering::default(); - for sort in spm.expr() { - let Some(updated_expr) = update_expr(&sort.expr, projection.expr(), false)? - else { - return Ok(None); - }; - updated_exprs.push(PhysicalSortExpr { - expr: updated_expr, - options: sort.options, - }); - } - - Ok(Some(Arc::new( - SortPreservingMergeExec::new( - updated_exprs, - make_with_child(projection, spm.input())?, - ) - .with_fetch(spm.fetch()), - ))) -} - -/// Tries to push `projection` down through `union`. If possible, performs the -/// pushdown and returns a new [`UnionExec`] as the top plan which has projections -/// as its children. Otherwise, returns `None`. -fn try_pushdown_through_union( - projection: &ProjectionExec, - union: &UnionExec, -) -> Result>> { - // If the projection doesn't narrow the schema, we shouldn't try to push it down. - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - let new_children = union - .children() - .into_iter() - .map(|child| make_with_child(projection, child)) - .collect::>>()?; - - Ok(Some(Arc::new(UnionExec::new(new_children)))) -} - trait EmbeddedProjection: ExecutionPlan + Sized { fn with_projection(&self, projection: Option>) -> Result; } @@ -964,35 +632,6 @@ fn is_projection_removable(projection: &ProjectionExec) -> bool { }) && exprs.len() == projection.input().schema().fields().len() } -/// Given the expression set of a projection, checks if the projection causes -/// any renaming or constructs a non-`Column` physical expression. -fn all_alias_free_columns(exprs: &[(Arc, String)]) -> bool { - exprs.iter().all(|(expr, alias)| { - expr.as_any() - .downcast_ref::() - .map(|column| column.name() == alias) - .unwrap_or(false) - }) -} - -/// Updates a source provider's projected columns according to the given -/// projection operator's expressions. To use this function safely, one must -/// ensure that all expressions are `Column` expressions without aliases. -fn new_projections_for_columns( - projection: &ProjectionExec, - source: &[usize], -) -> Vec { - projection - .expr() - .iter() - .filter_map(|(expr, _)| { - expr.as_any() - .downcast_ref::() - .map(|expr| source[expr.index()]) - }) - .collect() -} - /// The function operates in two modes: /// /// 1) When `sync_with_child` is `true`: @@ -1083,11 +722,6 @@ fn make_with_child( .map(|e| Arc::new(e) as _) } -/// Returns `true` if all the expressions in the argument are `Column`s. -fn all_columns(exprs: &[(Arc, String)]) -> bool { - exprs.iter().all(|(expr, _)| expr.as_any().is::()) -} - /// Downcasts all the expressions in `exprs` to `Column`s. If any of the given /// expressions is not a `Column`, returns `None`. fn physical_to_column_exprs( @@ -1354,6 +988,20 @@ mod tests { use super::*; use std::any::Any; + use crate::datasource::physical_plan::CsvExec; + use crate::physical_plan::memory::MemoryExec; + use crate::physical_plan::repartition::RepartitionExec; + use crate::physical_plan::sorts::sort::SortExec; + use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; + use datafusion_physical_expr::{ + Distribution, Partitioning, PhysicalExpr, PhysicalSortExpr, + PhysicalSortRequirement, + }; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; + use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; + use datafusion_physical_plan::streaming::StreamingTableExec; + use datafusion_physical_plan::union::UnionExec; + use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::FileScanConfig; diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index e107bb85d7b8..0a294fa8b1a5 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -25,6 +25,7 @@ use std::sync::Arc; use datafusion_execution::TaskContext; +use datafusion_physical_plan::projection_utils::{make_with_child, update_expr}; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, @@ -33,7 +34,7 @@ use datafusion_physical_plan::{ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Result, Statistics}; -use datafusion_physical_expr::{Distribution, LexRequirement}; +use datafusion_physical_expr::{Distribution, LexRequirement, PhysicalSortRequirement}; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{ExecutionPlanProperties, PlanProperties}; @@ -192,6 +193,56 @@ impl ExecutionPlan for OutputRequirementExec { fn statistics(&self) -> Result { self.input.statistics() } + + fn try_swapping_with_projection( + &self, + projection: &datafusion_physical_plan::projection::ProjectionExec, + ) -> Result>> { + // If the projection does not narrow the schema, we should not try to push it down: + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + + let mut updated_sort_reqs = LexRequirement::new(vec![]); + // None or empty_vec can be treated in the same way. + if let Some(reqs) = &self.required_input_ordering()[0] { + for req in &reqs.inner { + let Some(new_expr) = update_expr(&req.expr, projection.expr(), false)? + else { + return Ok(None); + }; + updated_sort_reqs.push(PhysicalSortRequirement { + expr: new_expr, + options: req.options, + }); + } + } + + let dist_req = match &self.required_input_distribution()[0] { + Distribution::HashPartitioned(exprs) => { + let mut updated_exprs = vec![]; + for expr in exprs { + let Some(new_expr) = update_expr(expr, projection.expr(), false)? + else { + return Ok(None); + }; + updated_exprs.push(new_expr); + } + Distribution::HashPartitioned(updated_exprs) + } + dist => dist.clone(), + }; + + make_with_child(projection, &self.input()) + .map(|input| { + OutputRequirementExec::new( + input, + (!updated_sort_reqs.is_empty()).then_some(updated_sort_reqs), + dist_req, + ) + }) + .map(|e| Some(Arc::new(e) as _)) + } } impl PhysicalOptimizerRule for OutputRequirements { diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 753234c09994..4278a9574c19 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -46,6 +46,7 @@ pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDi pub use crate::metrics::Metric; use crate::metrics::MetricsSet; pub use crate::ordering::InputOrderMode; +use crate::projection::ProjectionExec; use crate::repartition::RepartitionExec; use crate::sorts::sort_preserving_merge::SortPreservingMergeExec; pub use crate::stream::EmptyRecordBatchStream; @@ -431,6 +432,17 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { fn cardinality_effect(&self) -> CardinalityEffect { CardinalityEffect::Unknown } + + // If try_swapping_with_projection is not implemented we conservatively + // assume that pushing the projection down may hurt. + // When adding new operators, consider adding them here if you + // think pushing projections under them is beneficial. + fn try_swapping_with_projection( + &self, + _projection: &ProjectionExec, + ) -> Result>> { + Ok(None) + } } /// [`ExecutionPlan`] Invariant Level diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 8e7c14f0baed..c06487f86501 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -25,6 +25,8 @@ use super::{ RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::common::can_project; +use crate::projection::ProjectionExec; +use crate::projection_utils::{make_with_child, update_expr}; use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, DisplayFormatType, ExecutionPlan, @@ -399,6 +401,31 @@ impl ExecutionPlan for FilterExec { fn cardinality_effect(&self) -> CardinalityEffect { CardinalityEffect::LowerEqual } + + /// Tries to swap `projection` with its input (`filter`). If possible, performs + /// the swap and returns [`FilterExec`] as the top plan. Otherwise, returns `None`. + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + // If the projection does not narrow the schema, we should not try to push it down: + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + // Each column in the predicate expression must exist after the projection. + let Some(new_predicate) = + update_expr(self.predicate(), projection.expr(), false)? + else { + return Ok(None); + }; + + FilterExec::try_new(new_predicate, make_with_child(projection, self.input())?) + .and_then(|e| { + let selectivity = self.default_selectivity(); + e.with_default_selectivity(selectivity) + }) + .map(|e| Some(Arc::new(e) as _)) + } } /// This function ensures that all bounds in the `ExprBoundaries` vector are diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 5ad37f0b1ac0..db75b3095fd7 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -67,6 +67,7 @@ pub mod memory; pub mod metrics; pub mod placeholder_row; pub mod projection; +pub mod projection_utils; pub mod recursive_query; pub mod repartition; pub mod sorts; diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index fb58a04fcc20..5f2596d2079e 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -29,6 +29,8 @@ use super::{ Statistics, }; use crate::execution_plan::{Boundedness, EmissionType}; +use crate::projection::ProjectionExec; +use crate::projection_utils::{all_alias_free_columns, new_projections_for_columns}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; @@ -166,6 +168,30 @@ impl ExecutionPlan for MemoryExec { self.projection.clone(), )) } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + // If there is any non-column or alias-carrier expression, Projection should not be removed. + // This process can be moved into MemoryExec, but it would be an overlap of their responsibility. + all_alias_free_columns(projection.expr()) + .then(|| { + let all_projections = (0..self.schema().fields().len()).collect(); + let new_projections = new_projections_for_columns( + projection, + self.projection().as_ref().unwrap_or(&all_projections), + ); + + MemoryExec::try_new( + self.partitions(), + self.original_schema(), + Some(new_projections), + ) + .map(|e| Arc::new(e) as _) + }) + .transpose() + } } impl MemoryExec { diff --git a/datafusion/physical-plan/src/projection_utils.rs b/datafusion/physical-plan/src/projection_utils.rs new file mode 100644 index 000000000000..b1f4e741ad04 --- /dev/null +++ b/datafusion/physical-plan/src/projection_utils.rs @@ -0,0 +1,150 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use crate::projection::ProjectionExec; + +use super::ExecutionPlan; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::Result; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::PhysicalExpr; + +/// Given the expression set of a projection, checks if the projection causes +/// any renaming or constructs a non-`Column` physical expression. +pub fn all_alias_free_columns(exprs: &[(Arc, String)]) -> bool { + exprs.iter().all(|(expr, alias)| { + expr.as_any() + .downcast_ref::() + .map(|column| column.name() == alias) + .unwrap_or(false) + }) +} + +/// Updates a source provider's projected columns according to the given +/// projection operator's expressions. To use this function safely, one must +/// ensure that all expressions are `Column` expressions without aliases. +pub fn new_projections_for_columns( + projection: &ProjectionExec, + source: &[usize], +) -> Vec { + projection + .expr() + .iter() + .filter_map(|(expr, _)| { + expr.as_any() + .downcast_ref::() + .map(|expr| source[expr.index()]) + }) + .collect() +} + +/// Creates a new [`ProjectionExec`] instance with the given child plan and +/// projected expressions. +pub fn make_with_child( + projection: &ProjectionExec, + child: &Arc, +) -> Result> { + ProjectionExec::try_new(projection.expr().to_vec(), Arc::clone(child)) + .map(|e| Arc::new(e) as _) +} + +/// Returns `true` if all the expressions in the argument are `Column`s. +pub fn all_columns(exprs: &[(Arc, String)]) -> bool { + exprs.iter().all(|(expr, _)| expr.as_any().is::()) +} + +/// The function operates in two modes: +/// +/// 1) When `sync_with_child` is `true`: +/// +/// The function updates the indices of `expr` if the expression resides +/// in the input plan. For instance, given the expressions `a@1 + b@2` +/// and `c@0` with the input schema `c@2, a@0, b@1`, the expressions are +/// updated to `a@0 + b@1` and `c@2`. +/// +/// 2) When `sync_with_child` is `false`: +/// +/// The function determines how the expression would be updated if a projection +/// was placed before the plan associated with the expression. If the expression +/// cannot be rewritten after the projection, it returns `None`. For example, +/// given the expressions `c@0`, `a@1` and `b@2`, and the [`ProjectionExec`] with +/// an output schema of `a, c_new`, then `c@0` becomes `c_new@1`, `a@1` becomes +/// `a@0`, but `b@2` results in `None` since the projection does not include `b`. +pub fn update_expr( + expr: &Arc, + projected_exprs: &[(Arc, String)], + sync_with_child: bool, +) -> Result>> { + #[derive(Debug, PartialEq)] + enum RewriteState { + /// The expression is unchanged. + Unchanged, + /// Some part of the expression has been rewritten + RewrittenValid, + /// Some part of the expression has been rewritten, but some column + /// references could not be. + RewrittenInvalid, + } + + let mut state = RewriteState::Unchanged; + + let new_expr = Arc::clone(expr) + .transform_up(|expr: Arc| { + if state == RewriteState::RewrittenInvalid { + return Ok(Transformed::no(expr)); + } + + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::no(expr)); + }; + if sync_with_child { + state = RewriteState::RewrittenValid; + // Update the index of `column`: + Ok(Transformed::yes(Arc::clone( + &projected_exprs[column.index()].0, + ))) + } else { + // default to invalid, in case we can't find the relevant column + state = RewriteState::RewrittenInvalid; + // Determine how to update `column` to accommodate `projected_exprs` + projected_exprs + .iter() + .enumerate() + .find_map(|(index, (projected_expr, alias))| { + projected_expr.as_any().downcast_ref::().and_then( + |projected_column| { + (column.name().eq(projected_column.name()) + && column.index() == projected_column.index()) + .then(|| { + state = RewriteState::RewrittenValid; + Arc::new(Column::new(alias, index)) as _ + }) + }, + ) + }) + .map_or_else( + || Ok(Transformed::no(expr)), + |c| Ok(Transformed::yes(c)), + ) + } + }) + .data(); + + new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) +} diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 270aabeb553c..58da4639661d 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -32,6 +32,8 @@ use super::{ use crate::execution_plan::CardinalityEffect; use crate::hash_utils::create_hashes; use crate::metrics::BaselineMetrics; +use crate::projection::ProjectionExec; +use crate::projection_utils::{all_columns, make_with_child, update_expr}; use crate::repartition::distributor_channels::{ channels, partition_aware_channels, DistributionReceiver, DistributionSender, }; @@ -672,6 +674,46 @@ impl ExecutionPlan for RepartitionExec { fn cardinality_effect(&self) -> CardinalityEffect { CardinalityEffect::Equal } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + // If the projection does not narrow the schema, we should not try to push it down. + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + + // If pushdown is not beneficial or applicable, break it. + if projection.benefits_from_input_partitioning()[0] + || !all_columns(projection.expr()) + { + return Ok(None); + } + + let new_projection = make_with_child(projection, self.input())?; + + let new_partitioning = match self.partitioning() { + Partitioning::Hash(partitions, size) => { + let mut new_partitions = vec![]; + for partition in partitions { + let Some(new_partition) = + update_expr(partition, projection.expr(), false)? + else { + return Ok(None); + }; + new_partitions.push(new_partition); + } + Partitioning::Hash(new_partitions, *size) + } + others => others.clone(), + }; + + Ok(Some(Arc::new(RepartitionExec::try_new( + new_projection, + new_partitioning, + )?))) + } } impl RepartitionExec { diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index fd7e426a82c5..433c1d6f2efd 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -31,6 +31,8 @@ use crate::limit::LimitStream; use crate::metrics::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, }; +use crate::projection::ProjectionExec; +use crate::projection_utils::{make_with_child, update_expr}; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::{ get_record_batch_memory_size, read_spill_as_stream, spill_record_batches, @@ -1024,6 +1026,37 @@ impl ExecutionPlan for SortExec { CardinalityEffect::LowerEqual } } + + /// Tries to swap the projection with its input [`SortExec`]. If it can be done, + /// it returns the new swapped version having the [`SortExec`] as the top plan. + /// Otherwise, it returns None. + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + // If the projection does not narrow the schema, we should not try to push it down. + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + + let mut updated_exprs = LexOrdering::default(); + for sort in self.expr() { + let Some(new_expr) = update_expr(&sort.expr, projection.expr(), false)? + else { + return Ok(None); + }; + updated_exprs.push(PhysicalSortExpr { + expr: new_expr, + options: sort.options, + }); + } + + Ok(Some(Arc::new( + SortExec::new(updated_exprs, make_with_child(projection, self.input())?) + .with_fetch(self.fetch()) + .with_preserve_partitioning(self.preserve_partitioning()), + ))) + } } #[cfg(test)] diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index adcb28e538fd..fccc83cdddf4 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -23,6 +23,8 @@ use std::sync::Arc; use crate::common::spawn_buffered; use crate::limit::LimitStream; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use crate::projection::ProjectionExec; +use crate::projection_utils::{make_with_child, update_expr}; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, @@ -33,6 +35,7 @@ use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; +use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use log::{debug, trace}; @@ -334,6 +337,39 @@ impl ExecutionPlan for SortPreservingMergeExec { fn supports_limit_pushdown(&self) -> bool { true } + + /// Tries to swap the projection with its input [`SortPreservingMergeExec`]. + /// If this is possible, it returns the new [`SortPreservingMergeExec`] whose + /// child is a projection. Otherwise, it returns None. + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + // If the projection does not narrow the schema, we should not try to push it down. + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + + let mut updated_exprs = LexOrdering::default(); + for sort in self.expr() { + let Some(updated_expr) = update_expr(&sort.expr, projection.expr(), false)? + else { + return Ok(None); + }; + updated_exprs.push(PhysicalSortExpr { + expr: updated_expr, + options: sort.options, + }); + } + + Ok(Some(Arc::new( + SortPreservingMergeExec::new( + updated_exprs, + make_with_child(projection, self.input())?, + ) + .with_fetch(self.fetch()), + ))) + } } #[cfg(test)] diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index da8b0e877dcc..afd14b241799 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -24,6 +24,10 @@ use std::sync::Arc; use super::{DisplayAs, DisplayFormatType, PlanProperties}; use crate::display::{display_orderings, ProjectSchemaDisplay}; use crate::execution_plan::{Boundedness, EmissionType}; +use crate::projection::ProjectionExec; +use crate::projection_utils::{ + all_alias_free_columns, new_projections_for_columns, update_expr, +}; use crate::stream::RecordBatchStreamAdapter; use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; @@ -31,7 +35,7 @@ use arrow::datatypes::SchemaRef; use arrow_schema::Schema; use datafusion_common::{internal_err, plan_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; use crate::limit::LimitStream; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; @@ -272,6 +276,53 @@ impl ExecutionPlan for StreamingTableExec { }) } + /// Tries to embed `projection` to its input (`streaming table`). + /// If possible, returns [`StreamingTableExec`] as the top plan. Otherwise, + /// returns `None`. + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + if !all_alias_free_columns(projection.expr()) { + return Ok(None); + } + + let streaming_table_projections = + self.projection().as_ref().map(|i| i.as_ref().to_vec()); + let new_projections = new_projections_for_columns( + projection, + &streaming_table_projections + .unwrap_or((0..self.schema().fields().len()).collect()), + ); + + let mut lex_orderings = vec![]; + for lex_ordering in self.projected_output_ordering().into_iter() { + let mut orderings = LexOrdering::default(); + for order in lex_ordering { + let Some(new_ordering) = + update_expr(&order.expr, projection.expr(), false)? + else { + return Ok(None); + }; + orderings.push(PhysicalSortExpr { + expr: new_ordering, + options: order.options, + }); + } + lex_orderings.push(orderings); + } + + StreamingTableExec::try_new( + Arc::clone(self.partition_schema()), + self.partitions().clone(), + Some(new_projections.as_ref()), + lex_orderings, + self.is_infinite(), + self.limit(), + ) + .map(|e| Some(Arc::new(e) as _)) + } + fn metrics(&self) -> Option { Some(self.metrics.clone_inner()) } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index bcd9572f45c7..89d2603ec604 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -36,6 +36,8 @@ use crate::execution_plan::{ boundedness_from_children, emission_type_from_children, InvariantLevel, }; use crate::metrics::BaselineMetrics; +use crate::projection::ProjectionExec; +use crate::projection_utils::make_with_child; use crate::stream::ObservedStream; use arrow::datatypes::{Field, Schema, SchemaRef}; @@ -271,6 +273,27 @@ impl ExecutionPlan for UnionExec { fn supports_limit_pushdown(&self) -> bool { true } + + /// Tries to push `projection` down through `union`. If possible, performs the + /// pushdown and returns a new [`UnionExec`] as the top plan which has projections + /// as its children. Otherwise, returns `None`. + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + // If the projection doesn't narrow the schema, we shouldn't try to push it down. + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + + let new_children = self + .children() + .into_iter() + .map(|child| make_with_child(projection, child)) + .collect::>>()?; + + Ok(Some(Arc::new(UnionExec::new(new_children)))) + } } /// Combines multiple input streams by interleaving them. From 520dd34e91d35c1fa13e3743dafed661f3b3da76 Mon Sep 17 00:00:00 2001 From: buraksenn Date: Wed, 22 Jan 2025 23:24:53 +0300 Subject: [PATCH 02/12] wrap up rest of them --- .../physical_optimizer/projection_pushdown.rs | 901 +----------------- .../physical-plan/src/coalesce_partitions.rs | 17 + datafusion/physical-plan/src/filter.rs | 27 +- .../physical-plan/src/joins/cross_join.rs | 44 + .../physical-plan/src/joins/hash_join.rs | 50 + .../src/joins/nested_loop_join.rs | 47 + .../src/joins/sort_merge_join.rs | 59 ++ .../src/joins/symmetric_hash_join.rs | 79 ++ .../physical-plan/src/projection_utils.rs | 516 +++++++++- 9 files changed, 843 insertions(+), 897 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 956c73dd069c..7c3e90759fcc 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -24,31 +24,18 @@ use std::collections::HashMap; use std::sync::Arc; use crate::error::Result; -use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use crate::physical_plan::filter::FilterExec; -use crate::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; -use crate::physical_plan::joins::{ - CrossJoinExec, HashJoinExec, NestedLoopJoinExec, SortMergeJoinExec, - SymmetricHashJoinExec, -}; - use crate::physical_plan::projection::ProjectionExec; +use crate::physical_plan::ExecutionPlan; -use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; - -use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; -use datafusion_common::{internal_err, JoinSide}; use datafusion_physical_expr::expressions::{Column, Literal}; -use datafusion_physical_expr::{utils::collect_columns, PhysicalExpr, PhysicalExprRef}; -use datafusion_physical_plan::joins::utils::{JoinOn, JoinOnRef}; +use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_optimizer::PhysicalOptimizerRule; -use itertools::Itertools; +use datafusion_physical_plan::projection_utils::update_expr; /// This rule inspects [`ProjectionExec`]'s in the given physical plan and tries to /// remove or swap with its child. @@ -107,25 +94,6 @@ pub fn remove_unnecessary_projections( } else { Ok(Transformed::no(plan)) }; - } else if input.is::() { - try_swapping_with_coalesce_partitions(projection)? - } else if let Some(filter) = input.downcast_ref::() { - filter - .try_swapping_with_projection(projection)? - .map_or_else( - || try_embed_projection(projection, filter), - |e| Ok(Some(e)), - )? - } else if let Some(hash_join) = input.downcast_ref::() { - try_pushdown_through_hash_join(projection, hash_join)? - } else if let Some(cross_join) = input.downcast_ref::() { - try_swapping_with_cross_join(projection, cross_join)? - } else if let Some(nl_join) = input.downcast_ref::() { - try_pushdown_through_nested_loop_join(projection, nl_join)? - } else if let Some(sm_join) = input.downcast_ref::() { - try_swapping_with_sort_merge_join(projection, sm_join)? - } else if let Some(sym_join) = input.downcast_ref::() { - try_swapping_with_sym_hash_join(projection, sym_join)? } else { projection .input() @@ -190,434 +158,6 @@ fn is_expr_trivial(expr: &Arc) -> bool { || expr.as_any().downcast_ref::().is_some() } -/// Tries to swap `projection` with its input, which is known to be a -/// [`CoalescePartitionsExec`]. If possible, performs the swap and returns -/// [`CoalescePartitionsExec`] as the top plan. Otherwise, returns `None`. -fn try_swapping_with_coalesce_partitions( - projection: &ProjectionExec, -) -> Result>> { - // If the projection does not narrow the schema, we should not try to push it down: - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - // CoalescePartitionsExec always has a single child, so zero indexing is safe. - make_with_child(projection, projection.input().children()[0]) - .map(|e| Some(Arc::new(CoalescePartitionsExec::new(e)) as _)) -} - -trait EmbeddedProjection: ExecutionPlan + Sized { - fn with_projection(&self, projection: Option>) -> Result; -} - -impl EmbeddedProjection for HashJoinExec { - fn with_projection(&self, projection: Option>) -> Result { - self.with_projection(projection) - } -} - -impl EmbeddedProjection for NestedLoopJoinExec { - fn with_projection(&self, projection: Option>) -> Result { - self.with_projection(projection) - } -} - -impl EmbeddedProjection for FilterExec { - fn with_projection(&self, projection: Option>) -> Result { - self.with_projection(projection) - } -} - -/// Some projection can't be pushed down left input or right input of hash join because filter or on need may need some columns that won't be used in later. -/// By embed those projection to hash join, we can reduce the cost of build_batch_from_indices in hash join (build_batch_from_indices need to can compute::take() for each column) and avoid unnecessary output creation. -fn try_embed_projection( - projection: &ProjectionExec, - execution_plan: &Exec, -) -> Result>> { - // Collect all column indices from the given projection expressions. - let projection_index = collect_column_indices(projection.expr()); - - if projection_index.is_empty() { - return Ok(None); - }; - - // If the projection indices is the same as the input columns, we don't need to embed the projection to hash join. - // Check the projection_index is 0..n-1 and the length of projection_index is the same as the length of execution_plan schema fields. - if projection_index.len() == projection_index.last().unwrap() + 1 - && projection_index.len() == execution_plan.schema().fields().len() - { - return Ok(None); - } - - let new_execution_plan = - Arc::new(execution_plan.with_projection(Some(projection_index.to_vec()))?); - - // Build projection expressions for update_expr. Zip the projection_index with the new_execution_plan output schema fields. - let embed_project_exprs = projection_index - .iter() - .zip(new_execution_plan.schema().fields()) - .map(|(index, field)| { - ( - Arc::new(Column::new(field.name(), *index)) as Arc, - field.name().to_owned(), - ) - }) - .collect::>(); - - let mut new_projection_exprs = Vec::with_capacity(projection.expr().len()); - - for (expr, alias) in projection.expr() { - // update column index for projection expression since the input schema has been changed. - let Some(expr) = update_expr(expr, embed_project_exprs.as_slice(), false)? else { - return Ok(None); - }; - new_projection_exprs.push((expr, alias.clone())); - } - // Old projection may contain some alias or expression such as `a + 1` and `CAST('true' AS BOOLEAN)`, but our projection_exprs in hash join just contain column, so we need to create the new projection to keep the original projection. - let new_projection = Arc::new(ProjectionExec::try_new( - new_projection_exprs, - Arc::clone(&new_execution_plan) as _, - )?); - if is_projection_removable(&new_projection) { - Ok(Some(new_execution_plan)) - } else { - Ok(Some(new_projection)) - } -} - -/// Collect all column indices from the given projection expressions. -fn collect_column_indices(exprs: &[(Arc, String)]) -> Vec { - // Collect indices and remove duplicates. - let mut indices = exprs - .iter() - .flat_map(|(expr, _)| collect_columns(expr)) - .map(|x| x.index()) - .collect::>() - .into_iter() - .collect::>(); - indices.sort(); - indices -} - -struct JoinData { - projected_left_child: ProjectionExec, - projected_right_child: ProjectionExec, - join_filter: Option, - join_on: JoinOn, -} - -fn try_pushdown_through_join( - projection: &ProjectionExec, - join_left: &Arc, - join_right: &Arc, - join_on: JoinOnRef, - schema: SchemaRef, - filter: Option<&JoinFilter>, -) -> Result> { - // Convert projected expressions to columns. We can not proceed if this is not possible. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = - join_table_borders(join_left.schema().fields().len(), &projection_as_columns); - - if !join_allows_pushdown( - &projection_as_columns, - &schema, - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let new_filter = if let Some(filter) = filter { - match update_join_filter( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - filter, - join_left.schema().fields().len(), - ) { - Some(updated_filter) => Some(updated_filter), - None => return Ok(None), - } - } else { - None - }; - - let Some(new_on) = update_join_on( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - join_on, - join_left.schema().fields().len(), - ) else { - return Ok(None); - }; - - let (new_left, new_right) = new_join_children( - &projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - join_left, - join_right, - )?; - - Ok(Some(JoinData { - projected_left_child: new_left, - projected_right_child: new_right, - join_filter: new_filter, - join_on: new_on, - })) -} - -/// Tries to push `projection` down through `nested_loop_join`. If possible, performs the -/// pushdown and returns a new [`NestedLoopJoinExec`] as the top plan which has projections -/// as its children. Otherwise, returns `None`. -fn try_pushdown_through_nested_loop_join( - projection: &ProjectionExec, - nl_join: &NestedLoopJoinExec, -) -> Result>> { - // TODO: currently if there is projection in NestedLoopJoinExec, we can't push down projection to left or right input. Maybe we can pushdown the mixed projection later. - if nl_join.contains_projection() { - return Ok(None); - } - - if let Some(JoinData { - projected_left_child, - projected_right_child, - join_filter, - .. - }) = try_pushdown_through_join( - projection, - nl_join.left(), - nl_join.right(), - &[], - nl_join.schema(), - nl_join.filter(), - )? { - Ok(Some(Arc::new(NestedLoopJoinExec::try_new( - Arc::new(projected_left_child), - Arc::new(projected_right_child), - join_filter, - nl_join.join_type(), - // Returned early if projection is not None - None, - )?))) - } else { - try_embed_projection(projection, nl_join) - } -} - -/// Tries to push `projection` down through `hash_join`. If possible, performs the -/// pushdown and returns a new [`HashJoinExec`] as the top plan which has projections -/// as its children. Otherwise, returns `None`. -fn try_pushdown_through_hash_join( - projection: &ProjectionExec, - hash_join: &HashJoinExec, -) -> Result>> { - // TODO: currently if there is projection in HashJoinExec, we can't push down projection to left or right input. Maybe we can pushdown the mixed projection later. - if hash_join.contains_projection() { - return Ok(None); - } - - if let Some(JoinData { - projected_left_child, - projected_right_child, - join_filter, - join_on, - }) = try_pushdown_through_join( - projection, - hash_join.left(), - hash_join.right(), - hash_join.on(), - hash_join.schema(), - hash_join.filter(), - )? { - Ok(Some(Arc::new(HashJoinExec::try_new( - Arc::new(projected_left_child), - Arc::new(projected_right_child), - join_on, - join_filter, - hash_join.join_type(), - // Returned early if projection is not None - None, - *hash_join.partition_mode(), - hash_join.null_equals_null, - )?))) - } else { - try_embed_projection(projection, hash_join) - } -} - -/// Tries to swap the projection with its input [`CrossJoinExec`]. If it can be done, -/// it returns the new swapped version having the [`CrossJoinExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_cross_join( - projection: &ProjectionExec, - cross_join: &CrossJoinExec, -) -> Result>> { - // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( - cross_join.left().schema().fields().len(), - &projection_as_columns, - ); - - if !join_allows_pushdown( - &projection_as_columns, - &cross_join.schema(), - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let (new_left, new_right) = new_join_children( - &projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - cross_join.left(), - cross_join.right(), - )?; - - Ok(Some(Arc::new(CrossJoinExec::new( - Arc::new(new_left), - Arc::new(new_right), - )))) -} - -/// Tries to swap the projection with its input [`SortMergeJoinExec`]. If it can be done, -/// it returns the new swapped version having the [`SortMergeJoinExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_sort_merge_join( - projection: &ProjectionExec, - sm_join: &SortMergeJoinExec, -) -> Result>> { - // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( - sm_join.left().schema().fields().len(), - &projection_as_columns, - ); - - if !join_allows_pushdown( - &projection_as_columns, - &sm_join.schema(), - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let Some(new_on) = update_join_on( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - sm_join.on(), - sm_join.left().schema().fields().len(), - ) else { - return Ok(None); - }; - - let (new_left, new_right) = new_join_children( - &projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - sm_join.children()[0], - sm_join.children()[1], - )?; - - Ok(Some(Arc::new(SortMergeJoinExec::try_new( - Arc::new(new_left), - Arc::new(new_right), - new_on, - sm_join.filter.clone(), - sm_join.join_type, - sm_join.sort_options.clone(), - sm_join.null_equals_null, - )?))) -} - -/// Tries to swap the projection with its input [`SymmetricHashJoinExec`]. If it can be done, -/// it returns the new swapped version having the [`SymmetricHashJoinExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_sym_hash_join( - projection: &ProjectionExec, - sym_join: &SymmetricHashJoinExec, -) -> Result>> { - // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( - sym_join.left().schema().fields().len(), - &projection_as_columns, - ); - - if !join_allows_pushdown( - &projection_as_columns, - &sym_join.schema(), - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let Some(new_on) = update_join_on( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - sym_join.on(), - sym_join.left().schema().fields().len(), - ) else { - return Ok(None); - }; - - let new_filter = if let Some(filter) = sym_join.filter() { - match update_join_filter( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - filter, - sym_join.left().schema().fields().len(), - ) { - Some(updated_filter) => Some(updated_filter), - None => return Ok(None), - } - } else { - None - }; - - let (new_left, new_right) = new_join_children( - &projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - sym_join.left(), - sym_join.right(), - )?; - - Ok(Some(Arc::new(SymmetricHashJoinExec::try_new( - Arc::new(new_left), - Arc::new(new_right), - new_on, - new_filter, - sym_join.join_type(), - sym_join.null_equals_null(), - sym_join - .right() - .output_ordering() - .map(|p| LexOrdering::new(p.to_vec())), - sym_join - .left() - .output_ordering() - .map(|p| LexOrdering::new(p.to_vec())), - sym_join.partition_mode(), - )?))) -} - /// Compare the inputs and outputs of the projection. All expressions must be /// columns without alias, and projection does not change the order of fields. /// For example, if the input schema is `a, b`, `SELECT a, b` is removable, @@ -632,357 +172,6 @@ fn is_projection_removable(projection: &ProjectionExec) -> bool { }) && exprs.len() == projection.input().schema().fields().len() } -/// The function operates in two modes: -/// -/// 1) When `sync_with_child` is `true`: -/// -/// The function updates the indices of `expr` if the expression resides -/// in the input plan. For instance, given the expressions `a@1 + b@2` -/// and `c@0` with the input schema `c@2, a@0, b@1`, the expressions are -/// updated to `a@0 + b@1` and `c@2`. -/// -/// 2) When `sync_with_child` is `false`: -/// -/// The function determines how the expression would be updated if a projection -/// was placed before the plan associated with the expression. If the expression -/// cannot be rewritten after the projection, it returns `None`. For example, -/// given the expressions `c@0`, `a@1` and `b@2`, and the [`ProjectionExec`] with -/// an output schema of `a, c_new`, then `c@0` becomes `c_new@1`, `a@1` becomes -/// `a@0`, but `b@2` results in `None` since the projection does not include `b`. -fn update_expr( - expr: &Arc, - projected_exprs: &[(Arc, String)], - sync_with_child: bool, -) -> Result>> { - #[derive(Debug, PartialEq)] - enum RewriteState { - /// The expression is unchanged. - Unchanged, - /// Some part of the expression has been rewritten - RewrittenValid, - /// Some part of the expression has been rewritten, but some column - /// references could not be. - RewrittenInvalid, - } - - let mut state = RewriteState::Unchanged; - - let new_expr = Arc::clone(expr) - .transform_up(|expr: Arc| { - if state == RewriteState::RewrittenInvalid { - return Ok(Transformed::no(expr)); - } - - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::no(expr)); - }; - if sync_with_child { - state = RewriteState::RewrittenValid; - // Update the index of `column`: - Ok(Transformed::yes(Arc::clone( - &projected_exprs[column.index()].0, - ))) - } else { - // default to invalid, in case we can't find the relevant column - state = RewriteState::RewrittenInvalid; - // Determine how to update `column` to accommodate `projected_exprs` - projected_exprs - .iter() - .enumerate() - .find_map(|(index, (projected_expr, alias))| { - projected_expr.as_any().downcast_ref::().and_then( - |projected_column| { - (column.name().eq(projected_column.name()) - && column.index() == projected_column.index()) - .then(|| { - state = RewriteState::RewrittenValid; - Arc::new(Column::new(alias, index)) as _ - }) - }, - ) - }) - .map_or_else( - || Ok(Transformed::no(expr)), - |c| Ok(Transformed::yes(c)), - ) - } - }) - .data(); - - new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) -} - -/// Creates a new [`ProjectionExec`] instance with the given child plan and -/// projected expressions. -fn make_with_child( - projection: &ProjectionExec, - child: &Arc, -) -> Result> { - ProjectionExec::try_new(projection.expr().to_vec(), Arc::clone(child)) - .map(|e| Arc::new(e) as _) -} - -/// Downcasts all the expressions in `exprs` to `Column`s. If any of the given -/// expressions is not a `Column`, returns `None`. -fn physical_to_column_exprs( - exprs: &[(Arc, String)], -) -> Option> { - exprs - .iter() - .map(|(expr, alias)| { - expr.as_any() - .downcast_ref::() - .map(|col| (col.clone(), alias.clone())) - }) - .collect() -} - -/// Returns the last index before encountering a column coming from the right table when traveling -/// through the projection from left to right, and the last index before encountering a column -/// coming from the left table when traveling through the projection from right to left. -/// If there is no column in the projection coming from the left side, it returns (-1, ...), -/// if there is no column in the projection coming from the right side, it returns (..., projection length). -fn join_table_borders( - left_table_column_count: usize, - projection_as_columns: &[(Column, String)], -) -> (i32, i32) { - let far_right_left_col_ind = projection_as_columns - .iter() - .enumerate() - .take_while(|(_, (projection_column, _))| { - projection_column.index() < left_table_column_count - }) - .last() - .map(|(index, _)| index as i32) - .unwrap_or(-1); - - let far_left_right_col_ind = projection_as_columns - .iter() - .enumerate() - .rev() - .take_while(|(_, (projection_column, _))| { - projection_column.index() >= left_table_column_count - }) - .last() - .map(|(index, _)| index as i32) - .unwrap_or(projection_as_columns.len() as i32); - - (far_right_left_col_ind, far_left_right_col_ind) -} - -/// Tries to update the equi-join `Column`'s of a join as if the input of -/// the join was replaced by a projection. -fn update_join_on( - proj_left_exprs: &[(Column, String)], - proj_right_exprs: &[(Column, String)], - hash_join_on: &[(PhysicalExprRef, PhysicalExprRef)], - left_field_size: usize, -) -> Option> { - // TODO: Clippy wants the "map" call removed, but doing so generates - // a compilation error. Remove the clippy directive once this - // issue is fixed. - #[allow(clippy::map_identity)] - let (left_idx, right_idx): (Vec<_>, Vec<_>) = hash_join_on - .iter() - .map(|(left, right)| (left, right)) - .unzip(); - - let new_left_columns = new_columns_for_join_on(&left_idx, proj_left_exprs, 0); - let new_right_columns = - new_columns_for_join_on(&right_idx, proj_right_exprs, left_field_size); - - match (new_left_columns, new_right_columns) { - (Some(left), Some(right)) => Some(left.into_iter().zip(right).collect()), - _ => None, - } -} - -/// This function generates a new set of columns to be used in a hash join -/// operation based on a set of equi-join conditions (`hash_join_on`) and a -/// list of projection expressions (`projection_exprs`). -/// -/// Notes: Column indices in the projection expressions are based on the join schema, -/// whereas the join on expressions are based on the join child schema. `column_index_offset` -/// represents the offset between them. -fn new_columns_for_join_on( - hash_join_on: &[&PhysicalExprRef], - projection_exprs: &[(Column, String)], - column_index_offset: usize, -) -> Option> { - let new_columns = hash_join_on - .iter() - .filter_map(|on| { - // Rewrite all columns in `on` - Arc::clone(*on) - .transform(|expr| { - if let Some(column) = expr.as_any().downcast_ref::() { - // Find the column in the projection expressions - let new_column = projection_exprs - .iter() - .enumerate() - .find(|(_, (proj_column, _))| { - column.name() == proj_column.name() - && column.index() + column_index_offset - == proj_column.index() - }) - .map(|(index, (_, alias))| Column::new(alias, index)); - if let Some(new_column) = new_column { - Ok(Transformed::yes(Arc::new(new_column))) - } else { - // If the column is not found in the projection expressions, - // it means that the column is not projected. In this case, - // we cannot push the projection down. - internal_err!( - "Column {:?} not found in projection expressions", - column - ) - } - } else { - Ok(Transformed::no(expr)) - } - }) - .data() - .ok() - }) - .collect::>(); - (new_columns.len() == hash_join_on.len()).then_some(new_columns) -} - -/// Tries to update the column indices of a [`JoinFilter`] as if the input of -/// the join was replaced by a projection. -fn update_join_filter( - projection_left_exprs: &[(Column, String)], - projection_right_exprs: &[(Column, String)], - join_filter: &JoinFilter, - left_field_size: usize, -) -> Option { - let mut new_left_indices = new_indices_for_join_filter( - join_filter, - JoinSide::Left, - projection_left_exprs, - 0, - ) - .into_iter(); - let mut new_right_indices = new_indices_for_join_filter( - join_filter, - JoinSide::Right, - projection_right_exprs, - left_field_size, - ) - .into_iter(); - - // Check if all columns match: - (new_right_indices.len() + new_left_indices.len() - == join_filter.column_indices().len()) - .then(|| { - JoinFilter::new( - Arc::clone(join_filter.expression()), - join_filter - .column_indices() - .iter() - .map(|col_idx| ColumnIndex { - index: if col_idx.side == JoinSide::Left { - new_left_indices.next().unwrap() - } else { - new_right_indices.next().unwrap() - }, - side: col_idx.side, - }) - .collect(), - Arc::clone(join_filter.schema()), - ) - }) -} - -/// This function determines and returns a vector of indices representing the -/// positions of columns in `projection_exprs` that are involved in `join_filter`, -/// and correspond to a particular side (`join_side`) of the join operation. -/// -/// Notes: Column indices in the projection expressions are based on the join schema, -/// whereas the join filter is based on the join child schema. `column_index_offset` -/// represents the offset between them. -fn new_indices_for_join_filter( - join_filter: &JoinFilter, - join_side: JoinSide, - projection_exprs: &[(Column, String)], - column_index_offset: usize, -) -> Vec { - join_filter - .column_indices() - .iter() - .filter(|col_idx| col_idx.side == join_side) - .filter_map(|col_idx| { - projection_exprs - .iter() - .position(|(col, _)| col_idx.index + column_index_offset == col.index()) - }) - .collect() -} - -/// Checks three conditions for pushing a projection down through a join: -/// - Projection must narrow the join output schema. -/// - Columns coming from left/right tables must be collected at the left/right -/// sides of the output table. -/// - Left or right table is not lost after the projection. -fn join_allows_pushdown( - projection_as_columns: &[(Column, String)], - join_schema: &SchemaRef, - far_right_left_col_ind: i32, - far_left_right_col_ind: i32, -) -> bool { - // Projection must narrow the join output: - projection_as_columns.len() < join_schema.fields().len() - // Are the columns from different tables mixed? - && (far_right_left_col_ind + 1 == far_left_right_col_ind) - // Left or right table is not lost after the projection. - && far_right_left_col_ind >= 0 - && far_left_right_col_ind < projection_as_columns.len() as i32 -} - -/// If pushing down the projection over this join's children seems possible, -/// this function constructs the new [`ProjectionExec`]s that will come on top -/// of the original children of the join. -fn new_join_children( - projection_as_columns: &[(Column, String)], - far_right_left_col_ind: i32, - far_left_right_col_ind: i32, - left_child: &Arc, - right_child: &Arc, -) -> Result<(ProjectionExec, ProjectionExec)> { - let new_left = ProjectionExec::try_new( - projection_as_columns[0..=far_right_left_col_ind as _] - .iter() - .map(|(col, alias)| { - ( - Arc::new(Column::new(col.name(), col.index())) as _, - alias.clone(), - ) - }) - .collect_vec(), - Arc::clone(left_child), - )?; - let left_size = left_child.schema().fields().len() as i32; - let new_right = ProjectionExec::try_new( - projection_as_columns[far_left_right_col_ind as _..] - .iter() - .map(|(col, alias)| { - ( - Arc::new(Column::new( - col.name(), - // Align projected expressions coming from the right - // table with the new right child projection: - (col.index() as i32 - left_size) as _, - )) as _, - alias.clone(), - ) - }) - .collect_vec(), - Arc::clone(right_child), - )?; - - Ok((new_left, new_right)) -} - #[cfg(test)] mod tests { use super::*; @@ -999,6 +188,9 @@ mod tests { }; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; + use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; + use datafusion_physical_plan::filter::FilterExec; + use datafusion_physical_plan::joins::utils::{ColumnIndex, JoinFilter}; use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; @@ -1006,10 +198,12 @@ mod tests { use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::FileScanConfig; use crate::physical_plan::get_plan_string; - use crate::physical_plan::joins::StreamJoinPartitionMode; + use crate::physical_plan::joins::{ + HashJoinExec, NestedLoopJoinExec, StreamJoinPartitionMode, SymmetricHashJoinExec, + }; - use arrow_schema::{DataType, Field, Schema, SortOptions}; - use datafusion_common::{JoinType, ScalarValue}; + use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; + use datafusion_common::{JoinSide, JoinType, ScalarValue}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{ @@ -1022,6 +216,8 @@ mod tests { use datafusion_physical_plan::joins::PartitionMode; use datafusion_physical_plan::streaming::PartitionStream; + use itertools::Itertools; + /// Mocked UDF #[derive(Debug)] struct DummyUDF { @@ -1328,61 +524,6 @@ mod tests { Ok(()) } - #[test] - fn test_join_table_borders() -> Result<()> { - let projections = vec![ - (Column::new("b", 1), "b".to_owned()), - (Column::new("c", 2), "c".to_owned()), - (Column::new("e", 4), "e".to_owned()), - (Column::new("d", 3), "d".to_owned()), - (Column::new("c", 2), "c".to_owned()), - (Column::new("f", 5), "f".to_owned()), - (Column::new("h", 7), "h".to_owned()), - (Column::new("g", 6), "g".to_owned()), - ]; - let left_table_column_count = 5; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (4, 5) - ); - - let left_table_column_count = 8; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (7, 8) - ); - - let left_table_column_count = 1; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (-1, 0) - ); - - let projections = vec![ - (Column::new("a", 0), "a".to_owned()), - (Column::new("b", 1), "b".to_owned()), - (Column::new("d", 3), "d".to_owned()), - (Column::new("g", 6), "g".to_owned()), - (Column::new("e", 4), "e".to_owned()), - (Column::new("f", 5), "f".to_owned()), - (Column::new("e", 4), "e".to_owned()), - (Column::new("h", 7), "h".to_owned()), - ]; - let left_table_column_count = 5; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (2, 7) - ); - - let left_table_column_count = 7; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (6, 7) - ); - - Ok(()) - } - fn create_simple_csv_exec() -> Arc { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), @@ -2057,22 +1198,6 @@ mod tests { Ok(()) } - #[test] - fn test_collect_column_indices() -> Result<()> { - let expr = Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 7)), - Operator::Minus, - Arc::new(BinaryExpr::new( - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - Operator::Plus, - Arc::new(Column::new("a", 1)), - )), - )); - let column_indices = collect_column_indices(&[(expr, "b-(1+a)".to_string())]); - assert_eq!(column_indices, vec![1, 7]); - Ok(()) - } - #[test] fn test_nested_loop_join_after_projection() -> Result<()> { let left_csv = create_simple_csv_exec(); diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 7c1bdba2f339..71b7ca96e581 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -28,6 +28,7 @@ use super::{ Statistics, }; +use crate::projection_utils::make_with_child; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; use crate::execution_plan::CardinalityEffect; @@ -184,6 +185,22 @@ impl ExecutionPlan for CoalescePartitionsExec { fn cardinality_effect(&self) -> CardinalityEffect { CardinalityEffect::Equal } + + /// Tries to swap `projection` with its input, which is known to be a + /// [`CoalescePartitionsExec`]. If possible, performs the swap and returns + /// [`CoalescePartitionsExec`] as the top plan. Otherwise, returns `None`. + fn try_swapping_with_projection( + &self, + projection: &crate::projection::ProjectionExec, + ) -> Result>> { + // If the projection does not narrow the schema, we should not try to push it down: + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + // CoalescePartitionsExec always has a single child, so zero indexing is safe. + make_with_child(projection, projection.input().children()[0]) + .map(|e| Some(Arc::new(CoalescePartitionsExec::new(e)) as _)) + } } #[cfg(test)] diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index c06487f86501..d8ffac322642 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -26,7 +26,9 @@ use super::{ }; use crate::common::can_project; use crate::projection::ProjectionExec; -use crate::projection_utils::{make_with_child, update_expr}; +use crate::projection_utils::{ + make_with_child, try_embed_projection, update_expr, EmbeddedProjection, +}; use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, DisplayFormatType, ExecutionPlan, @@ -419,12 +421,23 @@ impl ExecutionPlan for FilterExec { return Ok(None); }; - FilterExec::try_new(new_predicate, make_with_child(projection, self.input())?) - .and_then(|e| { - let selectivity = self.default_selectivity(); - e.with_default_selectivity(selectivity) - }) - .map(|e| Some(Arc::new(e) as _)) + match FilterExec::try_new( + new_predicate, + make_with_child(projection, self.input())?, + ) + .and_then(|e| { + let selectivity = self.default_selectivity(); + e.with_default_selectivity(selectivity) + }) { + Ok(filter) => Ok(Some(Arc::new(filter) as _)), + Err(_) => try_embed_projection(projection, self), + } + } +} + +impl EmbeddedProjection for FilterExec { + fn with_projection(&self, projection: Option>) -> Result { + self.with_projection(projection) } } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 69300fce7745..bfa3bf282a0f 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -26,6 +26,9 @@ use super::utils::{ use crate::coalesce_partitions::CoalescePartitionsExec; use crate::execution_plan::{boundedness_from_children, EmissionType}; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use crate::projection_utils::{ + join_allows_pushdown, join_table_borders, new_join_children, physical_to_column_exprs, +}; use crate::{ handle_state, ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, PlanProperties, RecordBatchStream, @@ -335,6 +338,47 @@ impl ExecutionPlan for CrossJoinExec { self.right.statistics()?, )) } + + /// Tries to swap the projection with its input [`CrossJoinExec`]. If it can be done, + /// it returns the new swapped version having the [`CrossJoinExec`] as the top plan. + /// Otherwise, it returns None. + fn try_swapping_with_projection( + &self, + projection: &crate::projection::ProjectionExec, + ) -> Result>> { + // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. + let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) + else { + return Ok(None); + }; + + let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( + self.left().schema().fields().len(), + &projection_as_columns, + ); + + if !join_allows_pushdown( + &projection_as_columns, + &self.schema(), + far_right_left_col_ind, + far_left_right_col_ind, + ) { + return Ok(None); + } + + let (new_left, new_right) = new_join_children( + &projection_as_columns, + far_right_left_col_ind, + far_left_right_col_ind, + self.left(), + self.right(), + )?; + + Ok(Some(Arc::new(CrossJoinExec::new( + Arc::new(new_left), + Arc::new(new_right), + )))) + } } /// [left/right]_col_count are required in case the column statistics are None diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 4e224ea65d19..792ef7125d06 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -33,6 +33,9 @@ use super::{ PartitionMode, SharedBitmapBuilder, }; use crate::execution_plan::{boundedness_from_children, EmissionType}; +use crate::projection_utils::{ + try_embed_projection, try_pushdown_through_join, EmbeddedProjection, JoinData, +}; use crate::ExecutionPlanProperties; use crate::{ coalesce_partitions::CoalescePartitionsExec, @@ -864,6 +867,47 @@ impl ExecutionPlan for HashJoinExec { // Project statistics if there is a projection Ok(stats.project(self.projection.as_ref())) } + + /// Tries to push `projection` down through `hash_join`. If possible, performs the + /// pushdown and returns a new [`HashJoinExec`] as the top plan which has projections + /// as its children. Otherwise, returns `None`. + fn try_swapping_with_projection( + &self, + projection: &crate::projection::ProjectionExec, + ) -> Result>> { + // TODO: currently if there is projection in HashJoinExec, we can't push down projection to left or right input. Maybe we can pushdown the mixed projection later. + if self.contains_projection() { + return Ok(None); + } + + if let Some(JoinData { + projected_left_child, + projected_right_child, + join_filter, + join_on, + }) = try_pushdown_through_join( + projection, + self.left(), + self.right(), + self.on(), + self.schema(), + self.filter(), + )? { + Ok(Some(Arc::new(HashJoinExec::try_new( + Arc::new(projected_left_child), + Arc::new(projected_right_child), + join_on, + join_filter, + self.join_type(), + // Returned early if projection is not None + None, + *self.partition_mode(), + self.null_equals_null, + )?))) + } else { + try_embed_projection(projection, self) + } + } } /// Reads the left (build) side of the input, buffering it in memory, to build a @@ -1584,6 +1628,12 @@ impl Stream for HashJoinStream { } } +impl EmbeddedProjection for HashJoinExec { + fn with_projection(&self, projection: Option>) -> Result { + self.with_projection(projection) + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index d2fa2fdc7b4d..c0046d1a8572 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -37,6 +37,9 @@ use crate::joins::utils::{ BuildProbeJoinMetrics, ColumnIndex, JoinFilter, OnceAsync, OnceFut, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use crate::projection_utils::{ + try_embed_projection, try_pushdown_through_join, EmbeddedProjection, JoinData, +}; use crate::{ handle_state, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, PlanProperties, RecordBatchStream, @@ -555,6 +558,44 @@ impl ExecutionPlan for NestedLoopJoinExec { &self.join_schema, ) } + + /// Tries to push `projection` down through `nested_loop_join`. If possible, performs the + /// pushdown and returns a new [`NestedLoopJoinExec`] as the top plan which has projections + /// as its children. Otherwise, returns `None`. + fn try_swapping_with_projection( + &self, + projection: &crate::projection::ProjectionExec, + ) -> Result>> { + // TODO: currently if there is projection in NestedLoopJoinExec, we can't push down projection to left or right input. Maybe we can pushdown the mixed projection later. + if self.contains_projection() { + return Ok(None); + } + + if let Some(JoinData { + projected_left_child, + projected_right_child, + join_filter, + .. + }) = try_pushdown_through_join( + projection, + self.left(), + self.right(), + &[], + self.schema(), + self.filter(), + )? { + Ok(Some(Arc::new(NestedLoopJoinExec::try_new( + Arc::new(projected_left_child), + Arc::new(projected_right_child), + join_filter, + self.join_type(), + // Returned early if projection is not None + None, + )?))) + } else { + try_embed_projection(projection, self) + } + } } /// Asynchronously collect input into a single batch, and creates `JoinLeftData` from it @@ -979,6 +1020,12 @@ impl RecordBatchStream for NestedLoopJoinStr } } +impl EmbeddedProjection for NestedLoopJoinExec { + fn with_projection(&self, projection: Option>) -> Result { + self.with_projection(projection) + } +} + #[cfg(test)] pub(crate) mod tests { use super::*; diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 650d4c1de149..274b69ac9d10 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -62,6 +62,10 @@ use crate::joins::utils::{ JoinOnRef, }; use crate::metrics::{Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; +use crate::projection_utils::{ + join_allows_pushdown, join_table_borders, new_join_children, + physical_to_column_exprs, update_join_on, +}; use crate::spill::spill_record_batches; use crate::{ metrics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, @@ -504,6 +508,61 @@ impl ExecutionPlan for SortMergeJoinExec { &self.schema, ) } + + /// Tries to swap the projection with its input [`SortMergeJoinExec`]. If it can be done, + /// it returns the new swapped version having the [`SortMergeJoinExec`] as the top plan. + /// Otherwise, it returns None. + fn try_swapping_with_projection( + &self, + projection: &crate::projection::ProjectionExec, + ) -> Result>> { + // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. + let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) + else { + return Ok(None); + }; + + let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( + self.left().schema().fields().len(), + &projection_as_columns, + ); + + if !join_allows_pushdown( + &projection_as_columns, + &self.schema(), + far_right_left_col_ind, + far_left_right_col_ind, + ) { + return Ok(None); + } + + let Some(new_on) = update_join_on( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + self.on(), + self.left().schema().fields().len(), + ) else { + return Ok(None); + }; + + let (new_left, new_right) = new_join_children( + &projection_as_columns, + far_right_left_col_ind, + far_left_right_col_ind, + self.children()[0], + self.children()[1], + )?; + + Ok(Some(Arc::new(SortMergeJoinExec::try_new( + Arc::new(new_left), + Arc::new(new_right), + new_on, + self.filter.clone(), + self.join_type, + self.sort_options.clone(), + self.null_equals_null, + )?))) + } } /// Metrics for SortMergeJoinExec diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index b050d3adfeb5..0719dde89a30 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -47,6 +47,10 @@ use crate::joins::utils::{ BatchTransformer, ColumnIndex, JoinFilter, JoinHashMapType, JoinOn, JoinOnRef, NoopBatchTransformer, StatefulStreamResult, }; +use crate::projection_utils::{ + join_allows_pushdown, join_table_borders, new_join_children, + physical_to_column_exprs, update_join_filter, update_join_on, +}; use crate::{ joins::StreamJoinPartitionMode, metrics::{ExecutionPlanMetricsSet, MetricsSet}, @@ -556,6 +560,81 @@ impl ExecutionPlan for SymmetricHashJoinExec { })) } } + + /// Tries to swap the projection with its input [`SymmetricHashJoinExec`]. If it can be done, + /// it returns the new swapped version having the [`SymmetricHashJoinExec`] as the top plan. + /// Otherwise, it returns None. + fn try_swapping_with_projection( + &self, + projection: &crate::projection::ProjectionExec, + ) -> Result>> { + // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. + let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) + else { + return Ok(None); + }; + + let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( + self.left().schema().fields().len(), + &projection_as_columns, + ); + + if !join_allows_pushdown( + &projection_as_columns, + &self.schema(), + far_right_left_col_ind, + far_left_right_col_ind, + ) { + return Ok(None); + } + + let Some(new_on) = update_join_on( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + self.on(), + self.left().schema().fields().len(), + ) else { + return Ok(None); + }; + + let new_filter = if let Some(filter) = self.filter() { + match update_join_filter( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + filter, + self.left().schema().fields().len(), + ) { + Some(updated_filter) => Some(updated_filter), + None => return Ok(None), + } + } else { + None + }; + + let (new_left, new_right) = new_join_children( + &projection_as_columns, + far_right_left_col_ind, + far_left_right_col_ind, + self.left(), + self.right(), + )?; + + Ok(Some(Arc::new(SymmetricHashJoinExec::try_new( + Arc::new(new_left), + Arc::new(new_right), + new_on, + new_filter, + self.join_type(), + self.null_equals_null(), + self.right() + .output_ordering() + .map(|p| LexOrdering::new(p.to_vec())), + self.left() + .output_ordering() + .map(|p| LexOrdering::new(p.to_vec())), + self.partition_mode(), + )?))) + } } /// A stream that issues [RecordBatch]es as they arrive from the right of the join. diff --git a/datafusion/physical-plan/src/projection_utils.rs b/datafusion/physical-plan/src/projection_utils.rs index b1f4e741ad04..b468796ecc35 100644 --- a/datafusion/physical-plan/src/projection_utils.rs +++ b/datafusion/physical-plan/src/projection_utils.rs @@ -17,13 +17,18 @@ use std::sync::Arc; +use crate::joins::utils::{ColumnIndex, JoinFilter}; use crate::projection::ProjectionExec; use super::ExecutionPlan; +use arrow_schema::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::Result; +use datafusion_common::{internal_err, JoinSide, Result}; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr::utils::collect_columns; +use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef}; + +use itertools::Itertools; /// Given the expression set of a projection, checks if the projection causes /// any renaming or constructs a non-`Column` physical expression. @@ -148,3 +153,510 @@ pub fn update_expr( new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) } + +/// The on clause of the join, as vector of (left, right) columns. +pub type JoinOn = Vec<(PhysicalExprRef, PhysicalExprRef)>; +/// Reference for JoinOn. +pub type JoinOnRef<'a> = &'a [(PhysicalExprRef, PhysicalExprRef)]; + +pub struct JoinData { + pub projected_left_child: ProjectionExec, + pub projected_right_child: ProjectionExec, + pub join_filter: Option, + pub join_on: JoinOn, +} + +/// Downcasts all the expressions in `exprs` to `Column`s. If any of the given +/// expressions is not a `Column`, returns `None`. +pub fn physical_to_column_exprs( + exprs: &[(Arc, String)], +) -> Option> { + exprs + .iter() + .map(|(expr, alias)| { + expr.as_any() + .downcast_ref::() + .map(|col| (col.clone(), alias.clone())) + }) + .collect() +} + +/// Some projection can't be pushed down left input or right input of hash join because filter or on need may need some columns that won't be used in later. +/// By embed those projection to hash join, we can reduce the cost of build_batch_from_indices in hash join (build_batch_from_indices need to can compute::take() for each column) and avoid unnecessary output creation. +pub fn try_embed_projection( + projection: &ProjectionExec, + execution_plan: &Exec, +) -> Result>> { + // Collect all column indices from the given projection expressions. + let projection_index = collect_column_indices(projection.expr()); + + if projection_index.is_empty() { + return Ok(None); + }; + + // If the projection indices is the same as the input columns, we don't need to embed the projection to hash join. + // Check the projection_index is 0..n-1 and the length of projection_index is the same as the length of execution_plan schema fields. + if projection_index.len() == projection_index.last().unwrap() + 1 + && projection_index.len() == execution_plan.schema().fields().len() + { + return Ok(None); + } + + let new_execution_plan = + Arc::new(execution_plan.with_projection(Some(projection_index.to_vec()))?); + + // Build projection expressions for update_expr. Zip the projection_index with the new_execution_plan output schema fields. + let embed_project_exprs = projection_index + .iter() + .zip(new_execution_plan.schema().fields()) + .map(|(index, field)| { + ( + Arc::new(Column::new(field.name(), *index)) as Arc, + field.name().to_owned(), + ) + }) + .collect::>(); + + let mut new_projection_exprs = Vec::with_capacity(projection.expr().len()); + + for (expr, alias) in projection.expr() { + // update column index for projection expression since the input schema has been changed. + let Some(expr) = update_expr(expr, embed_project_exprs.as_slice(), false)? else { + return Ok(None); + }; + new_projection_exprs.push((expr, alias.clone())); + } + // Old projection may contain some alias or expression such as `a + 1` and `CAST('true' AS BOOLEAN)`, but our projection_exprs in hash join just contain column, so we need to create the new projection to keep the original projection. + let new_projection = Arc::new(ProjectionExec::try_new( + new_projection_exprs, + Arc::clone(&new_execution_plan) as _, + )?); + if is_projection_removable(&new_projection) { + Ok(Some(new_execution_plan)) + } else { + Ok(Some(new_projection)) + } +} + +pub fn try_pushdown_through_join( + projection: &ProjectionExec, + join_left: &Arc, + join_right: &Arc, + join_on: JoinOnRef, + schema: SchemaRef, + filter: Option<&JoinFilter>, +) -> Result> { + // Convert projected expressions to columns. We can not proceed if this is not possible. + let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { + return Ok(None); + }; + + let (far_right_left_col_ind, far_left_right_col_ind) = + join_table_borders(join_left.schema().fields().len(), &projection_as_columns); + + if !join_allows_pushdown( + &projection_as_columns, + &schema, + far_right_left_col_ind, + far_left_right_col_ind, + ) { + return Ok(None); + } + + let new_filter = if let Some(filter) = filter { + match update_join_filter( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + filter, + join_left.schema().fields().len(), + ) { + Some(updated_filter) => Some(updated_filter), + None => return Ok(None), + } + } else { + None + }; + + let Some(new_on) = update_join_on( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + join_on, + join_left.schema().fields().len(), + ) else { + return Ok(None); + }; + + let (new_left, new_right) = new_join_children( + &projection_as_columns, + far_right_left_col_ind, + far_left_right_col_ind, + join_left, + join_right, + )?; + + Ok(Some(JoinData { + projected_left_child: new_left, + projected_right_child: new_right, + join_filter: new_filter, + join_on: new_on, + })) +} + +/// If pushing down the projection over this join's children seems possible, +/// this function constructs the new [`ProjectionExec`]s that will come on top +/// of the original children of the join. +pub fn new_join_children( + projection_as_columns: &[(Column, String)], + far_right_left_col_ind: i32, + far_left_right_col_ind: i32, + left_child: &Arc, + right_child: &Arc, +) -> Result<(ProjectionExec, ProjectionExec)> { + let new_left = ProjectionExec::try_new( + projection_as_columns[0..=far_right_left_col_ind as _] + .iter() + .map(|(col, alias)| { + ( + Arc::new(Column::new(col.name(), col.index())) as _, + alias.clone(), + ) + }) + .collect_vec(), + Arc::clone(left_child), + )?; + let left_size = left_child.schema().fields().len() as i32; + let new_right = ProjectionExec::try_new( + projection_as_columns[far_left_right_col_ind as _..] + .iter() + .map(|(col, alias)| { + ( + Arc::new(Column::new( + col.name(), + // Align projected expressions coming from the right + // table with the new right child projection: + (col.index() as i32 - left_size) as _, + )) as _, + alias.clone(), + ) + }) + .collect_vec(), + Arc::clone(right_child), + )?; + + Ok((new_left, new_right)) +} + +/// Checks three conditions for pushing a projection down through a join: +/// - Projection must narrow the join output schema. +/// - Columns coming from left/right tables must be collected at the left/right +/// sides of the output table. +/// - Left or right table is not lost after the projection. +pub fn join_allows_pushdown( + projection_as_columns: &[(Column, String)], + join_schema: &SchemaRef, + far_right_left_col_ind: i32, + far_left_right_col_ind: i32, +) -> bool { + // Projection must narrow the join output: + projection_as_columns.len() < join_schema.fields().len() + // Are the columns from different tables mixed? + && (far_right_left_col_ind + 1 == far_left_right_col_ind) + // Left or right table is not lost after the projection. + && far_right_left_col_ind >= 0 + && far_left_right_col_ind < projection_as_columns.len() as i32 +} + +/// Returns the last index before encountering a column coming from the right table when traveling +/// through the projection from left to right, and the last index before encountering a column +/// coming from the left table when traveling through the projection from right to left. +/// If there is no column in the projection coming from the left side, it returns (-1, ...), +/// if there is no column in the projection coming from the right side, it returns (..., projection length). +pub fn join_table_borders( + left_table_column_count: usize, + projection_as_columns: &[(Column, String)], +) -> (i32, i32) { + let far_right_left_col_ind = projection_as_columns + .iter() + .enumerate() + .take_while(|(_, (projection_column, _))| { + projection_column.index() < left_table_column_count + }) + .last() + .map(|(index, _)| index as i32) + .unwrap_or(-1); + + let far_left_right_col_ind = projection_as_columns + .iter() + .enumerate() + .rev() + .take_while(|(_, (projection_column, _))| { + projection_column.index() >= left_table_column_count + }) + .last() + .map(|(index, _)| index as i32) + .unwrap_or(projection_as_columns.len() as i32); + + (far_right_left_col_ind, far_left_right_col_ind) +} + +/// Tries to update the equi-join `Column`'s of a join as if the input of +/// the join was replaced by a projection. +pub fn update_join_on( + proj_left_exprs: &[(Column, String)], + proj_right_exprs: &[(Column, String)], + hash_join_on: &[(PhysicalExprRef, PhysicalExprRef)], + left_field_size: usize, +) -> Option> { + // TODO: Clippy wants the "map" call removed, but doing so generates + // a compilation error. Remove the clippy directive once this + // issue is fixed. + #[allow(clippy::map_identity)] + let (left_idx, right_idx): (Vec<_>, Vec<_>) = hash_join_on + .iter() + .map(|(left, right)| (left, right)) + .unzip(); + + let new_left_columns = new_columns_for_join_on(&left_idx, proj_left_exprs, 0); + let new_right_columns = + new_columns_for_join_on(&right_idx, proj_right_exprs, left_field_size); + + match (new_left_columns, new_right_columns) { + (Some(left), Some(right)) => Some(left.into_iter().zip(right).collect()), + _ => None, + } +} + +/// Tries to update the column indices of a [`JoinFilter`] as if the input of +/// the join was replaced by a projection. +pub fn update_join_filter( + projection_left_exprs: &[(Column, String)], + projection_right_exprs: &[(Column, String)], + join_filter: &JoinFilter, + left_field_size: usize, +) -> Option { + let mut new_left_indices = new_indices_for_join_filter( + join_filter, + JoinSide::Left, + projection_left_exprs, + 0, + ) + .into_iter(); + let mut new_right_indices = new_indices_for_join_filter( + join_filter, + JoinSide::Right, + projection_right_exprs, + left_field_size, + ) + .into_iter(); + + // Check if all columns match: + (new_right_indices.len() + new_left_indices.len() + == join_filter.column_indices().len()) + .then(|| { + JoinFilter::new( + Arc::clone(join_filter.expression()), + join_filter + .column_indices() + .iter() + .map(|col_idx| ColumnIndex { + index: if col_idx.side == JoinSide::Left { + new_left_indices.next().unwrap() + } else { + new_right_indices.next().unwrap() + }, + side: col_idx.side, + }) + .collect(), + Arc::clone(join_filter.schema()), + ) + }) +} + +pub trait EmbeddedProjection: ExecutionPlan + Sized { + fn with_projection(&self, projection: Option>) -> Result; +} + +/// Collect all column indices from the given projection expressions. +fn collect_column_indices(exprs: &[(Arc, String)]) -> Vec { + // Collect indices and remove duplicates. + let mut indices = exprs + .iter() + .flat_map(|(expr, _)| collect_columns(expr)) + .map(|x| x.index()) + .collect::>() + .into_iter() + .collect::>(); + indices.sort(); + indices +} + +/// This function determines and returns a vector of indices representing the +/// positions of columns in `projection_exprs` that are involved in `join_filter`, +/// and correspond to a particular side (`join_side`) of the join operation. +/// +/// Notes: Column indices in the projection expressions are based on the join schema, +/// whereas the join filter is based on the join child schema. `column_index_offset` +/// represents the offset between them. +fn new_indices_for_join_filter( + join_filter: &JoinFilter, + join_side: JoinSide, + projection_exprs: &[(Column, String)], + column_index_offset: usize, +) -> Vec { + join_filter + .column_indices() + .iter() + .filter(|col_idx| col_idx.side == join_side) + .filter_map(|col_idx| { + projection_exprs + .iter() + .position(|(col, _)| col_idx.index + column_index_offset == col.index()) + }) + .collect() +} + +/// Compare the inputs and outputs of the projection. All expressions must be +/// columns without alias, and projection does not change the order of fields. +/// For example, if the input schema is `a, b`, `SELECT a, b` is removable, +/// but `SELECT b, a` and `SELECT a+1, b` and `SELECT a AS c, b` are not. +fn is_projection_removable(projection: &ProjectionExec) -> bool { + let exprs = projection.expr(); + exprs.iter().enumerate().all(|(idx, (expr, alias))| { + let Some(col) = expr.as_any().downcast_ref::() else { + return false; + }; + col.name() == alias && col.index() == idx + }) && exprs.len() == projection.input().schema().fields().len() +} + +/// This function generates a new set of columns to be used in a hash join +/// operation based on a set of equi-join conditions (`hash_join_on`) and a +/// list of projection expressions (`projection_exprs`). +/// +/// Notes: Column indices in the projection expressions are based on the join schema, +/// whereas the join on expressions are based on the join child schema. `column_index_offset` +/// represents the offset between them. +fn new_columns_for_join_on( + hash_join_on: &[&PhysicalExprRef], + projection_exprs: &[(Column, String)], + column_index_offset: usize, +) -> Option> { + let new_columns = hash_join_on + .iter() + .filter_map(|on| { + // Rewrite all columns in `on` + Arc::clone(*on) + .transform(|expr| { + if let Some(column) = expr.as_any().downcast_ref::() { + // Find the column in the projection expressions + let new_column = projection_exprs + .iter() + .enumerate() + .find(|(_, (proj_column, _))| { + column.name() == proj_column.name() + && column.index() + column_index_offset + == proj_column.index() + }) + .map(|(index, (_, alias))| Column::new(alias, index)); + if let Some(new_column) = new_column { + Ok(Transformed::yes(Arc::new(new_column))) + } else { + // If the column is not found in the projection expressions, + // it means that the column is not projected. In this case, + // we cannot push the projection down. + internal_err!( + "Column {:?} not found in projection expressions", + column + ) + } + } else { + Ok(Transformed::no(expr)) + } + }) + .data() + .ok() + }) + .collect::>(); + (new_columns.len() == hash_join_on.len()).then_some(new_columns) +} + +#[cfg(test)] +mod tests { + use super::*; + use std::sync::Arc; + + use datafusion_common::ScalarValue; + use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; + + #[test] + fn test_collect_column_indices() -> Result<()> { + let expr = Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 7)), + Operator::Minus, + Arc::new(BinaryExpr::new( + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), + Operator::Plus, + Arc::new(Column::new("a", 1)), + )), + )); + let column_indices = collect_column_indices(&[(expr, "b-(1+a)".to_string())]); + assert_eq!(column_indices, vec![1, 7]); + Ok(()) + } + + #[test] + fn test_join_table_borders() -> Result<()> { + let projections = vec![ + (Column::new("b", 1), "b".to_owned()), + (Column::new("c", 2), "c".to_owned()), + (Column::new("e", 4), "e".to_owned()), + (Column::new("d", 3), "d".to_owned()), + (Column::new("c", 2), "c".to_owned()), + (Column::new("f", 5), "f".to_owned()), + (Column::new("h", 7), "h".to_owned()), + (Column::new("g", 6), "g".to_owned()), + ]; + let left_table_column_count = 5; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (4, 5) + ); + + let left_table_column_count = 8; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (7, 8) + ); + + let left_table_column_count = 1; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (-1, 0) + ); + + let projections = vec![ + (Column::new("a", 0), "a".to_owned()), + (Column::new("b", 1), "b".to_owned()), + (Column::new("d", 3), "d".to_owned()), + (Column::new("g", 6), "g".to_owned()), + (Column::new("e", 4), "e".to_owned()), + (Column::new("f", 5), "f".to_owned()), + (Column::new("e", 4), "e".to_owned()), + (Column::new("h", 7), "h".to_owned()), + ]; + let left_table_column_count = 5; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (2, 7) + ); + + let left_table_column_count = 7; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (6, 7) + ); + + Ok(()) + } +} From c20f306a9d51f83e2b7de15187cc8ff3f6ba7fe0 Mon Sep 17 00:00:00 2001 From: buraksenn Date: Fri, 24 Jan 2025 01:16:42 +0300 Subject: [PATCH 03/12] refactor mroe --- .../core/src/datasource/physical_plan/csv.rs | 5 +- .../physical_optimizer/projection_pushdown.rs | 3 +- .../src/output_requirements.rs | 6 +- .../physical-plan/src/coalesce_partitions.rs | 4 +- datafusion/physical-plan/src/filter.rs | 22 +- .../physical-plan/src/joins/cross_join.rs | 7 +- .../physical-plan/src/joins/hash_join.rs | 5 +- .../src/joins/nested_loop_join.rs | 5 +- .../src/joins/sort_merge_join.rs | 6 +- .../src/joins/symmetric_hash_join.rs | 6 +- datafusion/physical-plan/src/lib.rs | 1 - datafusion/physical-plan/src/memory.rs | 5 +- datafusion/physical-plan/src/projection.rs | 645 ++++++++++++++++- .../physical-plan/src/projection_utils.rs | 662 ------------------ .../physical-plan/src/repartition/mod.rs | 3 +- datafusion/physical-plan/src/sorts/sort.rs | 3 +- .../src/sorts/sort_preserving_merge.rs | 3 +- datafusion/physical-plan/src/streaming.rs | 5 +- datafusion/physical-plan/src/union.rs | 3 +- 19 files changed, 682 insertions(+), 717 deletions(-) delete mode 100644 datafusion/physical-plan/src/projection_utils.rs diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 0635d5bfa023..55971f6f627c 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -45,9 +45,8 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::projection::ProjectionExec; -use datafusion_physical_plan::projection_utils::{ - all_alias_free_columns, new_projections_for_columns, +use datafusion_physical_plan::projection::{ + all_alias_free_columns, new_projections_for_columns, ProjectionExec, }; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 7c3e90759fcc..ac20875208d3 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -24,7 +24,6 @@ use std::collections::HashMap; use std::sync::Arc; use crate::error::Result; -use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::ExecutionPlan; use datafusion_common::config::ConfigOptions; @@ -35,7 +34,7 @@ use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_plan::projection_utils::update_expr; +use datafusion_physical_plan::projection::{update_expr, ProjectionExec}; /// This rule inspects [`ProjectionExec`]'s in the given physical plan and tries to /// remove or swap with its child. diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 0a294fa8b1a5..98962ced5661 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -25,7 +25,9 @@ use std::sync::Arc; use datafusion_execution::TaskContext; -use datafusion_physical_plan::projection_utils::{make_with_child, update_expr}; +use datafusion_physical_plan::projection::{ + make_with_child, update_expr, ProjectionExec, +}; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, @@ -196,7 +198,7 @@ impl ExecutionPlan for OutputRequirementExec { fn try_swapping_with_projection( &self, - projection: &datafusion_physical_plan::projection::ProjectionExec, + projection: &ProjectionExec, ) -> Result>> { // If the projection does not narrow the schema, we should not try to push it down: if projection.expr().len() >= projection.input().schema().fields().len() { diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 71b7ca96e581..61d6b8e0e865 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -28,7 +28,7 @@ use super::{ Statistics, }; -use crate::projection_utils::make_with_child; +use crate::projection::{make_with_child, ProjectionExec}; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; use crate::execution_plan::CardinalityEffect; @@ -191,7 +191,7 @@ impl ExecutionPlan for CoalescePartitionsExec { /// [`CoalescePartitionsExec`] as the top plan. Otherwise, returns `None`. fn try_swapping_with_projection( &self, - projection: &crate::projection::ProjectionExec, + projection: &ProjectionExec, ) -> Result>> { // If the projection does not narrow the schema, we should not try to push it down: if projection.expr().len() >= projection.input().schema().fields().len() { diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index d8ffac322642..b92255dbc4ad 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -25,9 +25,9 @@ use super::{ RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::common::can_project; -use crate::projection::ProjectionExec; -use crate::projection_utils::{ +use crate::projection::{ make_with_child, try_embed_projection, update_expr, EmbeddedProjection, + ProjectionExec, }; use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, @@ -421,17 +421,13 @@ impl ExecutionPlan for FilterExec { return Ok(None); }; - match FilterExec::try_new( - new_predicate, - make_with_child(projection, self.input())?, - ) - .and_then(|e| { - let selectivity = self.default_selectivity(); - e.with_default_selectivity(selectivity) - }) { - Ok(filter) => Ok(Some(Arc::new(filter) as _)), - Err(_) => try_embed_projection(projection, self), - } + FilterExec::try_new(new_predicate, make_with_child(projection, self.input())?) + .and_then(|e| { + let selectivity = self.default_selectivity(); + e.with_default_selectivity(selectivity) + }) + .map(|e| Some(Arc::new(e) as _))? + .map_or_else(|| try_embed_projection(projection, self), |e| Ok(Some(e))) } } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index bfa3bf282a0f..5a46897cb2b4 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -26,8 +26,9 @@ use super::utils::{ use crate::coalesce_partitions::CoalescePartitionsExec; use crate::execution_plan::{boundedness_from_children, EmissionType}; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::projection_utils::{ - join_allows_pushdown, join_table_borders, new_join_children, physical_to_column_exprs, +use crate::projection::{ + join_allows_pushdown, join_table_borders, new_join_children, + physical_to_column_exprs, ProjectionExec, }; use crate::{ handle_state, ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, @@ -344,7 +345,7 @@ impl ExecutionPlan for CrossJoinExec { /// Otherwise, it returns None. fn try_swapping_with_projection( &self, - projection: &crate::projection::ProjectionExec, + projection: &ProjectionExec, ) -> Result>> { // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 792ef7125d06..222dded4f13f 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -33,8 +33,9 @@ use super::{ PartitionMode, SharedBitmapBuilder, }; use crate::execution_plan::{boundedness_from_children, EmissionType}; -use crate::projection_utils::{ +use crate::projection::{ try_embed_projection, try_pushdown_through_join, EmbeddedProjection, JoinData, + ProjectionExec, }; use crate::ExecutionPlanProperties; use crate::{ @@ -873,7 +874,7 @@ impl ExecutionPlan for HashJoinExec { /// as its children. Otherwise, returns `None`. fn try_swapping_with_projection( &self, - projection: &crate::projection::ProjectionExec, + projection: &ProjectionExec, ) -> Result>> { // TODO: currently if there is projection in HashJoinExec, we can't push down projection to left or right input. Maybe we can pushdown the mixed projection later. if self.contains_projection() { diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index c0046d1a8572..0c4546d6c18c 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -37,8 +37,9 @@ use crate::joins::utils::{ BuildProbeJoinMetrics, ColumnIndex, JoinFilter, OnceAsync, OnceFut, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::projection_utils::{ +use crate::projection::{ try_embed_projection, try_pushdown_through_join, EmbeddedProjection, JoinData, + ProjectionExec, }; use crate::{ handle_state, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, @@ -564,7 +565,7 @@ impl ExecutionPlan for NestedLoopJoinExec { /// as its children. Otherwise, returns `None`. fn try_swapping_with_projection( &self, - projection: &crate::projection::ProjectionExec, + projection: &ProjectionExec, ) -> Result>> { // TODO: currently if there is projection in NestedLoopJoinExec, we can't push down projection to left or right input. Maybe we can pushdown the mixed projection later. if self.contains_projection() { diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 274b69ac9d10..efa8dcef50a8 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -62,9 +62,9 @@ use crate::joins::utils::{ JoinOnRef, }; use crate::metrics::{Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; -use crate::projection_utils::{ +use crate::projection::{ join_allows_pushdown, join_table_borders, new_join_children, - physical_to_column_exprs, update_join_on, + physical_to_column_exprs, update_join_on, ProjectionExec, }; use crate::spill::spill_record_batches; use crate::{ @@ -514,7 +514,7 @@ impl ExecutionPlan for SortMergeJoinExec { /// Otherwise, it returns None. fn try_swapping_with_projection( &self, - projection: &crate::projection::ProjectionExec, + projection: &ProjectionExec, ) -> Result>> { // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 0719dde89a30..7739f2481d77 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -47,9 +47,9 @@ use crate::joins::utils::{ BatchTransformer, ColumnIndex, JoinFilter, JoinHashMapType, JoinOn, JoinOnRef, NoopBatchTransformer, StatefulStreamResult, }; -use crate::projection_utils::{ +use crate::projection::{ join_allows_pushdown, join_table_borders, new_join_children, - physical_to_column_exprs, update_join_filter, update_join_on, + physical_to_column_exprs, update_join_filter, update_join_on, ProjectionExec, }; use crate::{ joins::StreamJoinPartitionMode, @@ -566,7 +566,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { /// Otherwise, it returns None. fn try_swapping_with_projection( &self, - projection: &crate::projection::ProjectionExec, + projection: &ProjectionExec, ) -> Result>> { // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index db75b3095fd7..5ad37f0b1ac0 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -67,7 +67,6 @@ pub mod memory; pub mod metrics; pub mod placeholder_row; pub mod projection; -pub mod projection_utils; pub mod recursive_query; pub mod repartition; pub mod sorts; diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 5f2596d2079e..b182aa927189 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -29,8 +29,9 @@ use super::{ Statistics, }; use crate::execution_plan::{Boundedness, EmissionType}; -use crate::projection::ProjectionExec; -use crate::projection_utils::{all_alias_free_columns, new_projections_for_columns}; +use crate::projection::{ + all_alias_free_columns, new_projections_for_columns, ProjectionExec, +}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index e37a6b0dfb85..2f4cd904dd6f 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -26,26 +26,33 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use super::expressions::Column; +use super::expressions::{CastExpr, Column, Literal}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{ DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; + +use crate::execution_plan::CardinalityEffect; +use crate::joins::utils::{ColumnIndex, JoinFilter}; use crate::{ColumnStatistics, DisplayFormatType, ExecutionPlan, PhysicalExpr}; -use arrow::datatypes::{Field, Schema, SchemaRef}; -use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::stats::Precision; -use datafusion_common::Result; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::{internal_err, JoinSide, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; -use datafusion_physical_expr::expressions::{CastExpr, Literal}; +use datafusion_physical_expr::utils::collect_columns; -use crate::execution_plan::CardinalityEffect; +use arrow::datatypes::{Field, Schema, SchemaRef}; +use arrow::record_batch::{RecordBatch, RecordBatchOptions}; + +use datafusion_physical_expr::PhysicalExprRef; use futures::stream::{Stream, StreamExt}; use log::trace; +use itertools::Itertools; + /// Execution plan for a projection #[derive(Debug, Clone)] pub struct ProjectionExec { @@ -353,15 +360,641 @@ impl RecordBatchStream for ProjectionStream { } } +/// Given the expression set of a projection, checks if the projection causes +/// any renaming or constructs a non-`Column` physical expression. +pub fn all_alias_free_columns(exprs: &[(Arc, String)]) -> bool { + exprs.iter().all(|(expr, alias)| { + expr.as_any() + .downcast_ref::() + .map(|column| column.name() == alias) + .unwrap_or(false) + }) +} + +/// Updates a source provider's projected columns according to the given +/// projection operator's expressions. To use this function safely, one must +/// ensure that all expressions are `Column` expressions without aliases. +pub fn new_projections_for_columns( + projection: &ProjectionExec, + source: &[usize], +) -> Vec { + projection + .expr() + .iter() + .filter_map(|(expr, _)| { + expr.as_any() + .downcast_ref::() + .map(|expr| source[expr.index()]) + }) + .collect() +} + +/// Creates a new [`ProjectionExec`] instance with the given child plan and +/// projected expressions. +pub fn make_with_child( + projection: &ProjectionExec, + child: &Arc, +) -> Result> { + ProjectionExec::try_new(projection.expr().to_vec(), Arc::clone(child)) + .map(|e| Arc::new(e) as _) +} + +/// Returns `true` if all the expressions in the argument are `Column`s. +pub fn all_columns(exprs: &[(Arc, String)]) -> bool { + exprs.iter().all(|(expr, _)| expr.as_any().is::()) +} + +/// The function operates in two modes: +/// +/// 1) When `sync_with_child` is `true`: +/// +/// The function updates the indices of `expr` if the expression resides +/// in the input plan. For instance, given the expressions `a@1 + b@2` +/// and `c@0` with the input schema `c@2, a@0, b@1`, the expressions are +/// updated to `a@0 + b@1` and `c@2`. +/// +/// 2) When `sync_with_child` is `false`: +/// +/// The function determines how the expression would be updated if a projection +/// was placed before the plan associated with the expression. If the expression +/// cannot be rewritten after the projection, it returns `None`. For example, +/// given the expressions `c@0`, `a@1` and `b@2`, and the [`ProjectionExec`] with +/// an output schema of `a, c_new`, then `c@0` becomes `c_new@1`, `a@1` becomes +/// `a@0`, but `b@2` results in `None` since the projection does not include `b`. +pub fn update_expr( + expr: &Arc, + projected_exprs: &[(Arc, String)], + sync_with_child: bool, +) -> Result>> { + #[derive(Debug, PartialEq)] + enum RewriteState { + /// The expression is unchanged. + Unchanged, + /// Some part of the expression has been rewritten + RewrittenValid, + /// Some part of the expression has been rewritten, but some column + /// references could not be. + RewrittenInvalid, + } + + let mut state = RewriteState::Unchanged; + + let new_expr = Arc::clone(expr) + .transform_up(|expr: Arc| { + if state == RewriteState::RewrittenInvalid { + return Ok(Transformed::no(expr)); + } + + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::no(expr)); + }; + if sync_with_child { + state = RewriteState::RewrittenValid; + // Update the index of `column`: + Ok(Transformed::yes(Arc::clone( + &projected_exprs[column.index()].0, + ))) + } else { + // default to invalid, in case we can't find the relevant column + state = RewriteState::RewrittenInvalid; + // Determine how to update `column` to accommodate `projected_exprs` + projected_exprs + .iter() + .enumerate() + .find_map(|(index, (projected_expr, alias))| { + projected_expr.as_any().downcast_ref::().and_then( + |projected_column| { + (column.name().eq(projected_column.name()) + && column.index() == projected_column.index()) + .then(|| { + state = RewriteState::RewrittenValid; + Arc::new(Column::new(alias, index)) as _ + }) + }, + ) + }) + .map_or_else( + || Ok(Transformed::no(expr)), + |c| Ok(Transformed::yes(c)), + ) + } + }) + .data(); + + new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) +} + +/// The on clause of the join, as vector of (left, right) columns. +pub type JoinOn = Vec<(PhysicalExprRef, PhysicalExprRef)>; +/// Reference for JoinOn. +pub type JoinOnRef<'a> = &'a [(PhysicalExprRef, PhysicalExprRef)]; + +pub struct JoinData { + pub projected_left_child: ProjectionExec, + pub projected_right_child: ProjectionExec, + pub join_filter: Option, + pub join_on: JoinOn, +} + +/// Downcasts all the expressions in `exprs` to `Column`s. If any of the given +/// expressions is not a `Column`, returns `None`. +pub fn physical_to_column_exprs( + exprs: &[(Arc, String)], +) -> Option> { + exprs + .iter() + .map(|(expr, alias)| { + expr.as_any() + .downcast_ref::() + .map(|col| (col.clone(), alias.clone())) + }) + .collect() +} + +/// Some projection can't be pushed down left input or right input of hash join because filter or on need may need some columns that won't be used in later. +/// By embed those projection to hash join, we can reduce the cost of build_batch_from_indices in hash join (build_batch_from_indices need to can compute::take() for each column) and avoid unnecessary output creation. +pub fn try_embed_projection( + projection: &ProjectionExec, + execution_plan: &Exec, +) -> Result>> { + // Collect all column indices from the given projection expressions. + let projection_index = collect_column_indices(projection.expr()); + + if projection_index.is_empty() { + return Ok(None); + }; + + // If the projection indices is the same as the input columns, we don't need to embed the projection to hash join. + // Check the projection_index is 0..n-1 and the length of projection_index is the same as the length of execution_plan schema fields. + if projection_index.len() == projection_index.last().unwrap() + 1 + && projection_index.len() == execution_plan.schema().fields().len() + { + return Ok(None); + } + + let new_execution_plan = + Arc::new(execution_plan.with_projection(Some(projection_index.to_vec()))?); + + // Build projection expressions for update_expr. Zip the projection_index with the new_execution_plan output schema fields. + let embed_project_exprs = projection_index + .iter() + .zip(new_execution_plan.schema().fields()) + .map(|(index, field)| { + ( + Arc::new(Column::new(field.name(), *index)) as Arc, + field.name().to_owned(), + ) + }) + .collect::>(); + + let mut new_projection_exprs = Vec::with_capacity(projection.expr().len()); + + for (expr, alias) in projection.expr() { + // update column index for projection expression since the input schema has been changed. + let Some(expr) = update_expr(expr, embed_project_exprs.as_slice(), false)? else { + return Ok(None); + }; + new_projection_exprs.push((expr, alias.clone())); + } + // Old projection may contain some alias or expression such as `a + 1` and `CAST('true' AS BOOLEAN)`, but our projection_exprs in hash join just contain column, so we need to create the new projection to keep the original projection. + let new_projection = Arc::new(ProjectionExec::try_new( + new_projection_exprs, + Arc::clone(&new_execution_plan) as _, + )?); + if is_projection_removable(&new_projection) { + Ok(Some(new_execution_plan)) + } else { + Ok(Some(new_projection)) + } +} + +pub fn try_pushdown_through_join( + projection: &ProjectionExec, + join_left: &Arc, + join_right: &Arc, + join_on: JoinOnRef, + schema: SchemaRef, + filter: Option<&JoinFilter>, +) -> Result> { + // Convert projected expressions to columns. We can not proceed if this is not possible. + let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { + return Ok(None); + }; + + let (far_right_left_col_ind, far_left_right_col_ind) = + join_table_borders(join_left.schema().fields().len(), &projection_as_columns); + + if !join_allows_pushdown( + &projection_as_columns, + &schema, + far_right_left_col_ind, + far_left_right_col_ind, + ) { + return Ok(None); + } + + let new_filter = if let Some(filter) = filter { + match update_join_filter( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + filter, + join_left.schema().fields().len(), + ) { + Some(updated_filter) => Some(updated_filter), + None => return Ok(None), + } + } else { + None + }; + + let Some(new_on) = update_join_on( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + join_on, + join_left.schema().fields().len(), + ) else { + return Ok(None); + }; + + let (new_left, new_right) = new_join_children( + &projection_as_columns, + far_right_left_col_ind, + far_left_right_col_ind, + join_left, + join_right, + )?; + + Ok(Some(JoinData { + projected_left_child: new_left, + projected_right_child: new_right, + join_filter: new_filter, + join_on: new_on, + })) +} + +/// If pushing down the projection over this join's children seems possible, +/// this function constructs the new [`ProjectionExec`]s that will come on top +/// of the original children of the join. +pub fn new_join_children( + projection_as_columns: &[(Column, String)], + far_right_left_col_ind: i32, + far_left_right_col_ind: i32, + left_child: &Arc, + right_child: &Arc, +) -> Result<(ProjectionExec, ProjectionExec)> { + let new_left = ProjectionExec::try_new( + projection_as_columns[0..=far_right_left_col_ind as _] + .iter() + .map(|(col, alias)| { + ( + Arc::new(Column::new(col.name(), col.index())) as _, + alias.clone(), + ) + }) + .collect_vec(), + Arc::clone(left_child), + )?; + let left_size = left_child.schema().fields().len() as i32; + let new_right = ProjectionExec::try_new( + projection_as_columns[far_left_right_col_ind as _..] + .iter() + .map(|(col, alias)| { + ( + Arc::new(Column::new( + col.name(), + // Align projected expressions coming from the right + // table with the new right child projection: + (col.index() as i32 - left_size) as _, + )) as _, + alias.clone(), + ) + }) + .collect_vec(), + Arc::clone(right_child), + )?; + + Ok((new_left, new_right)) +} + +/// Checks three conditions for pushing a projection down through a join: +/// - Projection must narrow the join output schema. +/// - Columns coming from left/right tables must be collected at the left/right +/// sides of the output table. +/// - Left or right table is not lost after the projection. +pub fn join_allows_pushdown( + projection_as_columns: &[(Column, String)], + join_schema: &SchemaRef, + far_right_left_col_ind: i32, + far_left_right_col_ind: i32, +) -> bool { + // Projection must narrow the join output: + projection_as_columns.len() < join_schema.fields().len() + // Are the columns from different tables mixed? + && (far_right_left_col_ind + 1 == far_left_right_col_ind) + // Left or right table is not lost after the projection. + && far_right_left_col_ind >= 0 + && far_left_right_col_ind < projection_as_columns.len() as i32 +} + +/// Returns the last index before encountering a column coming from the right table when traveling +/// through the projection from left to right, and the last index before encountering a column +/// coming from the left table when traveling through the projection from right to left. +/// If there is no column in the projection coming from the left side, it returns (-1, ...), +/// if there is no column in the projection coming from the right side, it returns (..., projection length). +pub fn join_table_borders( + left_table_column_count: usize, + projection_as_columns: &[(Column, String)], +) -> (i32, i32) { + let far_right_left_col_ind = projection_as_columns + .iter() + .enumerate() + .take_while(|(_, (projection_column, _))| { + projection_column.index() < left_table_column_count + }) + .last() + .map(|(index, _)| index as i32) + .unwrap_or(-1); + + let far_left_right_col_ind = projection_as_columns + .iter() + .enumerate() + .rev() + .take_while(|(_, (projection_column, _))| { + projection_column.index() >= left_table_column_count + }) + .last() + .map(|(index, _)| index as i32) + .unwrap_or(projection_as_columns.len() as i32); + + (far_right_left_col_ind, far_left_right_col_ind) +} + +/// Tries to update the equi-join `Column`'s of a join as if the input of +/// the join was replaced by a projection. +pub fn update_join_on( + proj_left_exprs: &[(Column, String)], + proj_right_exprs: &[(Column, String)], + hash_join_on: &[(PhysicalExprRef, PhysicalExprRef)], + left_field_size: usize, +) -> Option> { + // TODO: Clippy wants the "map" call removed, but doing so generates + // a compilation error. Remove the clippy directive once this + // issue is fixed. + #[allow(clippy::map_identity)] + let (left_idx, right_idx): (Vec<_>, Vec<_>) = hash_join_on + .iter() + .map(|(left, right)| (left, right)) + .unzip(); + + let new_left_columns = new_columns_for_join_on(&left_idx, proj_left_exprs, 0); + let new_right_columns = + new_columns_for_join_on(&right_idx, proj_right_exprs, left_field_size); + + match (new_left_columns, new_right_columns) { + (Some(left), Some(right)) => Some(left.into_iter().zip(right).collect()), + _ => None, + } +} + +/// Tries to update the column indices of a [`JoinFilter`] as if the input of +/// the join was replaced by a projection. +pub fn update_join_filter( + projection_left_exprs: &[(Column, String)], + projection_right_exprs: &[(Column, String)], + join_filter: &JoinFilter, + left_field_size: usize, +) -> Option { + let mut new_left_indices = new_indices_for_join_filter( + join_filter, + JoinSide::Left, + projection_left_exprs, + 0, + ) + .into_iter(); + let mut new_right_indices = new_indices_for_join_filter( + join_filter, + JoinSide::Right, + projection_right_exprs, + left_field_size, + ) + .into_iter(); + + // Check if all columns match: + (new_right_indices.len() + new_left_indices.len() + == join_filter.column_indices().len()) + .then(|| { + JoinFilter::new( + Arc::clone(join_filter.expression()), + join_filter + .column_indices() + .iter() + .map(|col_idx| ColumnIndex { + index: if col_idx.side == JoinSide::Left { + new_left_indices.next().unwrap() + } else { + new_right_indices.next().unwrap() + }, + side: col_idx.side, + }) + .collect(), + Arc::clone(join_filter.schema()), + ) + }) +} + +pub trait EmbeddedProjection: ExecutionPlan + Sized { + fn with_projection(&self, projection: Option>) -> Result; +} + +/// Collect all column indices from the given projection expressions. +fn collect_column_indices(exprs: &[(Arc, String)]) -> Vec { + // Collect indices and remove duplicates. + let mut indices = exprs + .iter() + .flat_map(|(expr, _)| collect_columns(expr)) + .map(|x| x.index()) + .collect::>() + .into_iter() + .collect::>(); + indices.sort(); + indices +} + +/// This function determines and returns a vector of indices representing the +/// positions of columns in `projection_exprs` that are involved in `join_filter`, +/// and correspond to a particular side (`join_side`) of the join operation. +/// +/// Notes: Column indices in the projection expressions are based on the join schema, +/// whereas the join filter is based on the join child schema. `column_index_offset` +/// represents the offset between them. +fn new_indices_for_join_filter( + join_filter: &JoinFilter, + join_side: JoinSide, + projection_exprs: &[(Column, String)], + column_index_offset: usize, +) -> Vec { + join_filter + .column_indices() + .iter() + .filter(|col_idx| col_idx.side == join_side) + .filter_map(|col_idx| { + projection_exprs + .iter() + .position(|(col, _)| col_idx.index + column_index_offset == col.index()) + }) + .collect() +} + +/// Compare the inputs and outputs of the projection. All expressions must be +/// columns without alias, and projection does not change the order of fields. +/// For example, if the input schema is `a, b`, `SELECT a, b` is removable, +/// but `SELECT b, a` and `SELECT a+1, b` and `SELECT a AS c, b` are not. +fn is_projection_removable(projection: &ProjectionExec) -> bool { + let exprs = projection.expr(); + exprs.iter().enumerate().all(|(idx, (expr, alias))| { + let Some(col) = expr.as_any().downcast_ref::() else { + return false; + }; + col.name() == alias && col.index() == idx + }) && exprs.len() == projection.input().schema().fields().len() +} + +/// This function generates a new set of columns to be used in a hash join +/// operation based on a set of equi-join conditions (`hash_join_on`) and a +/// list of projection expressions (`projection_exprs`). +/// +/// Notes: Column indices in the projection expressions are based on the join schema, +/// whereas the join on expressions are based on the join child schema. `column_index_offset` +/// represents the offset between them. +fn new_columns_for_join_on( + hash_join_on: &[&PhysicalExprRef], + projection_exprs: &[(Column, String)], + column_index_offset: usize, +) -> Option> { + let new_columns = hash_join_on + .iter() + .filter_map(|on| { + // Rewrite all columns in `on` + Arc::clone(*on) + .transform(|expr| { + if let Some(column) = expr.as_any().downcast_ref::() { + // Find the column in the projection expressions + let new_column = projection_exprs + .iter() + .enumerate() + .find(|(_, (proj_column, _))| { + column.name() == proj_column.name() + && column.index() + column_index_offset + == proj_column.index() + }) + .map(|(index, (_, alias))| Column::new(alias, index)); + if let Some(new_column) = new_column { + Ok(Transformed::yes(Arc::new(new_column))) + } else { + // If the column is not found in the projection expressions, + // it means that the column is not projected. In this case, + // we cannot push the projection down. + internal_err!( + "Column {:?} not found in projection expressions", + column + ) + } + } else { + Ok(Transformed::no(expr)) + } + }) + .data() + .ok() + }) + .collect::>(); + (new_columns.len() == hash_join_on.len()).then_some(new_columns) +} + #[cfg(test)] mod tests { use super::*; + use std::sync::Arc; + use crate::common::collect; use crate::test; use arrow_schema::DataType; use datafusion_common::ScalarValue; + use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; + + #[test] + fn test_collect_column_indices() -> Result<()> { + let expr = Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 7)), + Operator::Minus, + Arc::new(BinaryExpr::new( + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), + Operator::Plus, + Arc::new(Column::new("a", 1)), + )), + )); + let column_indices = collect_column_indices(&[(expr, "b-(1+a)".to_string())]); + assert_eq!(column_indices, vec![1, 7]); + Ok(()) + } + + #[test] + fn test_join_table_borders() -> Result<()> { + let projections = vec![ + (Column::new("b", 1), "b".to_owned()), + (Column::new("c", 2), "c".to_owned()), + (Column::new("e", 4), "e".to_owned()), + (Column::new("d", 3), "d".to_owned()), + (Column::new("c", 2), "c".to_owned()), + (Column::new("f", 5), "f".to_owned()), + (Column::new("h", 7), "h".to_owned()), + (Column::new("g", 6), "g".to_owned()), + ]; + let left_table_column_count = 5; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (4, 5) + ); + + let left_table_column_count = 8; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (7, 8) + ); + + let left_table_column_count = 1; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (-1, 0) + ); + + let projections = vec![ + (Column::new("a", 0), "a".to_owned()), + (Column::new("b", 1), "b".to_owned()), + (Column::new("d", 3), "d".to_owned()), + (Column::new("g", 6), "g".to_owned()), + (Column::new("e", 4), "e".to_owned()), + (Column::new("f", 5), "f".to_owned()), + (Column::new("e", 4), "e".to_owned()), + (Column::new("h", 7), "h".to_owned()), + ]; + let left_table_column_count = 5; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (2, 7) + ); + + let left_table_column_count = 7; + assert_eq!( + join_table_borders(left_table_column_count, &projections), + (6, 7) + ); + + Ok(()) + } + #[tokio::test] async fn project_no_column() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); diff --git a/datafusion/physical-plan/src/projection_utils.rs b/datafusion/physical-plan/src/projection_utils.rs deleted file mode 100644 index b468796ecc35..000000000000 --- a/datafusion/physical-plan/src/projection_utils.rs +++ /dev/null @@ -1,662 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::sync::Arc; - -use crate::joins::utils::{ColumnIndex, JoinFilter}; -use crate::projection::ProjectionExec; - -use super::ExecutionPlan; -use arrow_schema::SchemaRef; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::{internal_err, JoinSide, Result}; -use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::utils::collect_columns; -use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef}; - -use itertools::Itertools; - -/// Given the expression set of a projection, checks if the projection causes -/// any renaming or constructs a non-`Column` physical expression. -pub fn all_alias_free_columns(exprs: &[(Arc, String)]) -> bool { - exprs.iter().all(|(expr, alias)| { - expr.as_any() - .downcast_ref::() - .map(|column| column.name() == alias) - .unwrap_or(false) - }) -} - -/// Updates a source provider's projected columns according to the given -/// projection operator's expressions. To use this function safely, one must -/// ensure that all expressions are `Column` expressions without aliases. -pub fn new_projections_for_columns( - projection: &ProjectionExec, - source: &[usize], -) -> Vec { - projection - .expr() - .iter() - .filter_map(|(expr, _)| { - expr.as_any() - .downcast_ref::() - .map(|expr| source[expr.index()]) - }) - .collect() -} - -/// Creates a new [`ProjectionExec`] instance with the given child plan and -/// projected expressions. -pub fn make_with_child( - projection: &ProjectionExec, - child: &Arc, -) -> Result> { - ProjectionExec::try_new(projection.expr().to_vec(), Arc::clone(child)) - .map(|e| Arc::new(e) as _) -} - -/// Returns `true` if all the expressions in the argument are `Column`s. -pub fn all_columns(exprs: &[(Arc, String)]) -> bool { - exprs.iter().all(|(expr, _)| expr.as_any().is::()) -} - -/// The function operates in two modes: -/// -/// 1) When `sync_with_child` is `true`: -/// -/// The function updates the indices of `expr` if the expression resides -/// in the input plan. For instance, given the expressions `a@1 + b@2` -/// and `c@0` with the input schema `c@2, a@0, b@1`, the expressions are -/// updated to `a@0 + b@1` and `c@2`. -/// -/// 2) When `sync_with_child` is `false`: -/// -/// The function determines how the expression would be updated if a projection -/// was placed before the plan associated with the expression. If the expression -/// cannot be rewritten after the projection, it returns `None`. For example, -/// given the expressions `c@0`, `a@1` and `b@2`, and the [`ProjectionExec`] with -/// an output schema of `a, c_new`, then `c@0` becomes `c_new@1`, `a@1` becomes -/// `a@0`, but `b@2` results in `None` since the projection does not include `b`. -pub fn update_expr( - expr: &Arc, - projected_exprs: &[(Arc, String)], - sync_with_child: bool, -) -> Result>> { - #[derive(Debug, PartialEq)] - enum RewriteState { - /// The expression is unchanged. - Unchanged, - /// Some part of the expression has been rewritten - RewrittenValid, - /// Some part of the expression has been rewritten, but some column - /// references could not be. - RewrittenInvalid, - } - - let mut state = RewriteState::Unchanged; - - let new_expr = Arc::clone(expr) - .transform_up(|expr: Arc| { - if state == RewriteState::RewrittenInvalid { - return Ok(Transformed::no(expr)); - } - - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::no(expr)); - }; - if sync_with_child { - state = RewriteState::RewrittenValid; - // Update the index of `column`: - Ok(Transformed::yes(Arc::clone( - &projected_exprs[column.index()].0, - ))) - } else { - // default to invalid, in case we can't find the relevant column - state = RewriteState::RewrittenInvalid; - // Determine how to update `column` to accommodate `projected_exprs` - projected_exprs - .iter() - .enumerate() - .find_map(|(index, (projected_expr, alias))| { - projected_expr.as_any().downcast_ref::().and_then( - |projected_column| { - (column.name().eq(projected_column.name()) - && column.index() == projected_column.index()) - .then(|| { - state = RewriteState::RewrittenValid; - Arc::new(Column::new(alias, index)) as _ - }) - }, - ) - }) - .map_or_else( - || Ok(Transformed::no(expr)), - |c| Ok(Transformed::yes(c)), - ) - } - }) - .data(); - - new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) -} - -/// The on clause of the join, as vector of (left, right) columns. -pub type JoinOn = Vec<(PhysicalExprRef, PhysicalExprRef)>; -/// Reference for JoinOn. -pub type JoinOnRef<'a> = &'a [(PhysicalExprRef, PhysicalExprRef)]; - -pub struct JoinData { - pub projected_left_child: ProjectionExec, - pub projected_right_child: ProjectionExec, - pub join_filter: Option, - pub join_on: JoinOn, -} - -/// Downcasts all the expressions in `exprs` to `Column`s. If any of the given -/// expressions is not a `Column`, returns `None`. -pub fn physical_to_column_exprs( - exprs: &[(Arc, String)], -) -> Option> { - exprs - .iter() - .map(|(expr, alias)| { - expr.as_any() - .downcast_ref::() - .map(|col| (col.clone(), alias.clone())) - }) - .collect() -} - -/// Some projection can't be pushed down left input or right input of hash join because filter or on need may need some columns that won't be used in later. -/// By embed those projection to hash join, we can reduce the cost of build_batch_from_indices in hash join (build_batch_from_indices need to can compute::take() for each column) and avoid unnecessary output creation. -pub fn try_embed_projection( - projection: &ProjectionExec, - execution_plan: &Exec, -) -> Result>> { - // Collect all column indices from the given projection expressions. - let projection_index = collect_column_indices(projection.expr()); - - if projection_index.is_empty() { - return Ok(None); - }; - - // If the projection indices is the same as the input columns, we don't need to embed the projection to hash join. - // Check the projection_index is 0..n-1 and the length of projection_index is the same as the length of execution_plan schema fields. - if projection_index.len() == projection_index.last().unwrap() + 1 - && projection_index.len() == execution_plan.schema().fields().len() - { - return Ok(None); - } - - let new_execution_plan = - Arc::new(execution_plan.with_projection(Some(projection_index.to_vec()))?); - - // Build projection expressions for update_expr. Zip the projection_index with the new_execution_plan output schema fields. - let embed_project_exprs = projection_index - .iter() - .zip(new_execution_plan.schema().fields()) - .map(|(index, field)| { - ( - Arc::new(Column::new(field.name(), *index)) as Arc, - field.name().to_owned(), - ) - }) - .collect::>(); - - let mut new_projection_exprs = Vec::with_capacity(projection.expr().len()); - - for (expr, alias) in projection.expr() { - // update column index for projection expression since the input schema has been changed. - let Some(expr) = update_expr(expr, embed_project_exprs.as_slice(), false)? else { - return Ok(None); - }; - new_projection_exprs.push((expr, alias.clone())); - } - // Old projection may contain some alias or expression such as `a + 1` and `CAST('true' AS BOOLEAN)`, but our projection_exprs in hash join just contain column, so we need to create the new projection to keep the original projection. - let new_projection = Arc::new(ProjectionExec::try_new( - new_projection_exprs, - Arc::clone(&new_execution_plan) as _, - )?); - if is_projection_removable(&new_projection) { - Ok(Some(new_execution_plan)) - } else { - Ok(Some(new_projection)) - } -} - -pub fn try_pushdown_through_join( - projection: &ProjectionExec, - join_left: &Arc, - join_right: &Arc, - join_on: JoinOnRef, - schema: SchemaRef, - filter: Option<&JoinFilter>, -) -> Result> { - // Convert projected expressions to columns. We can not proceed if this is not possible. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = - join_table_borders(join_left.schema().fields().len(), &projection_as_columns); - - if !join_allows_pushdown( - &projection_as_columns, - &schema, - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let new_filter = if let Some(filter) = filter { - match update_join_filter( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - filter, - join_left.schema().fields().len(), - ) { - Some(updated_filter) => Some(updated_filter), - None => return Ok(None), - } - } else { - None - }; - - let Some(new_on) = update_join_on( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - join_on, - join_left.schema().fields().len(), - ) else { - return Ok(None); - }; - - let (new_left, new_right) = new_join_children( - &projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - join_left, - join_right, - )?; - - Ok(Some(JoinData { - projected_left_child: new_left, - projected_right_child: new_right, - join_filter: new_filter, - join_on: new_on, - })) -} - -/// If pushing down the projection over this join's children seems possible, -/// this function constructs the new [`ProjectionExec`]s that will come on top -/// of the original children of the join. -pub fn new_join_children( - projection_as_columns: &[(Column, String)], - far_right_left_col_ind: i32, - far_left_right_col_ind: i32, - left_child: &Arc, - right_child: &Arc, -) -> Result<(ProjectionExec, ProjectionExec)> { - let new_left = ProjectionExec::try_new( - projection_as_columns[0..=far_right_left_col_ind as _] - .iter() - .map(|(col, alias)| { - ( - Arc::new(Column::new(col.name(), col.index())) as _, - alias.clone(), - ) - }) - .collect_vec(), - Arc::clone(left_child), - )?; - let left_size = left_child.schema().fields().len() as i32; - let new_right = ProjectionExec::try_new( - projection_as_columns[far_left_right_col_ind as _..] - .iter() - .map(|(col, alias)| { - ( - Arc::new(Column::new( - col.name(), - // Align projected expressions coming from the right - // table with the new right child projection: - (col.index() as i32 - left_size) as _, - )) as _, - alias.clone(), - ) - }) - .collect_vec(), - Arc::clone(right_child), - )?; - - Ok((new_left, new_right)) -} - -/// Checks three conditions for pushing a projection down through a join: -/// - Projection must narrow the join output schema. -/// - Columns coming from left/right tables must be collected at the left/right -/// sides of the output table. -/// - Left or right table is not lost after the projection. -pub fn join_allows_pushdown( - projection_as_columns: &[(Column, String)], - join_schema: &SchemaRef, - far_right_left_col_ind: i32, - far_left_right_col_ind: i32, -) -> bool { - // Projection must narrow the join output: - projection_as_columns.len() < join_schema.fields().len() - // Are the columns from different tables mixed? - && (far_right_left_col_ind + 1 == far_left_right_col_ind) - // Left or right table is not lost after the projection. - && far_right_left_col_ind >= 0 - && far_left_right_col_ind < projection_as_columns.len() as i32 -} - -/// Returns the last index before encountering a column coming from the right table when traveling -/// through the projection from left to right, and the last index before encountering a column -/// coming from the left table when traveling through the projection from right to left. -/// If there is no column in the projection coming from the left side, it returns (-1, ...), -/// if there is no column in the projection coming from the right side, it returns (..., projection length). -pub fn join_table_borders( - left_table_column_count: usize, - projection_as_columns: &[(Column, String)], -) -> (i32, i32) { - let far_right_left_col_ind = projection_as_columns - .iter() - .enumerate() - .take_while(|(_, (projection_column, _))| { - projection_column.index() < left_table_column_count - }) - .last() - .map(|(index, _)| index as i32) - .unwrap_or(-1); - - let far_left_right_col_ind = projection_as_columns - .iter() - .enumerate() - .rev() - .take_while(|(_, (projection_column, _))| { - projection_column.index() >= left_table_column_count - }) - .last() - .map(|(index, _)| index as i32) - .unwrap_or(projection_as_columns.len() as i32); - - (far_right_left_col_ind, far_left_right_col_ind) -} - -/// Tries to update the equi-join `Column`'s of a join as if the input of -/// the join was replaced by a projection. -pub fn update_join_on( - proj_left_exprs: &[(Column, String)], - proj_right_exprs: &[(Column, String)], - hash_join_on: &[(PhysicalExprRef, PhysicalExprRef)], - left_field_size: usize, -) -> Option> { - // TODO: Clippy wants the "map" call removed, but doing so generates - // a compilation error. Remove the clippy directive once this - // issue is fixed. - #[allow(clippy::map_identity)] - let (left_idx, right_idx): (Vec<_>, Vec<_>) = hash_join_on - .iter() - .map(|(left, right)| (left, right)) - .unzip(); - - let new_left_columns = new_columns_for_join_on(&left_idx, proj_left_exprs, 0); - let new_right_columns = - new_columns_for_join_on(&right_idx, proj_right_exprs, left_field_size); - - match (new_left_columns, new_right_columns) { - (Some(left), Some(right)) => Some(left.into_iter().zip(right).collect()), - _ => None, - } -} - -/// Tries to update the column indices of a [`JoinFilter`] as if the input of -/// the join was replaced by a projection. -pub fn update_join_filter( - projection_left_exprs: &[(Column, String)], - projection_right_exprs: &[(Column, String)], - join_filter: &JoinFilter, - left_field_size: usize, -) -> Option { - let mut new_left_indices = new_indices_for_join_filter( - join_filter, - JoinSide::Left, - projection_left_exprs, - 0, - ) - .into_iter(); - let mut new_right_indices = new_indices_for_join_filter( - join_filter, - JoinSide::Right, - projection_right_exprs, - left_field_size, - ) - .into_iter(); - - // Check if all columns match: - (new_right_indices.len() + new_left_indices.len() - == join_filter.column_indices().len()) - .then(|| { - JoinFilter::new( - Arc::clone(join_filter.expression()), - join_filter - .column_indices() - .iter() - .map(|col_idx| ColumnIndex { - index: if col_idx.side == JoinSide::Left { - new_left_indices.next().unwrap() - } else { - new_right_indices.next().unwrap() - }, - side: col_idx.side, - }) - .collect(), - Arc::clone(join_filter.schema()), - ) - }) -} - -pub trait EmbeddedProjection: ExecutionPlan + Sized { - fn with_projection(&self, projection: Option>) -> Result; -} - -/// Collect all column indices from the given projection expressions. -fn collect_column_indices(exprs: &[(Arc, String)]) -> Vec { - // Collect indices and remove duplicates. - let mut indices = exprs - .iter() - .flat_map(|(expr, _)| collect_columns(expr)) - .map(|x| x.index()) - .collect::>() - .into_iter() - .collect::>(); - indices.sort(); - indices -} - -/// This function determines and returns a vector of indices representing the -/// positions of columns in `projection_exprs` that are involved in `join_filter`, -/// and correspond to a particular side (`join_side`) of the join operation. -/// -/// Notes: Column indices in the projection expressions are based on the join schema, -/// whereas the join filter is based on the join child schema. `column_index_offset` -/// represents the offset between them. -fn new_indices_for_join_filter( - join_filter: &JoinFilter, - join_side: JoinSide, - projection_exprs: &[(Column, String)], - column_index_offset: usize, -) -> Vec { - join_filter - .column_indices() - .iter() - .filter(|col_idx| col_idx.side == join_side) - .filter_map(|col_idx| { - projection_exprs - .iter() - .position(|(col, _)| col_idx.index + column_index_offset == col.index()) - }) - .collect() -} - -/// Compare the inputs and outputs of the projection. All expressions must be -/// columns without alias, and projection does not change the order of fields. -/// For example, if the input schema is `a, b`, `SELECT a, b` is removable, -/// but `SELECT b, a` and `SELECT a+1, b` and `SELECT a AS c, b` are not. -fn is_projection_removable(projection: &ProjectionExec) -> bool { - let exprs = projection.expr(); - exprs.iter().enumerate().all(|(idx, (expr, alias))| { - let Some(col) = expr.as_any().downcast_ref::() else { - return false; - }; - col.name() == alias && col.index() == idx - }) && exprs.len() == projection.input().schema().fields().len() -} - -/// This function generates a new set of columns to be used in a hash join -/// operation based on a set of equi-join conditions (`hash_join_on`) and a -/// list of projection expressions (`projection_exprs`). -/// -/// Notes: Column indices in the projection expressions are based on the join schema, -/// whereas the join on expressions are based on the join child schema. `column_index_offset` -/// represents the offset between them. -fn new_columns_for_join_on( - hash_join_on: &[&PhysicalExprRef], - projection_exprs: &[(Column, String)], - column_index_offset: usize, -) -> Option> { - let new_columns = hash_join_on - .iter() - .filter_map(|on| { - // Rewrite all columns in `on` - Arc::clone(*on) - .transform(|expr| { - if let Some(column) = expr.as_any().downcast_ref::() { - // Find the column in the projection expressions - let new_column = projection_exprs - .iter() - .enumerate() - .find(|(_, (proj_column, _))| { - column.name() == proj_column.name() - && column.index() + column_index_offset - == proj_column.index() - }) - .map(|(index, (_, alias))| Column::new(alias, index)); - if let Some(new_column) = new_column { - Ok(Transformed::yes(Arc::new(new_column))) - } else { - // If the column is not found in the projection expressions, - // it means that the column is not projected. In this case, - // we cannot push the projection down. - internal_err!( - "Column {:?} not found in projection expressions", - column - ) - } - } else { - Ok(Transformed::no(expr)) - } - }) - .data() - .ok() - }) - .collect::>(); - (new_columns.len() == hash_join_on.len()).then_some(new_columns) -} - -#[cfg(test)] -mod tests { - use super::*; - use std::sync::Arc; - - use datafusion_common::ScalarValue; - use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; - - #[test] - fn test_collect_column_indices() -> Result<()> { - let expr = Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 7)), - Operator::Minus, - Arc::new(BinaryExpr::new( - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - Operator::Plus, - Arc::new(Column::new("a", 1)), - )), - )); - let column_indices = collect_column_indices(&[(expr, "b-(1+a)".to_string())]); - assert_eq!(column_indices, vec![1, 7]); - Ok(()) - } - - #[test] - fn test_join_table_borders() -> Result<()> { - let projections = vec![ - (Column::new("b", 1), "b".to_owned()), - (Column::new("c", 2), "c".to_owned()), - (Column::new("e", 4), "e".to_owned()), - (Column::new("d", 3), "d".to_owned()), - (Column::new("c", 2), "c".to_owned()), - (Column::new("f", 5), "f".to_owned()), - (Column::new("h", 7), "h".to_owned()), - (Column::new("g", 6), "g".to_owned()), - ]; - let left_table_column_count = 5; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (4, 5) - ); - - let left_table_column_count = 8; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (7, 8) - ); - - let left_table_column_count = 1; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (-1, 0) - ); - - let projections = vec![ - (Column::new("a", 0), "a".to_owned()), - (Column::new("b", 1), "b".to_owned()), - (Column::new("d", 3), "d".to_owned()), - (Column::new("g", 6), "g".to_owned()), - (Column::new("e", 4), "e".to_owned()), - (Column::new("f", 5), "f".to_owned()), - (Column::new("e", 4), "e".to_owned()), - (Column::new("h", 7), "h".to_owned()), - ]; - let left_table_column_count = 5; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (2, 7) - ); - - let left_table_column_count = 7; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (6, 7) - ); - - Ok(()) - } -} diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 58da4639661d..262e0e648b30 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -32,8 +32,7 @@ use super::{ use crate::execution_plan::CardinalityEffect; use crate::hash_utils::create_hashes; use crate::metrics::BaselineMetrics; -use crate::projection::ProjectionExec; -use crate::projection_utils::{all_columns, make_with_child, update_expr}; +use crate::projection::{all_columns, make_with_child, update_expr, ProjectionExec}; use crate::repartition::distributor_channels::{ channels, partition_aware_channels, DistributionReceiver, DistributionSender, }; diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 433c1d6f2efd..caa9a754a0d2 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -31,8 +31,7 @@ use crate::limit::LimitStream; use crate::metrics::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, }; -use crate::projection::ProjectionExec; -use crate::projection_utils::{make_with_child, update_expr}; +use crate::projection::{make_with_child, update_expr, ProjectionExec}; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::spill::{ get_record_batch_memory_size, read_spill_as_stream, spill_record_batches, diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index fccc83cdddf4..152f3270f88d 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -23,8 +23,7 @@ use std::sync::Arc; use crate::common::spawn_buffered; use crate::limit::LimitStream; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; -use crate::projection::ProjectionExec; -use crate::projection_utils::{make_with_child, update_expr}; +use crate::projection::{make_with_child, update_expr, ProjectionExec}; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index afd14b241799..1af378baf03b 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -24,9 +24,8 @@ use std::sync::Arc; use super::{DisplayAs, DisplayFormatType, PlanProperties}; use crate::display::{display_orderings, ProjectSchemaDisplay}; use crate::execution_plan::{Boundedness, EmissionType}; -use crate::projection::ProjectionExec; -use crate::projection_utils::{ - all_alias_free_columns, new_projections_for_columns, update_expr, +use crate::projection::{ + all_alias_free_columns, new_projections_for_columns, update_expr, ProjectionExec, }; use crate::stream::RecordBatchStreamAdapter; use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 89d2603ec604..bacd02398ec0 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -36,8 +36,7 @@ use crate::execution_plan::{ boundedness_from_children, emission_type_from_children, InvariantLevel, }; use crate::metrics::BaselineMetrics; -use crate::projection::ProjectionExec; -use crate::projection_utils::make_with_child; +use crate::projection::{make_with_child, ProjectionExec}; use crate::stream::ObservedStream; use arrow::datatypes::{Field, Schema, SchemaRef}; From 2f4df62997268bae034ab44cc0b9b4bcd416d458 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 24 Jan 2025 16:14:47 +0300 Subject: [PATCH 04/12] fix the tests --- .../physical_optimizer/projection_pushdown.rs | 109 +--------------- datafusion/physical-plan/src/projection.rs | 120 +++++++++++++++--- 2 files changed, 107 insertions(+), 122 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index ac20875208d3..485b78720ce3 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -34,7 +34,9 @@ use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_plan::projection::{update_expr, ProjectionExec}; +use datafusion_physical_plan::projection::{ + remove_unnecessary_projections, update_expr, ProjectionExec, +}; /// This rule inspects [`ProjectionExec`]'s in the given physical plan and tries to /// remove or swap with its child. @@ -66,111 +68,6 @@ impl PhysicalOptimizerRule for ProjectionPushdown { } } -/// This function checks if `plan` is a [`ProjectionExec`], and inspects its -/// input(s) to test whether it can push `plan` under its input(s). This function -/// will operate on the entire tree and may ultimately remove `plan` entirely -/// by leveraging source providers with built-in projection capabilities. -pub fn remove_unnecessary_projections( - plan: Arc, -) -> Result>> { - let maybe_modified = if let Some(projection) = - plan.as_any().downcast_ref::() - { - // If the projection does not cause any change on the input, we can - // safely remove it: - if is_projection_removable(projection) { - return Ok(Transformed::yes(Arc::clone(projection.input()))); - } - // If it does, check if we can push it under its child(ren): - let input = projection.input().as_any(); - if let Some(child_projection) = input.downcast_ref::() { - let maybe_unified = try_unifying_projections(projection, child_projection)?; - return if let Some(new_plan) = maybe_unified { - // To unify 3 or more sequential projections: - remove_unnecessary_projections(new_plan) - .data() - .map(Transformed::yes) - } else { - Ok(Transformed::no(plan)) - }; - } else { - projection - .input() - .try_swapping_with_projection(projection)? - } - } else { - return Ok(Transformed::no(plan)); - }; - - Ok(maybe_modified.map_or(Transformed::no(plan), Transformed::yes)) -} - -/// Unifies `projection` with its input (which is also a [`ProjectionExec`]). -fn try_unifying_projections( - projection: &ProjectionExec, - child: &ProjectionExec, -) -> Result>> { - let mut projected_exprs = vec![]; - let mut column_ref_map: HashMap = HashMap::new(); - - // Collect the column references usage in the outer projection. - projection.expr().iter().for_each(|(expr, _)| { - expr.apply(|expr| { - Ok({ - if let Some(column) = expr.as_any().downcast_ref::() { - *column_ref_map.entry(column.clone()).or_default() += 1; - } - TreeNodeRecursion::Continue - }) - }) - .unwrap(); - }); - - // Merging these projections is not beneficial, e.g - // If an expression is not trivial and it is referred more than 1, unifies projections will be - // beneficial as caching mechanism for non-trivial computations. - // See discussion in: https://github.com/apache/datafusion/issues/8296 - if column_ref_map.iter().any(|(column, count)| { - *count > 1 && !is_expr_trivial(&Arc::clone(&child.expr()[column.index()].0)) - }) { - return Ok(None); - } - - for (expr, alias) in projection.expr() { - // If there is no match in the input projection, we cannot unify these - // projections. This case will arise if the projection expression contains - // a `PhysicalExpr` variant `update_expr` doesn't support. - let Some(expr) = update_expr(expr, child.expr(), true)? else { - return Ok(None); - }; - projected_exprs.push((expr, alias.clone())); - } - - ProjectionExec::try_new(projected_exprs, Arc::clone(child.input())) - .map(|e| Some(Arc::new(e) as _)) -} - -/// Checks if the given expression is trivial. -/// An expression is considered trivial if it is either a `Column` or a `Literal`. -fn is_expr_trivial(expr: &Arc) -> bool { - expr.as_any().downcast_ref::().is_some() - || expr.as_any().downcast_ref::().is_some() -} - -/// Compare the inputs and outputs of the projection. All expressions must be -/// columns without alias, and projection does not change the order of fields. -/// For example, if the input schema is `a, b`, `SELECT a, b` is removable, -/// but `SELECT b, a` and `SELECT a+1, b` and `SELECT a AS c, b` are not. -fn is_projection_removable(projection: &ProjectionExec) -> bool { - let exprs = projection.expr(); - exprs.iter().enumerate().all(|(idx, (expr, alias))| { - let Some(col) = expr.as_any().downcast_ref::() else { - return false; - }; - col.name() == alias && col.index() == idx - }) && exprs.len() == projection.input().schema().fields().len() -} - #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 2f4cd904dd6f..27f49b868eb5 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -29,7 +29,7 @@ use std::task::{Context, Poll}; use super::expressions::{CastExpr, Column, Literal}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{ - DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, + displayable, DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; @@ -38,7 +38,9 @@ use crate::joins::utils::{ColumnIndex, JoinFilter}; use crate::{ColumnStatistics, DisplayFormatType, ExecutionPlan, PhysicalExpr}; use datafusion_common::stats::Precision; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::tree_node::{ + Transformed, TransformedResult, TreeNode, TreeNodeRecursion, +}; use datafusion_common::{internal_err, JoinSide, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; @@ -246,6 +248,19 @@ impl ExecutionPlan for ProjectionExec { fn cardinality_effect(&self) -> CardinalityEffect { CardinalityEffect::Equal } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + let maybe_unified = try_unifying_projections(projection, self)?; + if let Some(new_plan) = maybe_unified { + // To unify 3 or more sequential projections: + remove_unnecessary_projections(new_plan).data().map(Some) + } else { + Ok(Some(Arc::new(projection.clone()))) + } + } } /// If 'e' is a direct column reference, returns the field level @@ -845,20 +860,6 @@ fn new_indices_for_join_filter( .collect() } -/// Compare the inputs and outputs of the projection. All expressions must be -/// columns without alias, and projection does not change the order of fields. -/// For example, if the input schema is `a, b`, `SELECT a, b` is removable, -/// but `SELECT b, a` and `SELECT a+1, b` and `SELECT a AS c, b` are not. -fn is_projection_removable(projection: &ProjectionExec) -> bool { - let exprs = projection.expr(); - exprs.iter().enumerate().all(|(idx, (expr, alias))| { - let Some(col) = expr.as_any().downcast_ref::() else { - return false; - }; - col.name() == alias && col.index() == idx - }) && exprs.len() == projection.input().schema().fields().len() -} - /// This function generates a new set of columns to be used in a hash join /// operation based on a set of equi-join conditions (`hash_join_on`) and a /// list of projection expressions (`projection_exprs`). @@ -910,6 +911,93 @@ fn new_columns_for_join_on( (new_columns.len() == hash_join_on.len()).then_some(new_columns) } +/// Unifies `projection` with its input (which is also a [`ProjectionExec`]). +fn try_unifying_projections( + projection: &ProjectionExec, + child: &ProjectionExec, +) -> Result>> { + let mut projected_exprs = vec![]; + let mut column_ref_map: HashMap = HashMap::new(); + + // Collect the column references usage in the outer projection. + projection.expr().iter().for_each(|(expr, _)| { + expr.apply(|expr| { + Ok({ + if let Some(column) = expr.as_any().downcast_ref::() { + *column_ref_map.entry(column.clone()).or_default() += 1; + } + TreeNodeRecursion::Continue + }) + }) + .unwrap(); + }); + // Merging these projections is not beneficial, e.g + // If an expression is not trivial and it is referred more than 1, unifies projections will be + // beneficial as caching mechanism for non-trivial computations. + // See discussion in: https://github.com/apache/datafusion/issues/8296 + if column_ref_map.iter().any(|(column, count)| { + *count > 1 && !is_expr_trivial(&Arc::clone(&child.expr()[column.index()].0)) + }) { + return Ok(None); + } + for (expr, alias) in projection.expr() { + // If there is no match in the input projection, we cannot unify these + // projections. This case will arise if the projection expression contains + // a `PhysicalExpr` variant `update_expr` doesn't support. + let Some(expr) = update_expr(expr, child.expr(), true)? else { + return Ok(None); + }; + projected_exprs.push((expr, alias.clone())); + } + ProjectionExec::try_new(projected_exprs, Arc::clone(child.input())) + .map(|e| Some(Arc::new(e) as _)) +} + +/// Checks if the given expression is trivial. +/// An expression is considered trivial if it is either a `Column` or a `Literal`. +fn is_expr_trivial(expr: &Arc) -> bool { + expr.as_any().downcast_ref::().is_some() + || expr.as_any().downcast_ref::().is_some() +} + +/// This function checks if `plan` is a [`ProjectionExec`], and inspects its +/// input(s) to test whether it can push `plan` under its input(s). This function +/// will operate on the entire tree and may ultimately remove `plan` entirely +/// by leveraging source providers with built-in projection capabilities. +pub fn remove_unnecessary_projections( + plan: Arc, +) -> Result>> { + let maybe_modified = + if let Some(projection) = plan.as_any().downcast_ref::() { + // If the projection does not cause any change on the input, we can + // safely remove it: + if is_projection_removable(projection) { + return Ok(Transformed::yes(Arc::clone(projection.input()))); + } + // If it does, check if we can push it under its child(ren): + projection + .input() + .try_swapping_with_projection(projection)? + } else { + return Ok(Transformed::no(plan)); + }; + Ok(maybe_modified.map_or(Transformed::no(plan), Transformed::yes)) +} + +/// Compare the inputs and outputs of the projection. All expressions must be +/// columns without alias, and projection does not change the order of fields. +/// For example, if the input schema is `a, b`, `SELECT a, b` is removable, +/// but `SELECT b, a` and `SELECT a+1, b` and `SELECT a AS c, b` are not. +fn is_projection_removable(projection: &ProjectionExec) -> bool { + let exprs = projection.expr(); + exprs.iter().enumerate().all(|(idx, (expr, alias))| { + let Some(col) = expr.as_any().downcast_ref::() else { + return false; + }; + col.name() == alias && col.index() == idx + }) && exprs.len() == projection.input().schema().fields().len() +} + #[cfg(test)] mod tests { use super::*; From 1cc56d53455a86583741bb7be44ddfda254fe3c2 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 24 Jan 2025 17:03:05 +0300 Subject: [PATCH 05/12] minor --- .../physical_optimizer/projection_pushdown.rs | 2 +- .../physical-plan/src/coalesce_partitions.rs | 3 +- .../physical-plan/src/execution_plan.rs | 49 ++++++++++--------- 3 files changed, 29 insertions(+), 25 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 485b78720ce3..ebe7de6809ce 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -70,9 +70,9 @@ impl PhysicalOptimizerRule for ProjectionPushdown { #[cfg(test)] mod tests { - use super::*; use std::any::Any; + use super::*; use crate::datasource::physical_plan::CsvExec; use crate::physical_plan::memory::MemoryExec; use crate::physical_plan::repartition::RepartitionExec; diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 61d6b8e0e865..3900bd1ddca9 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -27,11 +27,10 @@ use super::{ DisplayAs, ExecutionPlanProperties, PlanProperties, SendableRecordBatchStream, Statistics, }; - +use crate::execution_plan::CardinalityEffect; use crate::projection::{make_with_child, ProjectionExec}; use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; -use crate::execution_plan::CardinalityEffect; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 4278a9574c19..aa7371b2de99 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -15,43 +15,45 @@ // specific language governing permissions and limitations // under the License. -use std::any::Any; -use std::fmt::Debug; -use std::sync::Arc; - -use arrow::datatypes::SchemaRef; -use arrow::record_batch::RecordBatch; -use arrow_array::Array; -use futures::stream::{StreamExt, TryStreamExt}; -use tokio::task::JoinSet; +pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; +pub use crate::metrics::Metric; +pub use crate::ordering::InputOrderMode; +pub use crate::stream::EmptyRecordBatchStream; -use datafusion_common::config::ConfigOptions; pub use datafusion_common::hash_utils; pub use datafusion_common::utils::project_schema; -use datafusion_common::{exec_err, Constraints, Result}; pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; -use datafusion_execution::TaskContext; pub use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream}; pub use datafusion_expr::{Accumulator, ColumnarValue}; pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{ expressions, udf, Distribution, Partitioning, PhysicalExpr, }; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; + +use std::any::Any; +use std::fmt::Debug; +use std::sync::Arc; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::display::DisplayableExecutionPlan; -pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; -pub use crate::metrics::Metric; use crate::metrics::MetricsSet; -pub use crate::ordering::InputOrderMode; use crate::projection::ProjectionExec; use crate::repartition::RepartitionExec; use crate::sorts::sort_preserving_merge::SortPreservingMergeExec; -pub use crate::stream::EmptyRecordBatchStream; use crate::stream::RecordBatchStreamAdapter; +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; +use arrow_array::Array; +use datafusion_common::config::ConfigOptions; +use datafusion_common::{exec_err, Constraints, Result}; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion_physical_expr_common::sort_expr::LexRequirement; + +use futures::stream::{StreamExt, TryStreamExt}; +use tokio::task::JoinSet; + /// Represent nodes in the DataFusion Physical Plan. /// /// Calling [`execute`] produces an `async` [`SendableRecordBatchStream`] of @@ -433,10 +435,13 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { CardinalityEffect::Unknown } - // If try_swapping_with_projection is not implemented we conservatively - // assume that pushing the projection down may hurt. - // When adding new operators, consider adding them here if you - // think pushing projections under them is beneficial. + /// Attempts to push down the given projection into the input of this `ExecutionPlan`. + /// + /// If the operator supports this optimization, the resulting plan will be: + /// `self_new <- projection <- source`, starting from `projection <- self <- source`. + /// Otherwise, it returns the current `ExecutionPlan` as-is. + /// + /// Returns `Ok(Some(...))` if pushdown is applied, `Ok(None)` if unsupported, or `Err` on failure. fn try_swapping_with_projection( &self, _projection: &ProjectionExec, From 31f593c49d65d8b7518ad8f21f3c3d88ad574b94 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 24 Jan 2025 17:19:00 +0300 Subject: [PATCH 06/12] test changes --- .../sqllogictest/test_files/aggregate.slt | 17 ++--- .../sqllogictest/test_files/explain.slt | 9 +-- datafusion/sqllogictest/test_files/joins.slt | 62 ++++++++++--------- .../test_files/parquet_filter_pushdown.slt | 9 +-- .../sqllogictest/test_files/predicates.slt | 15 ++--- .../sqllogictest/test_files/subquery.slt | 25 ++++---- datafusion/sqllogictest/test_files/union.slt | 21 ++++--- datafusion/sqllogictest/test_files/window.slt | 11 ++-- 8 files changed, 90 insertions(+), 79 deletions(-) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index bd3b40089519..dfcdcac6dc25 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -6037,14 +6037,15 @@ logical_plan 03)----Aggregate: groupBy=[[having_test.v1, having_test.v2]], aggr=[[max(having_test.v1)]] 04)------TableScan: having_test projection=[v1, v2] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: max(having_test.v1)@2 = 3, projection=[v1@0, v2@1] -03)----AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -08)--------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[v1@0 as v1, v2@1 as v2] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: max(having_test.v1)@2 = 3 +04)------AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query error diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index f3fee4f1fca6..64329cc71a13 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -43,10 +43,11 @@ logical_plan 02)--Filter: aggregate_test_100.c2 > Int8(10) 03)----TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--FilterExec: c2@1 > 10, projection=[c1@0] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true +01)ProjectionExec: expr=[c1@0 as c1] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----FilterExec: c2@1 > 10 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true # explain_csv_exec_scan_config diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 68426f180d99..ba029d3a0793 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2082,14 +2082,15 @@ physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id] 02)--NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 03)----CoalescePartitionsExec -04)------CoalesceBatchesExec: target_batch_size=2 -05)--------FilterExec: t2_int@1 > 1, projection=[t2_id@0] -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] -08)----CoalesceBatchesExec: target_batch_size=2 -09)------FilterExec: t1_id@0 > 10 -10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)----------MemoryExec: partitions=1, partition_sizes=[1] +04)------ProjectionExec: expr=[t2_id@0 as t2_id] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------FilterExec: t2_int@1 > 1 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)----CoalesceBatchesExec: target_batch_size=2 +10)------FilterExec: t1_id@0 > 10 +11)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------MemoryExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -4151,9 +4152,10 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=3 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@3, currency_from@1)], filter=ts@0 >= ts@1, projection=[ts@0, sn@1, amount@2, currency@3, rate@6] 06)----------MemoryExec: partitions=1, partition_sizes=[0] -07)----------CoalesceBatchesExec: target_batch_size=3 -08)------------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] -09)--------------MemoryExec: partitions=1, partition_sizes=[0] +07)----------ProjectionExec: expr=[ts@0 as ts, currency_from@1 as currency_from, rate@3 as rate] +08)------------CoalesceBatchesExec: target_batch_size=3 +09)--------------FilterExec: currency_to@2 = USD +10)----------------MemoryExec: partitions=1, partition_sizes=[0] statement ok DROP TABLE sales_global; @@ -4346,15 +4348,16 @@ physical_plan 03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(a@0, a@0)] 04)------CoalesceBatchesExec: target_batch_size=3 05)--------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -06)----------CoalesceBatchesExec: target_batch_size=3 -07)------------FilterExec: b@1 > 3, projection=[a@0] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)----------------MemoryExec: partitions=1, partition_sizes=[1] -10)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] -11)--------CoalesceBatchesExec: target_batch_size=3 -12)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -13)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -14)--------------MemoryExec: partitions=1, partition_sizes=[1] +06)----------ProjectionExec: expr=[a@0 as a] +07)------------CoalesceBatchesExec: target_batch_size=3 +08)--------------FilterExec: b@1 > 3 +09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)------------------MemoryExec: partitions=1, partition_sizes=[1] +11)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] +12)--------CoalesceBatchesExec: target_batch_size=3 +13)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +14)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +15)--------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select * from test where a in (select a from test where b > 3) order by c desc nulls last; @@ -4373,15 +4376,16 @@ physical_plan 03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(a@0, a@0)] 04)------CoalesceBatchesExec: target_batch_size=3 05)--------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -06)----------CoalesceBatchesExec: target_batch_size=3 -07)------------FilterExec: b@1 > 3, projection=[a@0] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)----------------MemoryExec: partitions=1, partition_sizes=[1] -10)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] -11)--------CoalesceBatchesExec: target_batch_size=3 -12)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -13)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -14)--------------MemoryExec: partitions=1, partition_sizes=[1] +06)----------ProjectionExec: expr=[a@0 as a] +07)------------CoalesceBatchesExec: target_batch_size=3 +08)--------------FilterExec: b@1 > 3 +09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)------------------MemoryExec: partitions=1, partition_sizes=[1] +11)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] +12)--------CoalesceBatchesExec: target_batch_size=3 +13)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +14)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +15)--------------MemoryExec: partitions=1, partition_sizes=[1] query III select * from test where a in (select a from test where b > 3) order by c desc nulls first; diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index 806886b07170..392022dc386a 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -110,10 +110,11 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------FilterExec: b@1 > 2, projection=[a@0] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -06)----------ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], predicate=b@1 > 2, pruning_predicate=b_null_count@1 != b_row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----ProjectionExec: expr=[a@0 as a] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------FilterExec: b@1 > 2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +07)------------ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], predicate=b@1 > 2, pruning_predicate=b_null_count@1 != b_row_count@2 AND b_max@0 > 2, required_guarantees=[] # also test querying on columns that are not in all the files query T diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 878d7c8a4dfb..4695e37aa560 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -770,13 +770,14 @@ physical_plan 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true 10)----------CoalesceBatchesExec: target_batch_size=8192 11)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -12)--------------CoalesceBatchesExec: target_batch_size=8192 -13)----------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] -14)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true -16)------CoalesceBatchesExec: target_batch_size=8192 -17)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 -18)----------MemoryExec: partitions=1, partition_sizes=[1] +12)--------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +13)----------------CoalesceBatchesExec: target_batch_size=8192 +14)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23 +15)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true +17)------CoalesceBatchesExec: target_batch_size=8192 +18)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 +19)----------MemoryExec: partitions=1, partition_sizes=[1] # Inlist simplification diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 6933514951b3..5f7aedafaefe 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -1146,18 +1146,19 @@ logical_plan 05)------SubqueryAlias: __correlated_sq_1 06)--------TableScan: t2 projection=[t2_id] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--FilterExec: t1_id@0 > 40 OR NOT mark@3, projection=[t1_id@0, t1_name@1, t1_int@2] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=Partitioned, join_type=LeftMark, on=[(t1_id@0, t2_id@0)] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] -09)--------CoalesceBatchesExec: target_batch_size=2 -10)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -11)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)--------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----FilterExec: t1_id@0 > 40 OR NOT mark@3 +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------HashJoinExec: mode=Partitioned, join_type=LeftMark, on=[(t1_id@0, t2_id@0)] +06)----------CoalesceBatchesExec: target_batch_size=2 +07)------------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------MemoryExec: partitions=1, partition_sizes=[1] +10)----------CoalesceBatchesExec: target_batch_size=2 +11)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +12)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)----------------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 7b8992b966ad..214c7f4de1d8 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -522,16 +522,17 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=2 11)--------------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 12)----------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] -13)------------------------CoalesceBatchesExec: target_batch_size=2 -14)--------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] -15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true -17)------ProjectionExec: expr=[1 as cnt] -18)--------PlaceholderRowExec -19)------ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] -20)--------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -21)----------ProjectionExec: expr=[1 as c1] -22)------------PlaceholderRowExec +13)------------------------ProjectionExec: expr=[c1@0 as c1] +14)--------------------------CoalesceBatchesExec: target_batch_size=2 +15)----------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 +16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true +18)------ProjectionExec: expr=[1 as cnt] +19)--------PlaceholderRowExec +20)------ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] +21)--------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +22)----------ProjectionExec: expr=[1 as c1] +23)------------PlaceholderRowExec ######## diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 889810611cf1..64f2b0da5769 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1767,7 +1767,7 @@ logical_plan 01)Projection: count(*) AS global_count 02)--Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] 03)----SubqueryAlias: a -04)------Projection: +04)------Projection: 05)--------Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[]] 06)----------Projection: aggregate_test_100.c1 07)------------Filter: aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434") @@ -1782,10 +1782,11 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=4096 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] -10)------------------CoalesceBatchesExec: target_batch_size=4096 -11)--------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true +10)------------------ProjectionExec: expr=[c1@0 as c1] +11)--------------------CoalesceBatchesExec: target_batch_size=4096 +12)----------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 +13)------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true query I From 6673aa11573fd2746a185c2338eb8a5d2927461d Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Sat, 25 Jan 2025 18:29:48 +0300 Subject: [PATCH 07/12] Update projection.rs --- datafusion/physical-plan/src/projection.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 27f49b868eb5..f52e6dd4549e 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -29,7 +29,7 @@ use std::task::{Context, Poll}; use super::expressions::{CastExpr, Column, Literal}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{ - displayable, DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, + DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; From 304ee08522fc98a48d389490478c2aa7d27343bf Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Sat, 25 Jan 2025 18:32:29 +0300 Subject: [PATCH 08/12] investigate further --- .../test_files/tpch/plans/q1.slt.part | 4 +- .../test_files/tpch/plans/q10.slt.part | 26 ++-- .../test_files/tpch/plans/q11.slt.part | 57 ++++----- .../test_files/tpch/plans/q12.slt.part | 13 +- .../test_files/tpch/plans/q13.slt.part | 7 +- .../test_files/tpch/plans/q14.slt.part | 15 +-- .../test_files/tpch/plans/q15.slt.part | 34 +++--- .../test_files/tpch/plans/q16.slt.part | 10 +- .../test_files/tpch/plans/q17.slt.part | 22 ++-- .../test_files/tpch/plans/q18.slt.part | 15 +-- .../test_files/tpch/plans/q19.slt.part | 19 +-- .../test_files/tpch/plans/q2.slt.part | 113 +++++++++--------- .../test_files/tpch/plans/q20.slt.part | 62 +++++----- .../test_files/tpch/plans/q21.slt.part | 48 ++++---- .../test_files/tpch/plans/q22.slt.part | 9 +- .../test_files/tpch/plans/q3.slt.part | 30 ++--- .../test_files/tpch/plans/q4.slt.part | 18 +-- .../test_files/tpch/plans/q5.slt.part | 42 +++---- .../test_files/tpch/plans/q6.slt.part | 7 +- .../test_files/tpch/plans/q8.slt.part | 70 +++++------ .../test_files/tpch/plans/q9.slt.part | 43 +++---- 21 files changed, 347 insertions(+), 317 deletions(-) diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part index c60848cb56c1..b6417bc89306 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part @@ -56,5 +56,5 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(*)] 08)--------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5] -11)--------------------CsvExec: 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_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false \ No newline at end of file +10)------------------FilterExec: l_shipdate@6 <= 1998-09-02 +11)--------------------CsvExec: 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_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index e4376437ad4f..d9924f5f279f 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -92,15 +92,17 @@ physical_plan 21)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], has_header=false 22)----------------------------------CoalesceBatchesExec: target_batch_size=8192 23)------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -24)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)----------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] -26)------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false -27)--------------------------CoalesceBatchesExec: target_batch_size=8192 -28)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -29)------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -31)----------------------------------CsvExec: 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_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false -32)------------------CoalesceBatchesExec: target_batch_size=8192 -33)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -34)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -35)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +24)--------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] +25)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +26)------------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01 +27)--------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +28)--------------------------CoalesceBatchesExec: target_batch_size=8192 +29)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +30)------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +31)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)----------------------------------FilterExec: l_returnflag@3 = R +33)------------------------------------CsvExec: 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_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false +34)------------------CoalesceBatchesExec: target_batch_size=8192 +35)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +36)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +37)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index 6645ede2a73b..515d532562a8 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -96,31 +96,32 @@ physical_plan 21)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false 22)--------------------CoalesceBatchesExec: target_batch_size=8192 23)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -24)------------------------CoalesceBatchesExec: target_batch_size=8192 -25)--------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -26)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -27)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -28)------ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] -29)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -30)----------CoalescePartitionsExec -31)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -32)--------------CoalesceBatchesExec: target_batch_size=8192 -33)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -34)------------------CoalesceBatchesExec: target_batch_size=8192 -35)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -36)----------------------CoalesceBatchesExec: target_batch_size=8192 -37)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -38)--------------------------CoalesceBatchesExec: target_batch_size=8192 -39)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -40)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], has_header=false -41)--------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -43)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -44)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -45)------------------CoalesceBatchesExec: target_batch_size=8192 -46)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -47)----------------------CoalesceBatchesExec: target_batch_size=8192 -48)------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] -49)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -50)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false - +24)------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +25)--------------------------CoalesceBatchesExec: target_batch_size=8192 +26)----------------------------FilterExec: n_name@1 = GERMANY +27)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +28)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +29)------ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] +30)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +31)----------CoalescePartitionsExec +32)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +33)--------------CoalesceBatchesExec: target_batch_size=8192 +34)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +35)------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +37)----------------------CoalesceBatchesExec: target_batch_size=8192 +38)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +39)--------------------------CoalesceBatchesExec: target_batch_size=8192 +40)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +41)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], has_header=false +42)--------------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +44)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +45)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +46)------------------CoalesceBatchesExec: target_batch_size=8192 +47)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +48)----------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +49)------------------------CoalesceBatchesExec: target_batch_size=8192 +50)--------------------------FilterExec: n_name@1 = GERMANY +51)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +52)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index 34e5dd69f410..561cda701d5a 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -70,9 +70,10 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] -14)--------------------------CsvExec: 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_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], has_header=false -15)------------------CoalesceBatchesExec: target_batch_size=8192 -16)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -17)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], has_header=false +12)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01 +15)----------------------------CsvExec: 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_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], has_header=false +16)------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +18)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 2667bb23a459..8f36bca81cf0 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -70,6 +70,7 @@ physical_plan 15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false 16)----------------------CoalesceBatchesExec: target_batch_size=8192 17)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -18)--------------------------CoalesceBatchesExec: target_batch_size=8192 -19)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] -20)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false +18)--------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] +19)----------------------------CoalesceBatchesExec: target_batch_size=8192 +20)------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% +21)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part index 96cfbb02d07c..f791a89373a5 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -50,10 +50,11 @@ physical_plan 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4] 08)--------------CoalesceBatchesExec: target_batch_size=8192 09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] -12)----------------------CsvExec: 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_extendedprice, l_discount, l_shipdate], has_header=false -13)--------------CoalesceBatchesExec: target_batch_size=8192 -14)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -15)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +10)------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01 +13)------------------------CsvExec: 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_extendedprice, l_discount, l_shipdate], has_header=false +14)--------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +16)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index 4cf4c722eaf8..81c255396e6c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -88,19 +88,21 @@ physical_plan 15)--------------------CoalesceBatchesExec: target_batch_size=8192 16)----------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 17)------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -18)--------------------------CoalesceBatchesExec: target_batch_size=8192 -19)----------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -20)------------------------------CsvExec: 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_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false -21)--------CoalesceBatchesExec: target_batch_size=8192 -22)----------RepartitionExec: partitioning=Hash([max(revenue0.total_revenue)@0], 4), input_partitions=1 -23)------------AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] -24)--------------CoalescePartitionsExec -25)----------------AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] -26)------------------ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -27)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -28)----------------------CoalesceBatchesExec: target_batch_size=8192 -29)------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -30)--------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -31)----------------------------CoalesceBatchesExec: target_batch_size=8192 -32)------------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] -33)--------------------------------CsvExec: 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_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +18)--------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +19)----------------------------CoalesceBatchesExec: target_batch_size=8192 +20)------------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01 +21)--------------------------------CsvExec: 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_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +22)--------CoalesceBatchesExec: target_batch_size=8192 +23)----------RepartitionExec: partitioning=Hash([max(revenue0.total_revenue)@0], 4), input_partitions=1 +24)------------AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] +25)--------------CoalescePartitionsExec +26)----------------AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] +27)------------------ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +28)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +29)----------------------CoalesceBatchesExec: target_batch_size=8192 +30)------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +31)--------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +32)----------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +33)------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)--------------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01 +35)----------------------------------CsvExec: 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_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index e9486c39f08c..35e0ec305093 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -93,8 +93,8 @@ physical_plan 26)--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], has_header=false 27)--------------------------CoalesceBatchesExec: target_batch_size=8192 28)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -29)------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] -31)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -32)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], has_header=false - +29)------------------------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey] +30)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)----------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints% +32)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +33)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index f28a5ef54885..44bda7cfc4a1 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -64,14 +64,14 @@ physical_plan 11)--------------------CsvExec: 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], has_header=false 12)----------------CoalesceBatchesExec: target_batch_size=8192 13)------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -14)--------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] -16)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false -18)------------ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] -19)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -20)----------------CoalesceBatchesExec: target_batch_size=8192 -21)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -22)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -23)----------------------CsvExec: 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], has_header=false - +14)--------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +15)----------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX +17)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false +19)------------ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] +20)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +21)----------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +23)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +24)----------------------CsvExec: 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], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index 283de4dddfe3..ef1392fec430 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -91,10 +91,11 @@ physical_plan 22)--------------------CoalesceBatchesExec: target_batch_size=8192 23)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 24)------------------------CsvExec: 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_orderkey, l_quantity], has_header=false -25)----------------CoalesceBatchesExec: target_batch_size=8192 -26)------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] -27)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -28)----------------------CoalesceBatchesExec: target_batch_size=8192 -29)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -30)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -31)----------------------------CsvExec: 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_orderkey, l_quantity], has_header=false +25)----------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] +26)------------------CoalesceBatchesExec: target_batch_size=8192 +27)--------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2 +28)----------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +29)------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +31)----------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +32)------------------------------CsvExec: 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_orderkey, l_quantity], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part index 9eb1f8638292..38dc9729f003 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -72,12 +72,13 @@ physical_plan 06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@2, l_discount@3] 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] -11)--------------------CsvExec: 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], has_header=false -12)------------CoalesceBatchesExec: target_batch_size=8192 -13)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -14)----------------CoalesceBatchesExec: target_batch_size=8192 -15)------------------FilterExec: (p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 -16)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false +09)----------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------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 +12)----------------------CsvExec: 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], has_header=false +13)------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +15)----------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------FilterExec: (p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 +17)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index 8fdbb6884d14..eae0f69f8d9e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -123,58 +123,61 @@ physical_plan 22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] 23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 24)----------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] -27)----------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -28)------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], has_header=false -29)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -31)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false -32)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -34)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -35)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false -36)--------------------------CoalesceBatchesExec: target_batch_size=8192 -37)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -38)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -39)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false -40)------------------CoalesceBatchesExec: target_batch_size=8192 -41)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -42)----------------------CoalesceBatchesExec: target_batch_size=8192 -43)------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -44)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -45)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false -46)----------CoalesceBatchesExec: target_batch_size=8192 -47)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 -48)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -49)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -50)------------------CoalesceBatchesExec: target_batch_size=8192 -51)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -52)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -53)------------------------CoalesceBatchesExec: target_batch_size=8192 -54)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -55)----------------------------CoalesceBatchesExec: target_batch_size=8192 -56)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -57)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -58)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -59)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -60)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -61)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -62)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -63)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -64)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -65)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false -66)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -67)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -68)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -69)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -70)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -71)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -72)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -73)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -74)----------------------------CoalesceBatchesExec: target_batch_size=8192 -75)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -76)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -77)----------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] -78)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -79)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +25)------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] +26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS +28)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +29)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], has_header=false +30)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +32)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +33)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +35)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +36)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false +37)--------------------------CoalesceBatchesExec: target_batch_size=8192 +38)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +39)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +40)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false +41)------------------CoalesceBatchesExec: target_batch_size=8192 +42)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +43)----------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +44)------------------------CoalesceBatchesExec: target_batch_size=8192 +45)--------------------------FilterExec: r_name@1 = EUROPE +46)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +47)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +48)----------CoalesceBatchesExec: target_batch_size=8192 +49)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 +50)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +51)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +52)------------------CoalesceBatchesExec: target_batch_size=8192 +53)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +54)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +55)------------------------CoalesceBatchesExec: target_batch_size=8192 +56)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +57)----------------------------CoalesceBatchesExec: target_batch_size=8192 +58)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +59)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +60)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +61)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +62)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +63)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +64)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +65)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +66)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +67)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +68)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +69)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +70)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +71)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +72)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +73)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +74)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +75)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +76)----------------------------CoalesceBatchesExec: target_batch_size=8192 +77)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +78)--------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +79)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +80)------------------------------------FilterExec: r_name@1 = EUROPE +81)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +82)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index e3918ba62b02..aef9b0636e39 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -95,33 +95,35 @@ physical_plan 12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], has_header=false 13)----------------CoalesceBatchesExec: target_batch_size=8192 14)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -15)--------------------CoalesceBatchesExec: target_batch_size=8192 -16)----------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] -17)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -19)--------CoalesceBatchesExec: target_batch_size=8192 -20)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -21)------------CoalesceBatchesExec: target_batch_size=8192 -22)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] -23)----------------CoalesceBatchesExec: target_batch_size=8192 -24)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 -25)--------------------CoalesceBatchesExec: target_batch_size=8192 -26)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -27)------------------------CoalesceBatchesExec: target_batch_size=8192 -28)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -29)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], has_header=false -30)------------------------CoalesceBatchesExec: target_batch_size=8192 -31)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -32)----------------------------CoalesceBatchesExec: target_batch_size=8192 -33)------------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] -34)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -35)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false -36)----------------ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] -37)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -38)--------------------CoalesceBatchesExec: target_batch_size=8192 -39)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -40)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -41)--------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] -43)------------------------------CsvExec: 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_suppkey, l_quantity, l_shipdate], has_header=false - +15)--------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +16)----------------------CoalesceBatchesExec: target_batch_size=8192 +17)------------------------FilterExec: n_name@1 = CANADA +18)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +19)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +20)--------CoalesceBatchesExec: target_batch_size=8192 +21)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +22)------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] +24)----------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +26)--------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +28)------------------------CoalesceBatchesExec: target_batch_size=8192 +29)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +30)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], has_header=false +31)------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +33)----------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +34)------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)--------------------------------FilterExec: p_name@1 LIKE forest% +36)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +37)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +38)----------------ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] +39)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +40)--------------------CoalesceBatchesExec: target_batch_size=8192 +41)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +42)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +43)--------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] +44)----------------------------CoalesceBatchesExec: target_batch_size=8192 +45)------------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 +46)--------------------------------CsvExec: 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_suppkey, l_quantity, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index 939e01b1784e..f88a57536ed6 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -119,25 +119,29 @@ physical_plan 27)----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], has_header=false 28)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 29)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)----------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -32)------------------------------------------------------CsvExec: 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_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false -33)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] -37)----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], has_header=false -38)------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -40)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -41)------------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] -42)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -43)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -44)----------------------CoalesceBatchesExec: target_batch_size=8192 -45)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -46)--------------------------CsvExec: 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_orderkey, l_suppkey], has_header=false -47)------------------CoalesceBatchesExec: target_batch_size=8192 -48)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -49)----------------------CoalesceBatchesExec: target_batch_size=8192 -50)------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] -51)--------------------------CsvExec: 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_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false +30)--------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] +31)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 +33)--------------------------------------------------------CsvExec: 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_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false +34)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +36)------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] +37)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)----------------------------------------------FilterExec: o_orderstatus@1 = F +39)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], has_header=false +40)------------------------------CoalesceBatchesExec: target_batch_size=8192 +41)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +42)----------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +43)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------------------------FilterExec: n_name@1 = SAUDI ARABIA +45)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +46)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +47)----------------------CoalesceBatchesExec: target_batch_size=8192 +48)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +49)--------------------------CsvExec: 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_orderkey, l_suppkey], has_header=false +50)------------------CoalesceBatchesExec: target_batch_size=8192 +51)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +52)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] +53)------------------------CoalesceBatchesExec: target_batch_size=8192 +54)--------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 +55)----------------------------CsvExec: 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_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index a55d1e8fdfb1..802239d4e269 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -99,7 +99,8 @@ physical_plan 23)--------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] 24)----------------------CoalescePartitionsExec 25)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] -26)--------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]), projection=[c_acctbal@1] -28)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -29)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false +26)--------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] +27)----------------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]) +29)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +30)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part index 0f35239bfd9d..c4fb84a4bf44 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -73,17 +73,19 @@ physical_plan 13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 15)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] -18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false -20)--------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -22)------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------FilterExec: o_orderdate@2 < 1995-03-15 -24)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], has_header=false -25)------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -27)----------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] -29)--------------------------CsvExec: 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_orderkey, l_extendedprice, l_discount, l_shipdate], has_header=false +16)------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------FilterExec: c_mktsegment@1 = BUILDING +19)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +20)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false +21)--------------------------CoalesceBatchesExec: target_batch_size=8192 +22)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +23)------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)--------------------------------FilterExec: o_orderdate@2 < 1995-03-15 +25)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], has_header=false +26)------------------CoalesceBatchesExec: target_batch_size=8192 +27)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +28)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +29)------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------FilterExec: l_shipdate@3 > 1995-03-15 +31)----------------------------CsvExec: 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_orderkey, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part index 693d37a5b53d..854718bb50ac 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -64,11 +64,13 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] -14)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], has_header=false -15)------------------CoalesceBatchesExec: target_batch_size=8192 -16)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -17)----------------------CoalesceBatchesExec: target_batch_size=8192 -18)------------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] -19)--------------------------CsvExec: 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_orderkey, l_commitdate, l_receiptdate], has_header=false +12)----------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01 +15)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], has_header=false +16)------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +18)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] +19)------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------FilterExec: l_receiptdate@2 > l_commitdate@1 +21)----------------------------CsvExec: 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_orderkey, l_commitdate, l_receiptdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index 0c03e0498d64..e23d39f2437e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -98,23 +98,25 @@ physical_plan 29)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false 30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 31)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -32)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] -34)----------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false -35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -37)----------------------------------------------CsvExec: 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_orderkey, l_suppkey, l_extendedprice, l_discount], has_header=false -38)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 -40)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -41)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -42)--------------------------CoalesceBatchesExec: target_batch_size=8192 -43)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -44)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -45)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false -46)------------------CoalesceBatchesExec: target_batch_size=8192 -47)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -48)----------------------CoalesceBatchesExec: target_batch_size=8192 -49)------------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] -50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -51)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +32)------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] +33)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01 +35)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +36)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +38)----------------------------------------------CsvExec: 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_orderkey, l_suppkey, l_extendedprice, l_discount], has_header=false +39)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +40)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 +41)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +42)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +43)--------------------------CoalesceBatchesExec: target_batch_size=8192 +44)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +45)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +46)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false +47)------------------CoalesceBatchesExec: target_batch_size=8192 +48)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +49)----------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +50)------------------------CoalesceBatchesExec: target_batch_size=8192 +51)--------------------------FilterExec: r_name@1 = ASIA +52)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +53)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part index 6c818a1e2f42..3797a2593f7c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part @@ -38,6 +38,7 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2] -07)------------CsvExec: 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_quantity, l_extendedprice, l_discount, l_shipdate], has_header=false +05)--------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2 +08)--------------CsvExec: 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_quantity, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index 2bcab40dc985..babdbc28b18b 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -126,37 +126,39 @@ physical_plan 34)------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] 35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 36)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -37)------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)--------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] -39)----------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -40)------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false -41)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -43)------------------------------------------------------------------------CsvExec: 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_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false -44)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -45)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -46)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -47)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -48)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -49)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -50)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -51)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 -52)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false -53)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -54)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -55)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -56)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -57)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -58)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -59)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -60)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -61)----------------------------CoalesceBatchesExec: target_batch_size=8192 -62)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -63)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -64)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -65)--------------------CoalesceBatchesExec: target_batch_size=8192 -66)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -67)------------------------CoalesceBatchesExec: target_batch_size=8192 -68)--------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] -69)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -70)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +37)------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +38)--------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)----------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL +40)------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +41)--------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +42)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +44)------------------------------------------------------------------------CsvExec: 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_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false +45)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +46)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +47)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +48)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +49)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +50)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +51)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +52)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 +53)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +54)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +55)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +56)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +57)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +58)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +59)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +60)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +61)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +62)----------------------------CoalesceBatchesExec: target_batch_size=8192 +63)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +64)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +65)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +66)--------------------CoalesceBatchesExec: target_batch_size=8192 +67)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +68)------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +69)--------------------------CoalesceBatchesExec: target_batch_size=8192 +70)----------------------------FilterExec: r_name@1 = AMERICA +71)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +72)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 4a288893da95..0361f1b21cb5 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -103,24 +103,25 @@ physical_plan 26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] 27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 28)------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------------------FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] -31)------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -32)--------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false -33)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -35)--------------------------------------------------------CsvExec: 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_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false -36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -37)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -38)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -39)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -40)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -41)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -42)----------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false -43)----------------------------CoalesceBatchesExec: target_batch_size=8192 -44)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -45)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], has_header=false -46)--------------------CoalesceBatchesExec: target_batch_size=8192 -47)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -48)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -49)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +29)--------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +30)----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)------------------------------------------------------------FilterExec: p_name@1 LIKE %green% +32)--------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +33)----------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +34)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +36)--------------------------------------------------------CsvExec: 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_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false +37)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +39)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +40)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +41)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +42)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +43)----------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +44)----------------------------CoalesceBatchesExec: target_batch_size=8192 +45)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +46)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], has_header=false +47)--------------------CoalesceBatchesExec: target_batch_size=8192 +48)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +49)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +50)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false From dae14674d040a5eb29a92474182e496952ca045a Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Sat, 25 Jan 2025 19:37:43 +0300 Subject: [PATCH 09/12] revert test changes with the fix --- datafusion/physical-plan/src/filter.rs | 32 ++--- .../sqllogictest/test_files/aggregate.slt | 17 ++- datafusion/sqllogictest/test_files/ddl.slt | 1 - .../sqllogictest/test_files/encoding.slt | 2 +- .../sqllogictest/test_files/explain.slt | 9 +- datafusion/sqllogictest/test_files/expr.slt | 1 + datafusion/sqllogictest/test_files/joins.slt | 62 +++++----- .../test_files/parquet_filter_pushdown.slt | 9 +- .../sqllogictest/test_files/predicates.slt | 15 ++- .../sqllogictest/test_files/subquery.slt | 27 ++--- .../test_files/tpch/plans/q1.slt.part | 2 +- .../test_files/tpch/plans/q10.slt.part | 26 ++-- .../test_files/tpch/plans/q11.slt.part | 56 +++++---- .../test_files/tpch/plans/q12.slt.part | 13 +- .../test_files/tpch/plans/q13.slt.part | 7 +- .../test_files/tpch/plans/q14.slt.part | 15 ++- .../test_files/tpch/plans/q15.slt.part | 34 +++--- .../test_files/tpch/plans/q16.slt.part | 9 +- .../test_files/tpch/plans/q17.slt.part | 21 ++-- .../test_files/tpch/plans/q18.slt.part | 15 ++- .../test_files/tpch/plans/q19.slt.part | 19 ++- .../test_files/tpch/plans/q2.slt.part | 113 +++++++++--------- .../test_files/tpch/plans/q20.slt.part | 61 +++++----- .../test_files/tpch/plans/q21.slt.part | 48 ++++---- .../test_files/tpch/plans/q22.slt.part | 9 +- .../test_files/tpch/plans/q3.slt.part | 30 +++-- .../test_files/tpch/plans/q4.slt.part | 18 ++- .../test_files/tpch/plans/q5.slt.part | 42 ++++--- .../test_files/tpch/plans/q6.slt.part | 7 +- .../test_files/tpch/plans/q8.slt.part | 70 ++++++----- .../test_files/tpch/plans/q9.slt.part | 43 ++++--- .../sqllogictest/test_files/tpch/tpch.slt | 1 - datafusion/sqllogictest/test_files/union.slt | 21 ++-- datafusion/sqllogictest/test_files/window.slt | 9 +- 34 files changed, 409 insertions(+), 455 deletions(-) diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index b92255dbc4ad..78ab1e3441d5 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -411,23 +411,23 @@ impl ExecutionPlan for FilterExec { projection: &ProjectionExec, ) -> Result>> { // If the projection does not narrow the schema, we should not try to push it down: - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); + if projection.expr().len() < projection.input().schema().fields().len() { + // Each column in the predicate expression must exist after the projection. + if let Some(new_predicate) = + update_expr(self.predicate(), projection.expr(), false)? + { + return FilterExec::try_new( + new_predicate, + make_with_child(projection, self.input())?, + ) + .and_then(|e| { + let selectivity = self.default_selectivity(); + e.with_default_selectivity(selectivity) + }) + .map(|e| Some(Arc::new(e) as _)); + } } - // Each column in the predicate expression must exist after the projection. - let Some(new_predicate) = - update_expr(self.predicate(), projection.expr(), false)? - else { - return Ok(None); - }; - - FilterExec::try_new(new_predicate, make_with_child(projection, self.input())?) - .and_then(|e| { - let selectivity = self.default_selectivity(); - e.with_default_selectivity(selectivity) - }) - .map(|e| Some(Arc::new(e) as _))? - .map_or_else(|| try_embed_projection(projection, self), |e| Ok(Some(e))) + try_embed_projection(projection, self) } } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index dfcdcac6dc25..bd3b40089519 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -6037,15 +6037,14 @@ logical_plan 03)----Aggregate: groupBy=[[having_test.v1, having_test.v2]], aggr=[[max(having_test.v1)]] 04)------TableScan: having_test projection=[v1, v2] physical_plan -01)ProjectionExec: expr=[v1@0 as v1, v2@1 as v2] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: max(having_test.v1)@2 = 3 -04)------AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] -09)----------------MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: max(having_test.v1)@2 = 3, projection=[v1@0, v2@1] +03)----AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=4 +06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query error diff --git a/datafusion/sqllogictest/test_files/ddl.slt b/datafusion/sqllogictest/test_files/ddl.slt index 02f66f342b8a..0798c8cc1453 100644 --- a/datafusion/sqllogictest/test_files/ddl.slt +++ b/datafusion/sqllogictest/test_files/ddl.slt @@ -840,4 +840,3 @@ DROP TABLE t1; statement ok DROP TABLE t2; - diff --git a/datafusion/sqllogictest/test_files/encoding.slt b/datafusion/sqllogictest/test_files/encoding.slt index be1c5aa40583..960e81f4d14c 100644 --- a/datafusion/sqllogictest/test_files/encoding.slt +++ b/datafusion/sqllogictest/test_files/encoding.slt @@ -125,4 +125,4 @@ NULL NULL NULL R Ug 52 query T select encode(digest('hello', 'sha256'), 'hex'); ---- -2cf24dba5fb0a30e26e83b2ac5b9e29e1b161e5c1fa7425e73043362938b9824 \ No newline at end of file +2cf24dba5fb0a30e26e83b2ac5b9e29e1b161e5c1fa7425e73043362938b9824 diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 64329cc71a13..f3fee4f1fca6 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -43,11 +43,10 @@ logical_plan 02)--Filter: aggregate_test_100.c2 > Int8(10) 03)----TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] physical_plan -01)ProjectionExec: expr=[c1@0 as c1] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----FilterExec: c2@1 > 10 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: c2@1 > 10, projection=[c1@0] +03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true # explain_csv_exec_scan_config diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index a3592ba159e2..13dc74b6cb7d 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -2047,3 +2047,4 @@ select 1 where null between null and 2; query T select 'A' where null between 2 and null; --- +---- diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index ba029d3a0793..68426f180d99 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2082,15 +2082,14 @@ physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id] 02)--NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 03)----CoalescePartitionsExec -04)------ProjectionExec: expr=[t2_id@0 as t2_id] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------FilterExec: t2_int@1 > 1 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] -09)----CoalesceBatchesExec: target_batch_size=2 -10)------FilterExec: t1_id@0 > 10 -11)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------MemoryExec: partitions=1, partition_sizes=[1] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------FilterExec: t2_int@1 > 1, projection=[t2_id@0] +06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)----CoalesceBatchesExec: target_batch_size=2 +09)------FilterExec: t1_id@0 > 10 +10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)----------MemoryExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -4152,10 +4151,9 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=3 05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@3, currency_from@1)], filter=ts@0 >= ts@1, projection=[ts@0, sn@1, amount@2, currency@3, rate@6] 06)----------MemoryExec: partitions=1, partition_sizes=[0] -07)----------ProjectionExec: expr=[ts@0 as ts, currency_from@1 as currency_from, rate@3 as rate] -08)------------CoalesceBatchesExec: target_batch_size=3 -09)--------------FilterExec: currency_to@2 = USD -10)----------------MemoryExec: partitions=1, partition_sizes=[0] +07)----------CoalesceBatchesExec: target_batch_size=3 +08)------------FilterExec: currency_to@2 = USD, projection=[ts@0, currency_from@1, rate@3] +09)--------------MemoryExec: partitions=1, partition_sizes=[0] statement ok DROP TABLE sales_global; @@ -4348,16 +4346,15 @@ physical_plan 03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(a@0, a@0)] 04)------CoalesceBatchesExec: target_batch_size=3 05)--------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -06)----------ProjectionExec: expr=[a@0 as a] -07)------------CoalesceBatchesExec: target_batch_size=3 -08)--------------FilterExec: b@1 > 3 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------MemoryExec: partitions=1, partition_sizes=[1] -11)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] -12)--------CoalesceBatchesExec: target_batch_size=3 -13)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -14)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)--------------MemoryExec: partitions=1, partition_sizes=[1] +06)----------CoalesceBatchesExec: target_batch_size=3 +07)------------FilterExec: b@1 > 3, projection=[a@0] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)----------------MemoryExec: partitions=1, partition_sizes=[1] +10)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] +11)--------CoalesceBatchesExec: target_batch_size=3 +12)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +13)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)--------------MemoryExec: partitions=1, partition_sizes=[1] query TT explain select * from test where a in (select a from test where b > 3) order by c desc nulls last; @@ -4376,16 +4373,15 @@ physical_plan 03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(a@0, a@0)] 04)------CoalesceBatchesExec: target_batch_size=3 05)--------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -06)----------ProjectionExec: expr=[a@0 as a] -07)------------CoalesceBatchesExec: target_batch_size=3 -08)--------------FilterExec: b@1 > 3 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -10)------------------MemoryExec: partitions=1, partition_sizes=[1] -11)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] -12)--------CoalesceBatchesExec: target_batch_size=3 -13)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -14)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -15)--------------MemoryExec: partitions=1, partition_sizes=[1] +06)----------CoalesceBatchesExec: target_batch_size=3 +07)------------FilterExec: b@1 > 3, projection=[a@0] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)----------------MemoryExec: partitions=1, partition_sizes=[1] +10)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] +11)--------CoalesceBatchesExec: target_batch_size=3 +12)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +13)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)--------------MemoryExec: partitions=1, partition_sizes=[1] query III select * from test where a in (select a from test where b > 3) order by c desc nulls first; diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index 392022dc386a..806886b07170 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -110,11 +110,10 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----ProjectionExec: expr=[a@0 as a] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------FilterExec: b@1 > 2 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 -07)------------ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], predicate=b@1 > 2, pruning_predicate=b_null_count@1 != b_row_count@2 AND b_max@0 > 2, required_guarantees=[] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------FilterExec: b@1 > 2, projection=[a@0] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +06)----------ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], predicate=b@1 > 2, pruning_predicate=b_null_count@1 != b_row_count@2 AND b_max@0 > 2, required_guarantees=[] # also test querying on columns that are not in all the files query T diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 4695e37aa560..878d7c8a4dfb 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -770,14 +770,13 @@ physical_plan 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true 10)----------CoalesceBatchesExec: target_batch_size=8192 11)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -12)--------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -13)----------------CoalesceBatchesExec: target_batch_size=8192 -14)------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23 -15)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true -17)------CoalesceBatchesExec: target_batch_size=8192 -18)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 -19)----------MemoryExec: partitions=1, partition_sizes=[1] +12)--------------CoalesceBatchesExec: target_batch_size=8192 +13)----------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] +14)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true +16)------CoalesceBatchesExec: target_batch_size=8192 +17)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 +18)----------MemoryExec: partitions=1, partition_sizes=[1] # Inlist simplification diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 5f7aedafaefe..8895a2986103 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -1146,19 +1146,18 @@ logical_plan 05)------SubqueryAlias: __correlated_sq_1 06)--------TableScan: t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----FilterExec: t1_id@0 > 40 OR NOT mark@3 -04)------CoalesceBatchesExec: target_batch_size=2 -05)--------HashJoinExec: mode=Partitioned, join_type=LeftMark, on=[(t1_id@0, t2_id@0)] -06)----------CoalesceBatchesExec: target_batch_size=2 -07)------------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------MemoryExec: partitions=1, partition_sizes=[1] -10)----------CoalesceBatchesExec: target_batch_size=2 -11)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -12)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)----------------MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=2 +02)--FilterExec: t1_id@0 > 40 OR NOT mark@3, projection=[t1_id@0, t1_name@1, t1_int@2] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=Partitioned, join_type=LeftMark, on=[(t1_id@0, t2_id@0)] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)--------CoalesceBatchesExec: target_batch_size=2 +10)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +11)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -1386,4 +1385,4 @@ WHERE moving_sum > 60; item1 1970-01-01T00:00:03 75 statement ok -drop table source_table; \ No newline at end of file +drop table source_table; diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part index b6417bc89306..34d63f67efdf 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part @@ -56,5 +56,5 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[l_returnflag@5 as l_returnflag, l_linestatus@6 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(*)] 08)--------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------FilterExec: l_shipdate@6 <= 1998-09-02 +10)------------------FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5] 11)--------------------CsvExec: 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_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index d9924f5f279f..e4376437ad4f 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -92,17 +92,15 @@ physical_plan 21)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment], has_header=false 22)----------------------------------CoalesceBatchesExec: target_batch_size=8192 23)------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -24)--------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -25)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)------------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01 -27)--------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false -28)--------------------------CoalesceBatchesExec: target_batch_size=8192 -29)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -30)------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -31)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)----------------------------------FilterExec: l_returnflag@3 = R -33)------------------------------------CsvExec: 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_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false -34)------------------CoalesceBatchesExec: target_batch_size=8192 -35)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -36)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -37)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +24)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)----------------------------------------FilterExec: o_orderdate@2 >= 1993-10-01 AND o_orderdate@2 < 1994-01-01, projection=[o_orderkey@0, o_custkey@1] +26)------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +27)--------------------------CoalesceBatchesExec: target_batch_size=8192 +28)----------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +29)------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------------FilterExec: l_returnflag@3 = R, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +31)----------------------------------CsvExec: 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_orderkey, l_extendedprice, l_discount, l_returnflag], has_header=false +32)------------------CoalesceBatchesExec: target_batch_size=8192 +33)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +34)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +35)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index 515d532562a8..3928684a6824 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -96,32 +96,30 @@ physical_plan 21)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false 22)--------------------CoalesceBatchesExec: target_batch_size=8192 23)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -24)------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -25)--------------------------CoalesceBatchesExec: target_batch_size=8192 -26)----------------------------FilterExec: n_name@1 = GERMANY -27)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -28)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -29)------ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] -30)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -31)----------CoalescePartitionsExec -32)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] -33)--------------CoalesceBatchesExec: target_batch_size=8192 -34)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -35)------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -37)----------------------CoalesceBatchesExec: target_batch_size=8192 -38)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -39)--------------------------CoalesceBatchesExec: target_batch_size=8192 -40)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -41)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], has_header=false -42)--------------------------CoalesceBatchesExec: target_batch_size=8192 -43)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -44)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -45)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -46)------------------CoalesceBatchesExec: target_batch_size=8192 -47)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -48)----------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -49)------------------------CoalesceBatchesExec: target_batch_size=8192 -50)--------------------------FilterExec: n_name@1 = GERMANY -51)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -52)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +24)------------------------CoalesceBatchesExec: target_batch_size=8192 +25)--------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +26)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +28)------ProjectionExec: expr=[CAST(CAST(sum(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as sum(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] +29)--------AggregateExec: mode=Final, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +30)----------CoalescePartitionsExec +31)------------AggregateExec: mode=Partial, gby=[], aggr=[sum(partsupp.ps_supplycost * partsupp.ps_availqty)] +32)--------------CoalesceBatchesExec: target_batch_size=8192 +33)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +34)------------------CoalesceBatchesExec: target_batch_size=8192 +35)--------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +36)----------------------CoalesceBatchesExec: target_batch_size=8192 +37)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +38)--------------------------CoalesceBatchesExec: target_batch_size=8192 +39)----------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +40)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_suppkey, ps_availqty, ps_supplycost], has_header=false +41)--------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +43)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +44)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +45)------------------CoalesceBatchesExec: target_batch_size=8192 +46)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +47)----------------------CoalesceBatchesExec: target_batch_size=8192 +48)------------------------FilterExec: n_name@1 = GERMANY, projection=[n_nationkey@0] +49)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +50)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index 561cda701d5a..34e5dd69f410 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -70,10 +70,9 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -12)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01 -15)----------------------------CsvExec: 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_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], has_header=false -16)------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -18)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], has_header=false +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4] +14)--------------------------CsvExec: 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_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], has_header=false +15)------------------CoalesceBatchesExec: target_batch_size=8192 +16)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +17)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 8f36bca81cf0..2667bb23a459 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -70,7 +70,6 @@ physical_plan 15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false 16)----------------------CoalesceBatchesExec: target_batch_size=8192 17)------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -18)--------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -19)----------------------------CoalesceBatchesExec: target_batch_size=8192 -20)------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% -21)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false +18)--------------------------CoalesceBatchesExec: target_batch_size=8192 +19)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] +20)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part index f791a89373a5..96cfbb02d07c 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -50,11 +50,10 @@ physical_plan 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4] 08)--------------CoalesceBatchesExec: target_batch_size=8192 09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -10)------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01 -13)------------------------CsvExec: 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_extendedprice, l_discount, l_shipdate], has_header=false -14)--------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -16)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------FilterExec: l_shipdate@3 >= 1995-09-01 AND l_shipdate@3 < 1995-10-01, projection=[l_partkey@0, l_extendedprice@1, l_discount@2] +12)----------------------CsvExec: 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_extendedprice, l_discount, l_shipdate], has_header=false +13)--------------CoalesceBatchesExec: target_batch_size=8192 +14)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +15)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index 81c255396e6c..4cf4c722eaf8 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -88,21 +88,19 @@ physical_plan 15)--------------------CoalesceBatchesExec: target_batch_size=8192 16)----------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 17)------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -18)--------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -19)----------------------------CoalesceBatchesExec: target_batch_size=8192 -20)------------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01 -21)--------------------------------CsvExec: 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_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false -22)--------CoalesceBatchesExec: target_batch_size=8192 -23)----------RepartitionExec: partitioning=Hash([max(revenue0.total_revenue)@0], 4), input_partitions=1 -24)------------AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] -25)--------------CoalescePartitionsExec -26)----------------AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] -27)------------------ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -28)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -29)----------------------CoalesceBatchesExec: target_batch_size=8192 -30)------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -31)--------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -32)----------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -33)------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)--------------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01 -35)----------------------------------CsvExec: 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_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +18)--------------------------CoalesceBatchesExec: target_batch_size=8192 +19)----------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +20)------------------------------CsvExec: 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_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +21)--------CoalesceBatchesExec: target_batch_size=8192 +22)----------RepartitionExec: partitioning=Hash([max(revenue0.total_revenue)@0], 4), input_partitions=1 +23)------------AggregateExec: mode=Final, gby=[], aggr=[max(revenue0.total_revenue)] +24)--------------CoalescePartitionsExec +25)----------------AggregateExec: mode=Partial, gby=[], aggr=[max(revenue0.total_revenue)] +26)------------------ProjectionExec: expr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +27)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +28)----------------------CoalesceBatchesExec: target_batch_size=8192 +29)------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +30)--------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +31)----------------------------CoalesceBatchesExec: target_batch_size=8192 +32)------------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] +33)--------------------------------CsvExec: 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_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index 35e0ec305093..93021d371e6f 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -93,8 +93,7 @@ physical_plan 26)--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_type, p_size], has_header=false 27)--------------------------CoalesceBatchesExec: target_batch_size=8192 28)----------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -29)------------------------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey] -30)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)----------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints% -32)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -33)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], has_header=false +29)------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints%, projection=[s_suppkey@0] +31)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +32)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index 44bda7cfc4a1..5fc2973d6fa2 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -64,14 +64,13 @@ physical_plan 11)--------------------CsvExec: 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], has_header=false 12)----------------CoalesceBatchesExec: target_batch_size=8192 13)------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -14)--------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -15)----------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX -17)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false -19)------------ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] -20)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -21)----------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -23)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] -24)----------------------CsvExec: 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], has_header=false +14)--------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX, projection=[p_partkey@0] +16)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false +18)------------ProjectionExec: expr=[CAST(0.2 * CAST(avg(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * avg(lineitem.l_quantity), l_partkey@0 as l_partkey] +19)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +20)----------------CoalesceBatchesExec: target_batch_size=8192 +21)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +22)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] +23)----------------------CsvExec: 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], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index ef1392fec430..283de4dddfe3 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -91,11 +91,10 @@ physical_plan 22)--------------------CoalesceBatchesExec: target_batch_size=8192 23)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 24)------------------------CsvExec: 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_orderkey, l_quantity], has_header=false -25)----------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] -26)------------------CoalesceBatchesExec: target_batch_size=8192 -27)--------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2 -28)----------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -29)------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -31)----------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] -32)------------------------------CsvExec: 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_orderkey, l_quantity], has_header=false +25)----------------CoalesceBatchesExec: target_batch_size=8192 +26)------------------FilterExec: sum(lineitem.l_quantity)@1 > Some(30000),25,2, projection=[l_orderkey@0] +27)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +28)----------------------CoalesceBatchesExec: target_batch_size=8192 +29)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +30)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] +31)----------------------------CsvExec: 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_orderkey, l_quantity], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part index 38dc9729f003..9eb1f8638292 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -72,13 +72,12 @@ physical_plan 06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_extendedprice@2, l_discount@3] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -09)----------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------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 -12)----------------------CsvExec: 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], has_header=false -13)------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -15)----------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------FilterExec: (p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 -17)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false +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] +11)--------------------CsvExec: 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], has_header=false +12)------------CoalesceBatchesExec: target_batch_size=8192 +13)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +14)----------------CoalesceBatchesExec: target_batch_size=8192 +15)------------------FilterExec: (p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 +16)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index eae0f69f8d9e..8fdbb6884d14 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -123,61 +123,58 @@ physical_plan 22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)], projection=[p_partkey@0, p_mfgr@1, ps_suppkey@3, ps_supplycost@4] 23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 24)----------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -25)------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] -26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS -28)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -29)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], has_header=false -30)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -32)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false -33)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -35)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -36)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false -37)--------------------------CoalesceBatchesExec: target_batch_size=8192 -38)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -39)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -40)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false -41)------------------CoalesceBatchesExec: target_batch_size=8192 -42)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -43)----------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -44)------------------------CoalesceBatchesExec: target_batch_size=8192 -45)--------------------------FilterExec: r_name@1 = EUROPE -46)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -47)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false -48)----------CoalesceBatchesExec: target_batch_size=8192 -49)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 -50)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] -51)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -52)------------------CoalesceBatchesExec: target_batch_size=8192 -53)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -54)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] -55)------------------------CoalesceBatchesExec: target_batch_size=8192 -56)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -57)----------------------------CoalesceBatchesExec: target_batch_size=8192 -58)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -59)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -60)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -61)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -62)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -63)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -64)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] -65)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -66)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -67)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false -68)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -69)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -70)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -71)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -72)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -73)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -74)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -75)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -76)----------------------------CoalesceBatchesExec: target_batch_size=8192 -77)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -78)--------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -79)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -80)------------------------------------FilterExec: r_name@1 = EUROPE -81)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -82)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS, projection=[p_partkey@0, p_mfgr@1] +27)----------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +28)------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_mfgr, p_type, p_size], has_header=false +29)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +31)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +32)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +34)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +35)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false +36)--------------------------CoalesceBatchesExec: target_batch_size=8192 +37)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +38)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +39)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false +40)------------------CoalesceBatchesExec: target_batch_size=8192 +41)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +42)----------------------CoalesceBatchesExec: target_batch_size=8192 +43)------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +44)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +45)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +46)----------CoalesceBatchesExec: target_batch_size=8192 +47)------------RepartitionExec: partitioning=Hash([ps_partkey@1, min(partsupp.ps_supplycost)@0], 4), input_partitions=4 +48)--------------ProjectionExec: expr=[min(partsupp.ps_supplycost)@1 as min(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +49)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +50)------------------CoalesceBatchesExec: target_batch_size=8192 +51)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +52)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[min(partsupp.ps_supplycost)] +53)------------------------CoalesceBatchesExec: target_batch_size=8192 +54)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +55)----------------------------CoalesceBatchesExec: target_batch_size=8192 +56)------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +57)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +58)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +59)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +60)--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +61)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +62)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +63)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +64)----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +65)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +66)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +67)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +68)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +69)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +70)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +71)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +72)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +73)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +74)----------------------------CoalesceBatchesExec: target_batch_size=8192 +75)------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +76)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +77)----------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] +78)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +79)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index aef9b0636e39..4d0cb1bc7e1d 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -95,35 +95,32 @@ physical_plan 12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey], has_header=false 13)----------------CoalesceBatchesExec: target_batch_size=8192 14)------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -15)--------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -16)----------------------CoalesceBatchesExec: target_batch_size=8192 -17)------------------------FilterExec: n_name@1 = CANADA -18)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -20)--------CoalesceBatchesExec: target_batch_size=8192 -21)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -22)------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] -24)----------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 -26)--------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -28)------------------------CoalesceBatchesExec: target_batch_size=8192 -29)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -30)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], has_header=false -31)------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -33)----------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -34)------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)--------------------------------FilterExec: p_name@1 LIKE forest% -36)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -37)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false -38)----------------ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] -39)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -40)--------------------CoalesceBatchesExec: target_batch_size=8192 -41)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -42)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] -43)--------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] -44)----------------------------CoalesceBatchesExec: target_batch_size=8192 -45)------------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 -46)--------------------------------CsvExec: 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_suppkey, l_quantity, l_shipdate], has_header=false +15)--------------------CoalesceBatchesExec: target_batch_size=8192 +16)----------------------FilterExec: n_name@1 = CANADA, projection=[n_nationkey@0] +17)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +19)--------CoalesceBatchesExec: target_batch_size=8192 +20)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +21)------------CoalesceBatchesExec: target_batch_size=8192 +22)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * sum(lineitem.l_quantity)@1, projection=[ps_suppkey@1] +23)----------------CoalesceBatchesExec: target_batch_size=8192 +24)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +25)--------------------CoalesceBatchesExec: target_batch_size=8192 +26)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +27)------------------------CoalesceBatchesExec: target_batch_size=8192 +28)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +29)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_availqty], has_header=false +30)------------------------CoalesceBatchesExec: target_batch_size=8192 +31)--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +32)----------------------------CoalesceBatchesExec: target_batch_size=8192 +33)------------------------------FilterExec: p_name@1 LIKE forest%, projection=[p_partkey@0] +34)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +35)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +36)----------------ProjectionExec: expr=[0.5 * CAST(sum(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * sum(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] +37)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +38)--------------------CoalesceBatchesExec: target_batch_size=8192 +39)----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +40)------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[sum(lineitem.l_quantity)] +41)--------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] +43)------------------------------CsvExec: 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_suppkey, l_quantity, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index f88a57536ed6..939e01b1784e 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -119,29 +119,25 @@ physical_plan 27)----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_nationkey], has_header=false 28)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 29)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -30)--------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] -31)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 -33)--------------------------------------------------------CsvExec: 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_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false -34)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -36)------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] -37)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)----------------------------------------------FilterExec: o_orderstatus@1 = F -39)------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], has_header=false -40)------------------------------CoalesceBatchesExec: target_batch_size=8192 -41)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -42)----------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -43)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -44)--------------------------------------FilterExec: n_name@1 = SAUDI ARABIA -45)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -46)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -47)----------------------CoalesceBatchesExec: target_batch_size=8192 -48)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -49)--------------------------CsvExec: 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_orderkey, l_suppkey], has_header=false -50)------------------CoalesceBatchesExec: target_batch_size=8192 -51)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -52)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] -53)------------------------CoalesceBatchesExec: target_batch_size=8192 -54)--------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 -55)----------------------------CsvExec: 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_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false +30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)----------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +32)------------------------------------------------------CsvExec: 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_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false +33)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------------------FilterExec: o_orderstatus@1 = F, projection=[o_orderkey@0] +37)----------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderstatus], has_header=false +38)------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +40)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +41)------------------------------------FilterExec: n_name@1 = SAUDI ARABIA, projection=[n_nationkey@0] +42)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +43)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +44)----------------------CoalesceBatchesExec: target_batch_size=8192 +45)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +46)--------------------------CsvExec: 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_orderkey, l_suppkey], has_header=false +47)------------------CoalesceBatchesExec: target_batch_size=8192 +48)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +49)----------------------CoalesceBatchesExec: target_batch_size=8192 +50)------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] +51)--------------------------CsvExec: 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_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index 802239d4e269..a55d1e8fdfb1 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -99,8 +99,7 @@ physical_plan 23)--------------------AggregateExec: mode=Final, gby=[], aggr=[avg(customer.c_acctbal)] 24)----------------------CoalescePartitionsExec 25)------------------------AggregateExec: mode=Partial, gby=[], aggr=[avg(customer.c_acctbal)] -26)--------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] -27)----------------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]) -29)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false +26)--------------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]), projection=[c_acctbal@1] +28)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +29)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part index c4fb84a4bf44..0f35239bfd9d 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -73,19 +73,17 @@ physical_plan 13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 15)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -16)------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] -17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------FilterExec: c_mktsegment@1 = BUILDING -19)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -20)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false -21)--------------------------CoalesceBatchesExec: target_batch_size=8192 -22)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -23)------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)--------------------------------FilterExec: o_orderdate@2 < 1995-03-15 -25)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], has_header=false -26)------------------CoalesceBatchesExec: target_batch_size=8192 -27)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -28)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -29)------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------FilterExec: l_shipdate@3 > 1995-03-15 -31)----------------------------CsvExec: 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_orderkey, l_extendedprice, l_discount, l_shipdate], has_header=false +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------FilterExec: c_mktsegment@1 = BUILDING, projection=[c_custkey@0] +18)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +19)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false +20)--------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +22)------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------FilterExec: o_orderdate@2 < 1995-03-15 +24)----------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], has_header=false +25)------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +27)----------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] +29)--------------------------CsvExec: 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_orderkey, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part index 854718bb50ac..693d37a5b53d 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -64,13 +64,11 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -12)----------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01 -15)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], has_header=false -16)------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -18)----------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] -19)------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------FilterExec: l_receiptdate@2 > l_commitdate@1 -21)----------------------------CsvExec: 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_orderkey, l_commitdate, l_receiptdate], has_header=false +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------FilterExec: o_orderdate@1 >= 1993-07-01 AND o_orderdate@1 < 1993-10-01, projection=[o_orderkey@0, o_orderpriority@2] +14)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate, o_orderpriority], has_header=false +15)------------------CoalesceBatchesExec: target_batch_size=8192 +16)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +17)----------------------CoalesceBatchesExec: target_batch_size=8192 +18)------------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] +19)--------------------------CsvExec: 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_orderkey, l_commitdate, l_receiptdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index e23d39f2437e..0c03e0498d64 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -98,25 +98,23 @@ physical_plan 29)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false 30)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 31)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -32)------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -33)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01 -35)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false -36)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -37)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -38)----------------------------------------------CsvExec: 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_orderkey, l_suppkey, l_extendedprice, l_discount], has_header=false -39)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -40)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 -41)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -42)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -43)--------------------------CoalesceBatchesExec: target_batch_size=8192 -44)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -45)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -46)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false -47)------------------CoalesceBatchesExec: target_batch_size=8192 -48)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -49)----------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -50)------------------------CoalesceBatchesExec: target_batch_size=8192 -51)--------------------------FilterExec: r_name@1 = ASIA -52)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -53)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +32)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)--------------------------------------------------------FilterExec: o_orderdate@2 >= 1994-01-01 AND o_orderdate@2 < 1995-01-01, projection=[o_orderkey@0, o_custkey@1] +34)----------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +35)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +37)----------------------------------------------CsvExec: 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_orderkey, l_suppkey, l_extendedprice, l_discount], has_header=false +38)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 +40)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +41)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +42)--------------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +44)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +45)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name, n_regionkey], has_header=false +46)------------------CoalesceBatchesExec: target_batch_size=8192 +47)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +48)----------------------CoalesceBatchesExec: target_batch_size=8192 +49)------------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] +50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +51)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part index 3797a2593f7c..6c818a1e2f42 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part @@ -38,7 +38,6 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[sum(lineitem.l_extendedprice * lineitem.l_discount)] -05)--------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2 -08)--------------CsvExec: 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_quantity, l_extendedprice, l_discount, l_shipdate], has_header=false +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2] +07)------------CsvExec: 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_quantity, l_extendedprice, l_discount, l_shipdate], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index babdbc28b18b..2bcab40dc985 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -126,39 +126,37 @@ physical_plan 34)------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] 35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 36)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -37)------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -38)--------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)----------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL -40)------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -41)--------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false -42)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -43)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -44)------------------------------------------------------------------------CsvExec: 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_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false -45)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -46)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -47)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -48)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -49)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -50)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -51)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -52)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 -53)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false -54)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -55)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -56)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -57)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -58)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -59)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -60)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -61)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -62)----------------------------CoalesceBatchesExec: target_batch_size=8192 -63)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -64)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -65)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -66)--------------------CoalesceBatchesExec: target_batch_size=8192 -67)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -68)------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -69)--------------------------CoalesceBatchesExec: target_batch_size=8192 -70)----------------------------FilterExec: r_name@1 = AMERICA -71)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -72)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +37)------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)--------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL, projection=[p_partkey@0] +39)----------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +40)------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +41)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +43)------------------------------------------------------------------------CsvExec: 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_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false +44)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +45)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +46)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +47)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +48)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +49)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +50)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +51)----------------------------------------------------------FilterExec: o_orderdate@2 >= 1995-01-01 AND o_orderdate@2 <= 1996-12-31 +52)------------------------------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_orderdate], has_header=false +53)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +54)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +55)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +56)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +57)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +58)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +59)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +60)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +61)----------------------------CoalesceBatchesExec: target_batch_size=8192 +62)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +63)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +64)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +65)--------------------CoalesceBatchesExec: target_batch_size=8192 +66)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +67)------------------------CoalesceBatchesExec: target_batch_size=8192 +68)--------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] +69)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +70)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index 0361f1b21cb5..4a288893da95 100644 --- a/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -103,25 +103,24 @@ physical_plan 26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_partkey@2, l_suppkey@3, l_quantity@4, l_extendedprice@5, l_discount@6] 27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 28)------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -29)--------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -30)----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)------------------------------------------------------------FilterExec: p_name@1 LIKE %green% -32)--------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -33)----------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false -34)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -36)--------------------------------------------------------CsvExec: 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_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false -37)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -39)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -40)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -41)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -42)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -43)----------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false -44)----------------------------CoalesceBatchesExec: target_batch_size=8192 -45)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -46)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], has_header=false -47)--------------------CoalesceBatchesExec: target_batch_size=8192 -48)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -49)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -50)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------------------------------------FilterExec: p_name@1 LIKE %green%, projection=[p_partkey@0] +31)------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +32)--------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +33)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +35)--------------------------------------------------------CsvExec: 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_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false +36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +38)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +39)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +40)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +41)--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +42)----------------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:0..2932049], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:2932049..5864098], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:5864098..8796147], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/partsupp.tbl:8796147..11728193]]}, projection=[ps_partkey, ps_suppkey, ps_supplycost], has_header=false +43)----------------------------CoalesceBatchesExec: target_batch_size=8192 +44)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +45)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderdate], has_header=false +46)--------------------CoalesceBatchesExec: target_batch_size=8192 +47)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +48)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +49)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/tpch.slt b/datafusion/sqllogictest/test_files/tpch/tpch.slt index 127152daa580..c6d630997e29 100644 --- a/datafusion/sqllogictest/test_files/tpch/tpch.slt +++ b/datafusion/sqllogictest/test_files/tpch/tpch.slt @@ -28,4 +28,3 @@ set datafusion.optimizer.prefer_hash_join = false; include ./answers/q*.slt.part include ./drop_tables.slt.part - diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 214c7f4de1d8..7b8992b966ad 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -522,17 +522,16 @@ physical_plan 10)------------------CoalesceBatchesExec: target_batch_size=2 11)--------------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 12)----------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] -13)------------------------ProjectionExec: expr=[c1@0 as c1] -14)--------------------------CoalesceBatchesExec: target_batch_size=2 -15)----------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 -16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true -18)------ProjectionExec: expr=[1 as cnt] -19)--------PlaceholderRowExec -20)------ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] -21)--------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -22)----------ProjectionExec: expr=[1 as c1] -23)------------PlaceholderRowExec +13)------------------------CoalesceBatchesExec: target_batch_size=2 +14)--------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] +15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true +17)------ProjectionExec: expr=[1 as cnt] +18)--------PlaceholderRowExec +19)------ProjectionExec: expr=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as cnt] +20)--------BoundedWindowAggExec: wdw=[lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "lead(b.c1,Int64(1)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +21)----------ProjectionExec: expr=[1 as c1] +22)------------PlaceholderRowExec ######## diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 64f2b0da5769..3e387008ebcc 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1782,11 +1782,10 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=4096 08)--------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] -10)------------------ProjectionExec: expr=[c1@0 as c1] -11)--------------------CoalesceBatchesExec: target_batch_size=4096 -12)----------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 -13)------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -14)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true +10)------------------CoalesceBatchesExec: target_batch_size=4096 +11)--------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] +12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true query I From af91dc7e255183e8a5d75ced35334710e328a96b Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Sat, 25 Jan 2025 19:43:00 +0300 Subject: [PATCH 10/12] Update projection_pushdown.rs --- .../physical_optimizer/projection_pushdown.rs | 51 +++++++++---------- 1 file changed, 24 insertions(+), 27 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index ebe7de6809ce..88414e68e8c5 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -20,23 +20,16 @@ //! projections one by one if the operator below is amenable to this. If a //! projection reaches a source, it can even disappear from the plan entirely. -use std::collections::HashMap; use std::sync::Arc; use crate::error::Result; use crate::physical_plan::ExecutionPlan; use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{ - Transformed, TransformedResult, TreeNode, TreeNodeRecursion, -}; -use datafusion_physical_expr::expressions::{Column, Literal}; -use datafusion_physical_expr::PhysicalExpr; +use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_plan::projection::{ - remove_unnecessary_projections, update_expr, ProjectionExec, -}; +use datafusion_physical_plan::projection::remove_unnecessary_projections; /// This rule inspects [`ProjectionExec`]'s in the given physical plan and tries to /// remove or swap with its child. @@ -73,30 +66,18 @@ mod tests { use std::any::Any; use super::*; - use crate::datasource::physical_plan::CsvExec; - use crate::physical_plan::memory::MemoryExec; - use crate::physical_plan::repartition::RepartitionExec; - use crate::physical_plan::sorts::sort::SortExec; - use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; - use datafusion_physical_expr::{ - Distribution, Partitioning, PhysicalExpr, PhysicalSortExpr, - PhysicalSortRequirement, - }; - use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; - use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; - use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; - use datafusion_physical_plan::filter::FilterExec; - use datafusion_physical_plan::joins::utils::{ColumnIndex, JoinFilter}; - use datafusion_physical_plan::streaming::StreamingTableExec; - use datafusion_physical_plan::union::UnionExec; - use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::PartitionedFile; + use crate::datasource::physical_plan::CsvExec; use crate::datasource::physical_plan::FileScanConfig; use crate::physical_plan::get_plan_string; use crate::physical_plan::joins::{ HashJoinExec, NestedLoopJoinExec, StreamJoinPartitionMode, SymmetricHashJoinExec, }; + use crate::physical_plan::memory::MemoryExec; + use crate::physical_plan::repartition::RepartitionExec; + use crate::physical_plan::sorts::sort::SortExec; + use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; use datafusion_common::{JoinSide, JoinType, ScalarValue}; @@ -106,11 +87,27 @@ mod tests { ColumnarValue, Operator, ScalarUDF, ScalarUDFImpl, Signature, Volatility, }; use datafusion_physical_expr::expressions::{ - binary, col, BinaryExpr, CaseExpr, CastExpr, NegativeExpr, + binary, col, BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, }; use datafusion_physical_expr::ScalarFunctionExpr; + use datafusion_physical_expr::{ + Distribution, Partitioning, PhysicalExpr, PhysicalSortExpr, + PhysicalSortRequirement, + }; + use datafusion_physical_expr::{ + Distribution, Partitioning, PhysicalExpr, PhysicalSortExpr, + PhysicalSortRequirement, + }; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; + use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; + use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; + use datafusion_physical_plan::filter::FilterExec; + use datafusion_physical_plan::joins::utils::{ColumnIndex, JoinFilter}; use datafusion_physical_plan::joins::PartitionMode; + use datafusion_physical_plan::projection::{update_expr, ProjectionExec}; use datafusion_physical_plan::streaming::PartitionStream; + use datafusion_physical_plan::streaming::StreamingTableExec; + use datafusion_physical_plan::union::UnionExec; use itertools::Itertools; From d1c19ffc90a92031ae005af6841b67fa9cf2f60b Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Sat, 25 Jan 2025 19:46:55 +0300 Subject: [PATCH 11/12] minor fmt --- .../core/src/physical_optimizer/projection_pushdown.rs | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 88414e68e8c5..46fddbd3c936 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -31,7 +31,7 @@ use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::projection::remove_unnecessary_projections; -/// This rule inspects [`ProjectionExec`]'s in the given physical plan and tries to +/// This rule inspects `ProjectionExec`'s in the given physical plan and tries to /// remove or swap with its child. #[derive(Default, Debug)] pub struct ProjectionPushdown {} @@ -94,10 +94,6 @@ mod tests { Distribution, Partitioning, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; - use datafusion_physical_expr::{ - Distribution, Partitioning, PhysicalExpr, PhysicalSortExpr, - PhysicalSortRequirement, - }; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; From ac188931d204837d5cabdfc07ed7e620c54ea329 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Sat, 25 Jan 2025 21:29:45 +0300 Subject: [PATCH 12/12] fix imports --- .../src/output_requirements.rs | 15 +- .../physical-plan/src/execution_plan.rs | 3 +- datafusion/physical-plan/src/filter.rs | 2 +- .../physical-plan/src/joins/cross_join.rs | 5 +- .../physical-plan/src/joins/hash_join.rs | 6 +- .../src/joins/nested_loop_join.rs | 2 +- .../src/joins/sort_merge_join.rs | 38 +- .../src/joins/symmetric_hash_join.rs | 2 +- datafusion/physical-plan/src/memory.rs | 2 +- datafusion/physical-plan/src/projection.rs | 441 +++++++++--------- .../physical-plan/src/repartition/mod.rs | 2 +- .../src/sorts/sort_preserving_merge.rs | 2 +- datafusion/physical-plan/src/streaming.rs | 4 +- datafusion/sqllogictest/test_files/expr.slt | 1 - 14 files changed, 261 insertions(+), 264 deletions(-) diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 98962ced5661..90a570894a44 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -24,24 +24,23 @@ use std::sync::Arc; +use crate::PhysicalOptimizerRule; + +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::{Result, Statistics}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::{Distribution, LexRequirement, PhysicalSortRequirement}; use datafusion_physical_plan::projection::{ make_with_child, update_expr, ProjectionExec, }; use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, SendableRecordBatchStream, }; - -use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::{Result, Statistics}; -use datafusion_physical_expr::{Distribution, LexRequirement, PhysicalSortRequirement}; -use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{ExecutionPlanProperties, PlanProperties}; -use crate::PhysicalOptimizerRule; - /// This rule either adds or removes [`OutputRequirements`]s to/from the physical /// plan according to its `mode` attribute, which is set by the constructors /// `new_add_mode` and `new_remove_mode`. With this rule, we can keep track of diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index aa7371b2de99..a54b46111f53 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -441,7 +441,8 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// `self_new <- projection <- source`, starting from `projection <- self <- source`. /// Otherwise, it returns the current `ExecutionPlan` as-is. /// - /// Returns `Ok(Some(...))` if pushdown is applied, `Ok(None)` if unsupported, or `Err` on failure. + /// Returns `Ok(Some(...))` if pushdown is applied, `Ok(None)` if it is not supported + /// or not possible, or `Err` on failure. fn try_swapping_with_projection( &self, _projection: &ProjectionExec, diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 78ab1e3441d5..ae4a15ba5249 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -25,6 +25,7 @@ use super::{ RecordBatchStream, SendableRecordBatchStream, Statistics, }; use crate::common::can_project; +use crate::execution_plan::CardinalityEffect; use crate::projection::{ make_with_child, try_embed_projection, update_expr, EmbeddedProjection, ProjectionExec, @@ -53,7 +54,6 @@ use datafusion_physical_expr::{ ExprBoundaries, PhysicalExpr, }; -use crate::execution_plan::CardinalityEffect; use futures::stream::{Stream, StreamExt}; use log::trace; diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 5a46897cb2b4..87fd0f96586a 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -18,6 +18,8 @@ //! Defines the cross join plan for loading the left side of the cross join //! and producing batches in parallel for the right partitions +use std::{any::Any, sync::Arc, task::Poll}; + use super::utils::{ adjust_right_output_partitioning, reorder_output_after_swap, BatchSplitter, BatchTransformer, BuildProbeJoinMetrics, NoopBatchTransformer, OnceAsync, OnceFut, @@ -35,9 +37,8 @@ use crate::{ ExecutionPlan, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use arrow::compute::concat_batches; -use std::{any::Any, sync::Arc, task::Poll}; +use arrow::compute::concat_batches; use arrow::datatypes::{Fields, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow_array::RecordBatchOptions; diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 222dded4f13f..bac72e8a0cc7 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -37,6 +37,7 @@ use crate::projection::{ try_embed_projection, try_pushdown_through_join, EmbeddedProjection, JoinData, ProjectionExec, }; +use crate::spill::get_record_batch_memory_size; use crate::ExecutionPlanProperties; use crate::{ coalesce_partitions::CoalescePartitionsExec, @@ -73,15 +74,14 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; +use datafusion_expr::Operator; use datafusion_physical_expr::equivalence::{ join_equivalence_properties, ProjectionMapping, }; use datafusion_physical_expr::PhysicalExprRef; +use datafusion_physical_expr_common::datum::compare_op_for_nested; -use crate::spill::get_record_batch_memory_size; use ahash::RandomState; -use datafusion_expr::Operator; -use datafusion_physical_expr_common::datum::compare_op_for_nested; use futures::{ready, Stream, StreamExt, TryStreamExt}; use parking_lot::Mutex; diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 0c4546d6c18c..ce960df32ec2 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -36,6 +36,7 @@ use crate::joins::utils::{ build_join_schema, check_join_is_valid, estimate_join_statistics, BuildProbeJoinMetrics, ColumnIndex, JoinFilter, OnceAsync, OnceFut, }; +use crate::joins::SharedBitmapBuilder; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::projection::{ try_embed_projection, try_pushdown_through_join, EmbeddedProjection, JoinData, @@ -61,7 +62,6 @@ use datafusion_physical_expr::equivalence::{ join_equivalence_properties, ProjectionMapping, }; -use crate::joins::SharedBitmapBuilder; use futures::{ready, Stream, StreamExt, TryStreamExt}; use parking_lot::Mutex; diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index efa8dcef50a8..e03aca76409d 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -34,6 +34,25 @@ use std::sync::atomic::Ordering::Relaxed; use std::sync::Arc; use std::task::{Context, Poll}; +use crate::execution_plan::{boundedness_from_children, EmissionType}; +use crate::expressions::PhysicalSortExpr; +use crate::joins::utils::{ + build_join_schema, check_join_is_valid, estimate_join_statistics, + reorder_output_after_swap, symmetric_join_output_partitioning, JoinFilter, JoinOn, + JoinOnRef, +}; +use crate::metrics::{Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; +use crate::projection::{ + join_allows_pushdown, join_table_borders, new_join_children, + physical_to_column_exprs, update_join_on, ProjectionExec, +}; +use crate::spill::spill_record_batches; +use crate::{ + metrics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, + ExecutionPlanProperties, PhysicalExpr, PlanProperties, RecordBatchStream, + SendableRecordBatchStream, Statistics, +}; + use arrow::array::*; use arrow::compute::{ self, concat_batches, filter_record_batch, is_not_null, take, SortOptions, @@ -54,25 +73,6 @@ use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::PhysicalExprRef; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; -use crate::execution_plan::{boundedness_from_children, EmissionType}; -use crate::expressions::PhysicalSortExpr; -use crate::joins::utils::{ - build_join_schema, check_join_is_valid, estimate_join_statistics, - reorder_output_after_swap, symmetric_join_output_partitioning, JoinFilter, JoinOn, - JoinOnRef, -}; -use crate::metrics::{Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; -use crate::projection::{ - join_allows_pushdown, join_table_borders, new_join_children, - physical_to_column_exprs, update_join_on, ProjectionExec, -}; -use crate::spill::spill_record_batches; -use crate::{ - metrics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, - ExecutionPlanProperties, PhysicalExpr, PlanProperties, RecordBatchStream, - SendableRecordBatchStream, Statistics, -}; - use futures::{Stream, StreamExt}; /// Join execution plan that executes equi-join predicates on multiple partitions using Sort-Merge diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 7739f2481d77..2dbff25e703e 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -75,9 +75,9 @@ use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; use datafusion_physical_expr::PhysicalExprRef; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use ahash::RandomState; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::{ready, Stream, StreamExt}; use parking_lot::Mutex; diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index b182aa927189..5ad3c4881b39 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -17,7 +17,6 @@ //! Execution plan for reading in-memory batches of data -use parking_lot::RwLock; use std::any::Any; use std::fmt; use std::sync::Arc; @@ -48,6 +47,7 @@ use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use futures::Stream; +use parking_lot::RwLock; /// Execution plan for reading in-memory batches of data #[derive(Clone)] diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index f52e6dd4549e..b364d4a870e3 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -32,11 +32,12 @@ use super::{ DisplayAs, ExecutionPlanProperties, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; - use crate::execution_plan::CardinalityEffect; use crate::joins::utils::{ColumnIndex, JoinFilter}; use crate::{ColumnStatistics, DisplayFormatType, ExecutionPlan, PhysicalExpr}; +use arrow::datatypes::{Field, Schema, SchemaRef}; +use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::stats::Precision; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, @@ -45,15 +46,11 @@ use datafusion_common::{internal_err, JoinSide, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::utils::collect_columns; - -use arrow::datatypes::{Field, Schema, SchemaRef}; -use arrow::record_batch::{RecordBatch, RecordBatchOptions}; - use datafusion_physical_expr::PhysicalExprRef; -use futures::stream::{Stream, StreamExt}; -use log::trace; +use futures::stream::{Stream, StreamExt}; use itertools::Itertools; +use log::trace; /// Execution plan for a projection #[derive(Debug, Clone)] @@ -375,6 +372,181 @@ impl RecordBatchStream for ProjectionStream { } } +pub trait EmbeddedProjection: ExecutionPlan + Sized { + fn with_projection(&self, projection: Option>) -> Result; +} + +/// Some projection can't be pushed down left input or right input of hash join because filter or on need may need some columns that won't be used in later. +/// By embed those projection to hash join, we can reduce the cost of build_batch_from_indices in hash join (build_batch_from_indices need to can compute::take() for each column) and avoid unnecessary output creation. +pub fn try_embed_projection( + projection: &ProjectionExec, + execution_plan: &Exec, +) -> Result>> { + // Collect all column indices from the given projection expressions. + let projection_index = collect_column_indices(projection.expr()); + + if projection_index.is_empty() { + return Ok(None); + }; + + // If the projection indices is the same as the input columns, we don't need to embed the projection to hash join. + // Check the projection_index is 0..n-1 and the length of projection_index is the same as the length of execution_plan schema fields. + if projection_index.len() == projection_index.last().unwrap() + 1 + && projection_index.len() == execution_plan.schema().fields().len() + { + return Ok(None); + } + + let new_execution_plan = + Arc::new(execution_plan.with_projection(Some(projection_index.to_vec()))?); + + // Build projection expressions for update_expr. Zip the projection_index with the new_execution_plan output schema fields. + let embed_project_exprs = projection_index + .iter() + .zip(new_execution_plan.schema().fields()) + .map(|(index, field)| { + ( + Arc::new(Column::new(field.name(), *index)) as Arc, + field.name().to_owned(), + ) + }) + .collect::>(); + + let mut new_projection_exprs = Vec::with_capacity(projection.expr().len()); + + for (expr, alias) in projection.expr() { + // update column index for projection expression since the input schema has been changed. + let Some(expr) = update_expr(expr, embed_project_exprs.as_slice(), false)? else { + return Ok(None); + }; + new_projection_exprs.push((expr, alias.clone())); + } + // Old projection may contain some alias or expression such as `a + 1` and `CAST('true' AS BOOLEAN)`, but our projection_exprs in hash join just contain column, so we need to create the new projection to keep the original projection. + let new_projection = Arc::new(ProjectionExec::try_new( + new_projection_exprs, + Arc::clone(&new_execution_plan) as _, + )?); + if is_projection_removable(&new_projection) { + Ok(Some(new_execution_plan)) + } else { + Ok(Some(new_projection)) + } +} + +/// The on clause of the join, as vector of (left, right) columns. +pub type JoinOn = Vec<(PhysicalExprRef, PhysicalExprRef)>; +/// Reference for JoinOn. +pub type JoinOnRef<'a> = &'a [(PhysicalExprRef, PhysicalExprRef)]; + +pub struct JoinData { + pub projected_left_child: ProjectionExec, + pub projected_right_child: ProjectionExec, + pub join_filter: Option, + pub join_on: JoinOn, +} + +pub fn try_pushdown_through_join( + projection: &ProjectionExec, + join_left: &Arc, + join_right: &Arc, + join_on: JoinOnRef, + schema: SchemaRef, + filter: Option<&JoinFilter>, +) -> Result> { + // Convert projected expressions to columns. We can not proceed if this is not possible. + let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { + return Ok(None); + }; + + let (far_right_left_col_ind, far_left_right_col_ind) = + join_table_borders(join_left.schema().fields().len(), &projection_as_columns); + + if !join_allows_pushdown( + &projection_as_columns, + &schema, + far_right_left_col_ind, + far_left_right_col_ind, + ) { + return Ok(None); + } + + let new_filter = if let Some(filter) = filter { + match update_join_filter( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + filter, + join_left.schema().fields().len(), + ) { + Some(updated_filter) => Some(updated_filter), + None => return Ok(None), + } + } else { + None + }; + + let Some(new_on) = update_join_on( + &projection_as_columns[0..=far_right_left_col_ind as _], + &projection_as_columns[far_left_right_col_ind as _..], + join_on, + join_left.schema().fields().len(), + ) else { + return Ok(None); + }; + + let (new_left, new_right) = new_join_children( + &projection_as_columns, + far_right_left_col_ind, + far_left_right_col_ind, + join_left, + join_right, + )?; + + Ok(Some(JoinData { + projected_left_child: new_left, + projected_right_child: new_right, + join_filter: new_filter, + join_on: new_on, + })) +} + +/// This function checks if `plan` is a [`ProjectionExec`], and inspects its +/// input(s) to test whether it can push `plan` under its input(s). This function +/// will operate on the entire tree and may ultimately remove `plan` entirely +/// by leveraging source providers with built-in projection capabilities. +pub fn remove_unnecessary_projections( + plan: Arc, +) -> Result>> { + let maybe_modified = + if let Some(projection) = plan.as_any().downcast_ref::() { + // If the projection does not cause any change on the input, we can + // safely remove it: + if is_projection_removable(projection) { + return Ok(Transformed::yes(Arc::clone(projection.input()))); + } + // If it does, check if we can push it under its child(ren): + projection + .input() + .try_swapping_with_projection(projection)? + } else { + return Ok(Transformed::no(plan)); + }; + Ok(maybe_modified.map_or(Transformed::no(plan), Transformed::yes)) +} + +/// Compare the inputs and outputs of the projection. All expressions must be +/// columns without alias, and projection does not change the order of fields. +/// For example, if the input schema is `a, b`, `SELECT a, b` is removable, +/// but `SELECT b, a` and `SELECT a+1, b` and `SELECT a AS c, b` are not. +fn is_projection_removable(projection: &ProjectionExec) -> bool { + let exprs = projection.expr(); + exprs.iter().enumerate().all(|(idx, (expr, alias))| { + let Some(col) = expr.as_any().downcast_ref::() else { + return false; + }; + col.name() == alias && col.index() == idx + }) && exprs.len() == projection.input().schema().fields().len() +} + /// Given the expression set of a projection, checks if the projection causes /// any renaming or constructs a non-`Column` physical expression. pub fn all_alias_free_columns(exprs: &[(Arc, String)]) -> bool { @@ -499,18 +671,6 @@ pub fn update_expr( new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) } -/// The on clause of the join, as vector of (left, right) columns. -pub type JoinOn = Vec<(PhysicalExprRef, PhysicalExprRef)>; -/// Reference for JoinOn. -pub type JoinOnRef<'a> = &'a [(PhysicalExprRef, PhysicalExprRef)]; - -pub struct JoinData { - pub projected_left_child: ProjectionExec, - pub projected_right_child: ProjectionExec, - pub join_filter: Option, - pub join_on: JoinOn, -} - /// Downcasts all the expressions in `exprs` to `Column`s. If any of the given /// expressions is not a `Column`, returns `None`. pub fn physical_to_column_exprs( @@ -526,127 +686,6 @@ pub fn physical_to_column_exprs( .collect() } -/// Some projection can't be pushed down left input or right input of hash join because filter or on need may need some columns that won't be used in later. -/// By embed those projection to hash join, we can reduce the cost of build_batch_from_indices in hash join (build_batch_from_indices need to can compute::take() for each column) and avoid unnecessary output creation. -pub fn try_embed_projection( - projection: &ProjectionExec, - execution_plan: &Exec, -) -> Result>> { - // Collect all column indices from the given projection expressions. - let projection_index = collect_column_indices(projection.expr()); - - if projection_index.is_empty() { - return Ok(None); - }; - - // If the projection indices is the same as the input columns, we don't need to embed the projection to hash join. - // Check the projection_index is 0..n-1 and the length of projection_index is the same as the length of execution_plan schema fields. - if projection_index.len() == projection_index.last().unwrap() + 1 - && projection_index.len() == execution_plan.schema().fields().len() - { - return Ok(None); - } - - let new_execution_plan = - Arc::new(execution_plan.with_projection(Some(projection_index.to_vec()))?); - - // Build projection expressions for update_expr. Zip the projection_index with the new_execution_plan output schema fields. - let embed_project_exprs = projection_index - .iter() - .zip(new_execution_plan.schema().fields()) - .map(|(index, field)| { - ( - Arc::new(Column::new(field.name(), *index)) as Arc, - field.name().to_owned(), - ) - }) - .collect::>(); - - let mut new_projection_exprs = Vec::with_capacity(projection.expr().len()); - - for (expr, alias) in projection.expr() { - // update column index for projection expression since the input schema has been changed. - let Some(expr) = update_expr(expr, embed_project_exprs.as_slice(), false)? else { - return Ok(None); - }; - new_projection_exprs.push((expr, alias.clone())); - } - // Old projection may contain some alias or expression such as `a + 1` and `CAST('true' AS BOOLEAN)`, but our projection_exprs in hash join just contain column, so we need to create the new projection to keep the original projection. - let new_projection = Arc::new(ProjectionExec::try_new( - new_projection_exprs, - Arc::clone(&new_execution_plan) as _, - )?); - if is_projection_removable(&new_projection) { - Ok(Some(new_execution_plan)) - } else { - Ok(Some(new_projection)) - } -} - -pub fn try_pushdown_through_join( - projection: &ProjectionExec, - join_left: &Arc, - join_right: &Arc, - join_on: JoinOnRef, - schema: SchemaRef, - filter: Option<&JoinFilter>, -) -> Result> { - // Convert projected expressions to columns. We can not proceed if this is not possible. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = - join_table_borders(join_left.schema().fields().len(), &projection_as_columns); - - if !join_allows_pushdown( - &projection_as_columns, - &schema, - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let new_filter = if let Some(filter) = filter { - match update_join_filter( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - filter, - join_left.schema().fields().len(), - ) { - Some(updated_filter) => Some(updated_filter), - None => return Ok(None), - } - } else { - None - }; - - let Some(new_on) = update_join_on( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - join_on, - join_left.schema().fields().len(), - ) else { - return Ok(None); - }; - - let (new_left, new_right) = new_join_children( - &projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - join_left, - join_right, - )?; - - Ok(Some(JoinData { - projected_left_child: new_left, - projected_right_child: new_right, - join_filter: new_filter, - join_on: new_on, - })) -} - /// If pushing down the projection over this join's children seems possible, /// this function constructs the new [`ProjectionExec`]s that will come on top /// of the original children of the join. @@ -817,8 +856,46 @@ pub fn update_join_filter( }) } -pub trait EmbeddedProjection: ExecutionPlan + Sized { - fn with_projection(&self, projection: Option>) -> Result; +/// Unifies `projection` with its input (which is also a [`ProjectionExec`]). +fn try_unifying_projections( + projection: &ProjectionExec, + child: &ProjectionExec, +) -> Result>> { + let mut projected_exprs = vec![]; + let mut column_ref_map: HashMap = HashMap::new(); + + // Collect the column references usage in the outer projection. + projection.expr().iter().for_each(|(expr, _)| { + expr.apply(|expr| { + Ok({ + if let Some(column) = expr.as_any().downcast_ref::() { + *column_ref_map.entry(column.clone()).or_default() += 1; + } + TreeNodeRecursion::Continue + }) + }) + .unwrap(); + }); + // Merging these projections is not beneficial, e.g + // If an expression is not trivial and it is referred more than 1, unifies projections will be + // beneficial as caching mechanism for non-trivial computations. + // See discussion in: https://github.com/apache/datafusion/issues/8296 + if column_ref_map.iter().any(|(column, count)| { + *count > 1 && !is_expr_trivial(&Arc::clone(&child.expr()[column.index()].0)) + }) { + return Ok(None); + } + for (expr, alias) in projection.expr() { + // If there is no match in the input projection, we cannot unify these + // projections. This case will arise if the projection expression contains + // a `PhysicalExpr` variant `update_expr` doesn't support. + let Some(expr) = update_expr(expr, child.expr(), true)? else { + return Ok(None); + }; + projected_exprs.push((expr, alias.clone())); + } + ProjectionExec::try_new(projected_exprs, Arc::clone(child.input())) + .map(|e| Some(Arc::new(e) as _)) } /// Collect all column indices from the given projection expressions. @@ -911,48 +988,6 @@ fn new_columns_for_join_on( (new_columns.len() == hash_join_on.len()).then_some(new_columns) } -/// Unifies `projection` with its input (which is also a [`ProjectionExec`]). -fn try_unifying_projections( - projection: &ProjectionExec, - child: &ProjectionExec, -) -> Result>> { - let mut projected_exprs = vec![]; - let mut column_ref_map: HashMap = HashMap::new(); - - // Collect the column references usage in the outer projection. - projection.expr().iter().for_each(|(expr, _)| { - expr.apply(|expr| { - Ok({ - if let Some(column) = expr.as_any().downcast_ref::() { - *column_ref_map.entry(column.clone()).or_default() += 1; - } - TreeNodeRecursion::Continue - }) - }) - .unwrap(); - }); - // Merging these projections is not beneficial, e.g - // If an expression is not trivial and it is referred more than 1, unifies projections will be - // beneficial as caching mechanism for non-trivial computations. - // See discussion in: https://github.com/apache/datafusion/issues/8296 - if column_ref_map.iter().any(|(column, count)| { - *count > 1 && !is_expr_trivial(&Arc::clone(&child.expr()[column.index()].0)) - }) { - return Ok(None); - } - for (expr, alias) in projection.expr() { - // If there is no match in the input projection, we cannot unify these - // projections. This case will arise if the projection expression contains - // a `PhysicalExpr` variant `update_expr` doesn't support. - let Some(expr) = update_expr(expr, child.expr(), true)? else { - return Ok(None); - }; - projected_exprs.push((expr, alias.clone())); - } - ProjectionExec::try_new(projected_exprs, Arc::clone(child.input())) - .map(|e| Some(Arc::new(e) as _)) -} - /// Checks if the given expression is trivial. /// An expression is considered trivial if it is either a `Column` or a `Literal`. fn is_expr_trivial(expr: &Arc) -> bool { @@ -960,44 +995,6 @@ fn is_expr_trivial(expr: &Arc) -> bool { || expr.as_any().downcast_ref::().is_some() } -/// This function checks if `plan` is a [`ProjectionExec`], and inspects its -/// input(s) to test whether it can push `plan` under its input(s). This function -/// will operate on the entire tree and may ultimately remove `plan` entirely -/// by leveraging source providers with built-in projection capabilities. -pub fn remove_unnecessary_projections( - plan: Arc, -) -> Result>> { - let maybe_modified = - if let Some(projection) = plan.as_any().downcast_ref::() { - // If the projection does not cause any change on the input, we can - // safely remove it: - if is_projection_removable(projection) { - return Ok(Transformed::yes(Arc::clone(projection.input()))); - } - // If it does, check if we can push it under its child(ren): - projection - .input() - .try_swapping_with_projection(projection)? - } else { - return Ok(Transformed::no(plan)); - }; - Ok(maybe_modified.map_or(Transformed::no(plan), Transformed::yes)) -} - -/// Compare the inputs and outputs of the projection. All expressions must be -/// columns without alias, and projection does not change the order of fields. -/// For example, if the input schema is `a, b`, `SELECT a, b` is removable, -/// but `SELECT b, a` and `SELECT a+1, b` and `SELECT a AS c, b` are not. -fn is_projection_removable(projection: &ProjectionExec) -> bool { - let exprs = projection.expr(); - exprs.iter().enumerate().all(|(idx, (expr, alias))| { - let Some(col) = expr.as_any().downcast_ref::() else { - return false; - }; - col.name() == alias && col.index() == idx - }) && exprs.len() == projection.input().schema().fields().len() -} - #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 262e0e648b30..8d180c212eba 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -45,6 +45,7 @@ use arrow::datatypes::{SchemaRef, UInt32Type}; use arrow::record_batch::RecordBatch; use arrow_array::{PrimitiveArray, RecordBatchOptions}; use datafusion_common::utils::transpose; +use datafusion_common::HashMap; use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; @@ -52,7 +53,6 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_common::HashMap; use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; use log::trace; diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 152f3270f88d..4f4237204fb1 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -33,9 +33,9 @@ use crate::{ use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; - use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; + use log::{debug, trace}; /// Sort preserving merge execution plan diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 1af378baf03b..751af9921448 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -24,6 +24,8 @@ use std::sync::Arc; use super::{DisplayAs, DisplayFormatType, PlanProperties}; use crate::display::{display_orderings, ProjectSchemaDisplay}; use crate::execution_plan::{Boundedness, EmissionType}; +use crate::limit::LimitStream; +use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::projection::{ all_alias_free_columns, new_projections_for_columns, update_expr, ProjectionExec, }; @@ -36,8 +38,6 @@ use datafusion_common::{internal_err, plan_err, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; -use crate::limit::LimitStream; -use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use async_trait::async_trait; use futures::stream::StreamExt; use log::debug; diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index 13dc74b6cb7d..b2aa4b3fac8f 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -2046,5 +2046,4 @@ select 1 where null between null and 2; query T select 'A' where null between 2 and null; ---- ----