diff --git a/Cargo.toml b/Cargo.toml index 1dae101d2f8fc..ae3ce0bf6cbf6 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -17,7 +17,7 @@ [workspace] exclude = ["datafusion-cli"] -members = ["datafusion/common", "datafusion/core", "datafusion/expr", "datafusion/execution", "datafusion/optimizer", "datafusion/physical-expr", "datafusion/proto", "datafusion/proto/gen", "datafusion/sql", "datafusion/sqllogictest", "datafusion/substrait", "datafusion-examples", "test-utils", "benchmarks", +members = ["datafusion/common", "datafusion/core", "datafusion/expr", "datafusion/execution", "datafusion/optimizer", "datafusion/physical-expr", "datafusion/physical-plan", "datafusion/proto", "datafusion/proto/gen", "datafusion/sql", "datafusion/sqllogictest", "datafusion/substrait", "datafusion-examples", "test-utils", "benchmarks", ] resolver = "2" diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 51d36c5a90d6b..f2cd44a95706a 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1066,6 +1066,7 @@ dependencies = [ "datafusion-expr", "datafusion-optimizer", "datafusion-physical-expr", + "datafusion-physical-plan", "datafusion-sql", "flate2", "futures", @@ -1077,7 +1078,6 @@ dependencies = [ "log", "num_cpus", "object_store", - "once_cell", "parking_lot", "parquet", "percent-encoding", @@ -1112,7 +1112,7 @@ dependencies = [ "parking_lot", "predicates", "regex", - "rstest", + "rstest 0.17.0", "rustyline", "tokio", "url", @@ -1218,6 +1218,37 @@ dependencies = [ "uuid", ] +[[package]] +name = "datafusion-physical-plan" +version = "31.0.0" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-schema", + "async-trait", + "chrono", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "futures", + "half", + "hashbrown 0.14.0", + "indexmap 2.0.0", + "itertools 0.11.0", + "log", + "once_cell", + "parking_lot", + "pin-project-lite", + "rand", + "rstest 0.18.2", + "tempfile", + "termtree", + "tokio", +] + [[package]] name = "datafusion-sql" version = "31.0.0" @@ -2603,6 +2634,12 @@ version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dbb5fb1acd8a1a18b3dd5be62d25485eb770e05afb408a9627d14d451bae12da" +[[package]] +name = "relative-path" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c707298afce11da2efef2f600116fa93ffa7a032b5d7b628aa17711ec81383ca" + [[package]] name = "reqwest" version = "0.11.20" @@ -2667,7 +2704,19 @@ checksum = "de1bb486a691878cd320c2f0d319ba91eeaa2e894066d8b5f8f117c000e9d962" dependencies = [ "futures", "futures-timer", - "rstest_macros", + "rstest_macros 0.17.0", + "rustc_version", +] + +[[package]] +name = "rstest" +version = "0.18.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97eeab2f3c0a199bc4be135c36c924b6590b88c377d416494288c14f2db30199" +dependencies = [ + "futures", + "futures-timer", + "rstest_macros 0.18.2", "rustc_version", ] @@ -2685,6 +2734,23 @@ dependencies = [ "unicode-ident", ] +[[package]] +name = "rstest_macros" +version = "0.18.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d428f8247852f894ee1be110b375111b586d4fa431f6c46e64ba5a0dcccbe605" +dependencies = [ + "cfg-if", + "glob", + "proc-macro2", + "quote", + "regex", + "relative-path", + "rustc_version", + "syn 2.0.33", + "unicode-ident", +] + [[package]] name = "rustc-demangle" version = "0.1.23" diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 5f02d92e50c9b..420bcd963c305 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -63,6 +63,9 @@ pub use table_reference::{OwnedTableReference, ResolvedTableReference, TableRefe pub use unnest::UnnestOptions; pub use utils::project_schema; +/// Reexport arrow crate +pub use arrow; + /// Downcast an Arrow Array to a concrete type, return an `DataFusionError::Internal` if the cast is /// not possible. In normal usage of DataFusion the downcast should always succeed. /// diff --git a/datafusion/common/src/test_util.rs b/datafusion/common/src/test_util.rs index 048b0d58d8c4d..60f1df7fd11ac 100644 --- a/datafusion/common/src/test_util.rs +++ b/datafusion/common/src/test_util.rs @@ -19,6 +19,85 @@ use std::{error::Error, path::PathBuf}; +/// Compares formatted output of a record batch with an expected +/// vector of strings, with the result of pretty formatting record +/// batches. This is a macro so errors appear on the correct line +/// +/// Designed so that failure output can be directly copy/pasted +/// into the test code as expected results. +/// +/// Expects to be called about like this: +/// +/// `assert_batch_eq!(expected_lines: &[&str], batches: &[RecordBatch])` +#[macro_export] +macro_rules! assert_batches_eq { + ($EXPECTED_LINES: expr, $CHUNKS: expr) => { + let expected_lines: Vec = + $EXPECTED_LINES.iter().map(|&s| s.into()).collect(); + + let formatted = $crate::arrow::util::pretty::pretty_format_batches_with_options( + $CHUNKS, + &$crate::format::DEFAULT_FORMAT_OPTIONS, + ) + .unwrap() + .to_string(); + + let actual_lines: Vec<&str> = formatted.trim().lines().collect(); + + assert_eq!( + expected_lines, actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + }; +} + +/// Compares formatted output of a record batch with an expected +/// vector of strings in a way that order does not matter. +/// This is a macro so errors appear on the correct line +/// +/// Designed so that failure output can be directly copy/pasted +/// into the test code as expected results. +/// +/// Expects to be called about like this: +/// +/// `assert_batch_sorted_eq!(expected_lines: &[&str], batches: &[RecordBatch])` +#[macro_export] +macro_rules! assert_batches_sorted_eq { + ($EXPECTED_LINES: expr, $CHUNKS: expr) => { + let mut expected_lines: Vec = + $EXPECTED_LINES.iter().map(|&s| s.into()).collect(); + + // sort except for header + footer + let num_lines = expected_lines.len(); + if num_lines > 3 { + expected_lines.as_mut_slice()[2..num_lines - 1].sort_unstable() + } + + let formatted = $crate::arrow::util::pretty::pretty_format_batches_with_options( + $CHUNKS, + &$crate::format::DEFAULT_FORMAT_OPTIONS, + ) + .unwrap() + .to_string(); + // fix for windows: \r\n --> + + let mut actual_lines: Vec<&str> = formatted.trim().lines().collect(); + + // sort except for header + footer + let num_lines = actual_lines.len(); + if num_lines > 3 { + actual_lines.as_mut_slice()[2..num_lines - 1].sort_unstable() + } + + assert_eq!( + expected_lines, actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + }; +} + /// A macro to assert that one string is contained within another with /// a nice error message if they are not. /// diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index aec31dea1047c..f414d875bf6c7 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -68,6 +68,7 @@ datafusion-execution = { path = "../execution", version = "31.0.0" } datafusion-expr = { path = "../expr", version = "31.0.0" } datafusion-optimizer = { path = "../optimizer", version = "31.0.0", default-features = false } datafusion-physical-expr = { path = "../physical-expr", version = "31.0.0", default-features = false } +datafusion-physical-plan = { path = "../physical-plan", version = "31.0.0", default-features = false } datafusion-sql = { path = "../sql", version = "31.0.0" } flate2 = { version = "1.0.24", optional = true } futures = "0.3" @@ -80,7 +81,6 @@ log = "^0.4" num-traits = { version = "0.2", optional = true } num_cpus = "1.13.0" object_store = "0.7.0" -once_cell = "1.18.0" parking_lot = "0.12" parquet = { workspace = true } percent-encoding = "2.2.0" @@ -112,7 +112,6 @@ rand_distr = "0.4.3" regex = "1.5.4" rstest = "0.18.0" rust_decimal = { version = "1.27.0", features = ["tokio-pg"] } -termtree = "0.4.1" test-utils = { path = "../../test-utils" } thiserror = "1.0.37" tokio-postgres = "0.7.7" diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index e9ce5238c5962..819bfabae2902 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -496,11 +496,10 @@ fn create_output_array( #[cfg(test)] mod tests { + use arrow_array::Int32Array; + use super::*; - use crate::{ - test::{build_table_i32, columns}, - test_util::aggr_test_schema, - }; + use crate::{test::columns, test_util::aggr_test_schema}; #[test] fn physical_plan_config_no_projection() { @@ -776,4 +775,27 @@ mod tests { infinite_source: false, } } + + /// returns record batch with 3 columns of i32 in memory + pub fn build_table_i32( + a: (&str, &Vec), + b: (&str, &Vec), + c: (&str, &Vec), + ) -> RecordBatch { + let schema = Schema::new(vec![ + Field::new(a.0, DataType::Int32, false), + Field::new(b.0, DataType::Int32, false), + Field::new(c.0, DataType::Int32, false), + ]); + + RecordBatch::try_new( + Arc::new(schema), + vec![ + Arc::new(Int32Array::from(a.1.clone())), + Arc::new(Int32Array::from(b.1.clone())), + Arc::new(Int32Array::from(c.1.clone())), + ], + ) + .unwrap() + } } diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index c0ed13c8e063d..576f66a5ed7c1 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -428,7 +428,6 @@ pub mod datasource; pub mod error; pub mod execution; pub mod physical_optimizer; -pub mod physical_plan; pub mod physical_planner; pub mod prelude; pub mod scalar; @@ -467,6 +466,15 @@ pub mod physical_expr { pub use datafusion_physical_expr::*; } +/// re-export of [`datafusion_physical_plan`] crate +pub mod physical_plan { + pub use datafusion_physical_plan::*; +} + +// Reexport testing macros for compatibility +pub use datafusion_common::assert_batches_eq; +pub use datafusion_common::assert_batches_sorted_eq; + /// re-export of [`datafusion_sql`] crate pub mod sql { pub use datafusion_sql::*; diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index de47f3fbee730..40b2bcc3e140e 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -50,74 +50,59 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { _config: &ConfigOptions, ) -> Result> { plan.transform_down(&|plan| { - let transformed = plan.as_any().downcast_ref::().and_then( - |AggregateExec { - mode: final_mode, - input: final_input, - group_by: final_group_by, - aggr_expr: final_aggr_expr, - filter_expr: final_filter_expr, - .. - }| { - if matches!( - final_mode, - AggregateMode::Final | AggregateMode::FinalPartitioned - ) { - final_input - .as_any() - .downcast_ref::() - .and_then( - |AggregateExec { - mode: input_mode, - input: partial_input, - group_by: input_group_by, - aggr_expr: input_aggr_expr, - filter_expr: input_filter_expr, - order_by_expr: input_order_by_expr, - input_schema, - .. - }| { - if matches!(input_mode, AggregateMode::Partial) - && can_combine( - ( - final_group_by, - final_aggr_expr, - final_filter_expr, - ), - ( - input_group_by, - input_aggr_expr, - input_filter_expr, - ), - ) - { - let mode = if *final_mode == AggregateMode::Final - { - AggregateMode::Single - } else { - AggregateMode::SinglePartitioned - }; + let transformed = + plan.as_any() + .downcast_ref::() + .and_then(|agg_exec| { + if matches!( + agg_exec.mode(), + AggregateMode::Final | AggregateMode::FinalPartitioned + ) { + agg_exec + .input() + .as_any() + .downcast_ref::() + .and_then(|input_agg_exec| { + if matches!( + input_agg_exec.mode(), + AggregateMode::Partial + ) && can_combine( + ( + agg_exec.group_by(), + agg_exec.aggr_expr(), + agg_exec.filter_expr(), + ), + ( + input_agg_exec.group_by(), + input_agg_exec.aggr_expr(), + input_agg_exec.filter_expr(), + ), + ) { + let mode = + if agg_exec.mode() == &AggregateMode::Final { + AggregateMode::Single + } else { + AggregateMode::SinglePartitioned + }; AggregateExec::try_new( mode, - input_group_by.clone(), - input_aggr_expr.to_vec(), - input_filter_expr.to_vec(), - input_order_by_expr.to_vec(), - partial_input.clone(), - input_schema.clone(), + input_agg_exec.group_by().clone(), + input_agg_exec.aggr_expr().to_vec(), + input_agg_exec.filter_expr().to_vec(), + input_agg_exec.order_by_expr().to_vec(), + input_agg_exec.input().clone(), + input_agg_exec.input_schema().clone(), ) .ok() .map(Arc::new) } else { None } - }, - ) - } else { - None - } - }, - ); + }) + } else { + None + } + }); Ok(if let Some(transformed) = transformed { Transformed::Yes(transformed) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 3bfe9ae3df5d0..77d6e7d7123de 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -379,7 +379,7 @@ fn adjust_input_keys_ordering( )?) } else if let Some(aggregate_exec) = plan_any.downcast_ref::() { if !parent_required.is_empty() { - match aggregate_exec.mode { + match aggregate_exec.mode() { AggregateMode::FinalPartitioned => Some(reorder_aggregate_keys( requirements.plan.clone(), &parent_required, @@ -391,9 +391,8 @@ fn adjust_input_keys_ordering( // Keep everything unchanged None } - } else if let Some(ProjectionExec { expr, .. }) = - plan_any.downcast_ref::() - { + } else if let Some(proj) = plan_any.downcast_ref::() { + let expr = proj.expr(); // For Projection, we need to transform the requirements to the columns before the Projection // And then to push down the requirements // Construct a mapping from new name to the the orginal Column @@ -488,7 +487,7 @@ fn reorder_aggregate_keys( agg_exec: &AggregateExec, ) -> Result { let out_put_columns = agg_exec - .group_by + .group_by() .expr() .iter() .enumerate() @@ -501,7 +500,7 @@ fn reorder_aggregate_keys( .collect::>(); if parent_required.len() != out_put_exprs.len() - || !agg_exec.group_by.null_expr().is_empty() + || !agg_exec.group_by().null_expr().is_empty() || expr_list_eq_strict_order(&out_put_exprs, parent_required) { Ok(PlanWithKeyRequirements::new(agg_plan)) @@ -510,7 +509,9 @@ fn reorder_aggregate_keys( match new_positions { None => Ok(PlanWithKeyRequirements::new(agg_plan)), Some(positions) => { - let new_partial_agg = if let Some(AggregateExec { + let new_partial_agg = if let Some(agg_exec) = + agg_exec.input().as_any().downcast_ref::() + /*AggregateExec { mode, group_by, aggr_expr, @@ -520,12 +521,13 @@ fn reorder_aggregate_keys( input_schema, .. }) = - agg_exec.input.as_any().downcast_ref::() + */ { - if matches!(mode, AggregateMode::Partial) { + if matches!(agg_exec.mode(), &AggregateMode::Partial) { let mut new_group_exprs = vec![]; for idx in positions.iter() { - new_group_exprs.push(group_by.expr()[*idx].clone()); + new_group_exprs + .push(agg_exec.group_by().expr()[*idx].clone()); } let new_partial_group_by = PhysicalGroupBy::new_single(new_group_exprs); @@ -533,11 +535,11 @@ fn reorder_aggregate_keys( Some(Arc::new(AggregateExec::try_new( AggregateMode::Partial, new_partial_group_by, - aggr_expr.clone(), - filter_expr.clone(), - order_by_expr.clone(), - input.clone(), - input_schema.clone(), + agg_exec.aggr_expr().to_vec(), + agg_exec.filter_expr().to_vec(), + agg_exec.order_by_expr().to_vec(), + agg_exec.input().clone(), + agg_exec.input_schema.clone(), )?)) } else { None @@ -565,11 +567,11 @@ fn reorder_aggregate_keys( let new_final_agg = Arc::new(AggregateExec::try_new( AggregateMode::FinalPartitioned, new_group_by, - agg_exec.aggr_expr.to_vec(), - agg_exec.filter_expr.to_vec(), - agg_exec.order_by_expr.to_vec(), + agg_exec.aggr_expr().to_vec(), + agg_exec.filter_expr().to_vec(), + agg_exec.order_by_expr().to_vec(), partial_agg, - agg_exec.input_schema.clone(), + agg_exec.input_schema().clone(), )?); // Need to create a new projection to change the expr ordering back diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 628cc1da3b1eb..4cff4a8f6c555 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -579,14 +579,14 @@ fn apply_subrules( #[cfg(test)] mod tests_statistical { + use super::*; use crate::{ physical_plan::{ displayable, joins::PartitionMode, ColumnStatistics, Statistics, }, - test::exec::StatisticsExec, + test::StatisticsExec, }; - use super::*; use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema}; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 81c2e76b74c61..629011cb0faa5 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -140,7 +140,7 @@ pub(crate) fn pushdown_sorts( let parent_required_expr = PhysicalSortRequirement::to_sort_exprs( parent_required.ok_or_else(err)?.iter().cloned(), ); - new_plan = sort_exec.input.clone(); + new_plan = sort_exec.input().clone(); add_sort_above(&mut new_plan, parent_required_expr, sort_exec.fetch())?; }; let required_ordering = new_plan @@ -221,12 +221,12 @@ fn pushdown_requirement_to_children( ])) } else if let Some(smj) = plan.as_any().downcast_ref::() { // If the current plan is SortMergeJoinExec - let left_columns_len = smj.left.schema().fields().len(); + let left_columns_len = smj.left().schema().fields().len(); let parent_required_expr = PhysicalSortRequirement::to_sort_exprs( parent_required.ok_or_else(err)?.iter().cloned(), ); let expr_source_side = - expr_source_sides(&parent_required_expr, smj.join_type, left_columns_len); + expr_source_sides(&parent_required_expr, smj.join_type(), left_columns_len); match expr_source_side { Some(JoinSide::Left) => try_pushdown_requirements_to_join( smj, @@ -236,7 +236,7 @@ fn pushdown_requirement_to_children( ), Some(JoinSide::Right) => { let right_offset = - smj.schema().fields.len() - smj.right.schema().fields.len(); + smj.schema().fields.len() - smj.right().schema().fields.len(); let new_right_required = shift_right_required(parent_required.ok_or_else(err)?, right_offset)?; let new_right_required_expr = PhysicalSortRequirement::to_sort_exprs( @@ -331,8 +331,8 @@ fn try_pushdown_requirements_to_join( sort_expr: Vec, push_side: JoinSide, ) -> Result>>>> { - let left_ordering = smj.left.output_ordering().unwrap_or(&[]); - let right_ordering = smj.right.output_ordering().unwrap_or(&[]); + let left_ordering = smj.left().output_ordering().unwrap_or(&[]); + let right_ordering = smj.right().output_ordering().unwrap_or(&[]); let (new_left_ordering, new_right_ordering) = match push_side { JoinSide::Left => (sort_expr.as_slice(), right_ordering), JoinSide::Right => (left_ordering, sort_expr.as_slice()), @@ -340,11 +340,11 @@ fn try_pushdown_requirements_to_join( let new_output_ordering = calculate_join_output_ordering( new_left_ordering, new_right_ordering, - smj.join_type, - &smj.on, - smj.left.schema().fields.len(), + smj.join_type(), + smj.on(), + smj.left().schema().fields.len(), &smj.maintains_input_order(), - Some(SortMergeJoinExec::probe_side(&smj.join_type)), + Some(SortMergeJoinExec::probe_side(&smj.join_type())), )?; Ok(ordering_satisfy_requirement( new_output_ordering.as_deref(), diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/core/src/physical_optimizer/topk_aggregation.rs index f862675bf2051..4789226d7aa9c 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/topk_aggregation.rs @@ -56,7 +56,7 @@ impl TopKAggregation { if !kt.is_primitive() && kt != DataType::Utf8 { return None; } - if aggr.filter_expr.iter().any(|e| e.is_some()) { + if aggr.filter_expr().iter().any(|e| e.is_some()) { return None; } @@ -67,17 +67,17 @@ impl TopKAggregation { } // We found what we want: clone, copy the limit down, and return modified node - let mut new_aggr = AggregateExec::try_new( - aggr.mode, - aggr.group_by.clone(), - aggr.aggr_expr.clone(), - aggr.filter_expr.clone(), - aggr.order_by_expr.clone(), - aggr.input.clone(), - aggr.input_schema.clone(), + let new_aggr = AggregateExec::try_new( + *aggr.mode(), + aggr.group_by().clone(), + aggr.aggr_expr().to_vec(), + aggr.filter_expr().to_vec(), + aggr.order_by_expr().to_vec(), + aggr.input().clone(), + aggr.input_schema().clone(), ) - .expect("Unable to copy Aggregate!"); - new_aggr.limit = Some(limit); + .expect("Unable to copy Aggregate!") + .with_limit(Some(limit)); Some(Arc::new(new_aggr)) } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 1a9dd7bff0c0a..2328ffce235d6 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -817,8 +817,8 @@ impl DefaultPhysicalPlanner { // into a LAST_VALUE with the reverse ordering requirement. // To reflect such changes to subsequent stages, use the updated // `AggregateExpr`/`PhysicalSortExpr` objects. - let updated_aggregates = initial_aggr.aggr_expr.clone(); - let updated_order_bys = initial_aggr.order_by_expr.clone(); + let updated_aggregates = initial_aggr.aggr_expr().to_vec(); + let updated_order_bys = initial_aggr.order_by_expr().to_vec(); let (initial_aggr, next_partition_mode): ( Arc, diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 011d2e64281ad..a26be4857d4c2 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -17,38 +17,37 @@ //! Common unit test utility methods -use crate::arrow::array::UInt32Array; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; use crate::datasource::{MemTable, TableProvider}; use crate::error::Result; use crate::logical_expr::LogicalPlan; -use crate::physical_plan::memory::MemoryExec; use crate::physical_plan::ExecutionPlan; use crate::test::object_store::local_unpartitioned_file; use crate::test_util::{aggr_test_schema, arrow_test_data}; use array::ArrayRef; use arrow::array::{self, Array, Decimal128Builder, Int32Array}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use arrow::record_batch::{RecordBatch, RecordBatchOptions}; +use arrow::record_batch::RecordBatch; #[cfg(feature = "compression")] use bzip2::write::BzEncoder; #[cfg(feature = "compression")] use bzip2::Compression as BzCompression; use datafusion_common::{DataFusionError, Statistics}; use datafusion_common::{FileCompressionType, FileType}; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::{Partitioning, PhysicalSortExpr}; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; #[cfg(feature = "compression")] use flate2::write::GzEncoder; #[cfg(feature = "compression")] use flate2::Compression as GzCompression; -use futures::{Future, FutureExt}; +use std::any::Any; use std::fs::File; use std::io::prelude::*; use std::io::{BufReader, BufWriter}; use std::path::Path; -use std::pin::Pin; use std::sync::Arc; #[cfg(feature = "compression")] use xz2::write::XzEncoder; @@ -214,40 +213,6 @@ pub fn assert_fields_eq(plan: &LogicalPlan, expected: Vec<&str>) { assert_eq!(actual, expected); } -/// returns record batch with 3 columns of i32 in memory -pub fn build_table_i32( - a: (&str, &Vec), - b: (&str, &Vec), - c: (&str, &Vec), -) -> RecordBatch { - let schema = Schema::new(vec![ - Field::new(a.0, DataType::Int32, false), - Field::new(b.0, DataType::Int32, false), - Field::new(c.0, DataType::Int32, false), - ]); - - RecordBatch::try_new( - Arc::new(schema), - vec![ - Arc::new(Int32Array::from(a.1.clone())), - Arc::new(Int32Array::from(b.1.clone())), - Arc::new(Int32Array::from(c.1.clone())), - ], - ) - .unwrap() -} - -/// returns memory table scan wrapped around record batch with 3 columns of i32 -pub fn build_table_scan_i32( - a: (&str, &Vec), - b: (&str, &Vec), - c: (&str, &Vec), -) -> Arc { - let batch = build_table_i32(a, b, c); - let schema = batch.schema(); - Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) -} - /// Returns the column names on the schema pub fn columns(schema: &Schema) -> Vec { schema.fields().iter().map(|f| f.name().clone()).collect() @@ -280,14 +245,6 @@ pub fn make_partition(sz: i32) -> RecordBatch { RecordBatch::try_new(schema, vec![arr]).unwrap() } -/// Return a RecordBatch with a single array with row_count sz -pub fn make_batch_no_column(sz: usize) -> RecordBatch { - let schema = Arc::new(Schema::empty()); - - let options = RecordBatchOptions::new().with_row_count(Option::from(sz)); - RecordBatch::try_new_with_options(schema, vec![], &options).unwrap() -} - /// Return a new table which provide this decimal column pub fn table_with_decimal() -> Arc { let batch_decimal = make_decimal(); @@ -312,25 +269,6 @@ fn make_decimal() -> RecordBatch { RecordBatch::try_new(Arc::new(schema), vec![Arc::new(array)]).unwrap() } -/// Asserts that given future is pending. -pub fn assert_is_pending<'a, T>(fut: &mut Pin + Send + 'a>>) { - let waker = futures::task::noop_waker(); - let mut cx = futures::task::Context::from_waker(&waker); - let poll = fut.poll_unpin(&mut cx); - - assert!(poll.is_pending()); -} - -/// Create vector batches -pub fn create_vec_batches(schema: &Schema, n: usize) -> Vec { - let batch = create_batch(schema); - let mut vec = Vec::with_capacity(n); - for _ in 0..n { - vec.push(batch.clone()); - } - vec -} - /// Created a sorted Csv exec pub fn csv_exec_sorted( schema: &SchemaRef, @@ -359,15 +297,88 @@ pub fn csv_exec_sorted( )) } -/// Create batch -fn create_batch(schema: &Schema) -> RecordBatch { - RecordBatch::try_new( - Arc::new(schema.clone()), - vec![Arc::new(UInt32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8]))], - ) - .unwrap() +/// A mock execution plan that simply returns the provided statistics +#[derive(Debug, Clone)] +pub struct StatisticsExec { + stats: Statistics, + schema: Arc, +} +impl StatisticsExec { + pub fn new(stats: Statistics, schema: Schema) -> Self { + assert!( + stats + .column_statistics + .as_ref() + .map(|cols| cols.len() == schema.fields().len()) + .unwrap_or(true), + "if defined, the column statistics vector length should be the number of fields" + ); + Self { + stats, + schema: Arc::new(schema), + } + } +} + +impl DisplayAs for StatisticsExec { + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!( + f, + "StatisticsExec: col_count={}, row_count={:?}", + self.schema.fields().len(), + self.stats.num_rows, + ) + } + } + } +} + +impl ExecutionPlan for StatisticsExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(2) + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + None + } + + fn children(&self) -> Vec> { + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> Result { + unimplemented!("This plan only serves for testing statistics") + } + + fn statistics(&self) -> Statistics { + self.stats.clone() + } } -pub mod exec; pub mod object_store; pub mod variable; diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index ab29cecbb8eaa..bd52c3eedaa4f 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -50,84 +50,8 @@ pub use datafusion_common::test_util::{ arrow_test_data, get_data_dir, parquet_test_data, }; -/// Compares formatted output of a record batch with an expected -/// vector of strings, with the result of pretty formatting record -/// batches. This is a macro so errors appear on the correct line -/// -/// Designed so that failure output can be directly copy/pasted -/// into the test code as expected results. -/// -/// Expects to be called about like this: -/// -/// `assert_batch_eq!(expected_lines: &[&str], batches: &[RecordBatch])` -#[macro_export] -macro_rules! assert_batches_eq { - ($EXPECTED_LINES: expr, $CHUNKS: expr) => { - let expected_lines: Vec = - $EXPECTED_LINES.iter().map(|&s| s.into()).collect(); - - let formatted = $crate::arrow::util::pretty::pretty_format_batches_with_options( - $CHUNKS, - &$crate::common::format::DEFAULT_FORMAT_OPTIONS, - ) - .unwrap() - .to_string(); - - let actual_lines: Vec<&str> = formatted.trim().lines().collect(); - - assert_eq!( - expected_lines, actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines - ); - }; -} - -/// Compares formatted output of a record batch with an expected -/// vector of strings in a way that order does not matter. -/// This is a macro so errors appear on the correct line -/// -/// Designed so that failure output can be directly copy/pasted -/// into the test code as expected results. -/// -/// Expects to be called about like this: -/// -/// `assert_batch_sorted_eq!(expected_lines: &[&str], batches: &[RecordBatch])` -#[macro_export] -macro_rules! assert_batches_sorted_eq { - ($EXPECTED_LINES: expr, $CHUNKS: expr) => { - let mut expected_lines: Vec = - $EXPECTED_LINES.iter().map(|&s| s.into()).collect(); - - // sort except for header + footer - let num_lines = expected_lines.len(); - if num_lines > 3 { - expected_lines.as_mut_slice()[2..num_lines - 1].sort_unstable() - } - - let formatted = $crate::arrow::util::pretty::pretty_format_batches_with_options( - $CHUNKS, - &$crate::common::format::DEFAULT_FORMAT_OPTIONS, - ) - .unwrap() - .to_string(); - // fix for windows: \r\n --> - - let mut actual_lines: Vec<&str> = formatted.trim().lines().collect(); - - // sort except for header + footer - let num_lines = actual_lines.len(); - if num_lines > 3 { - actual_lines.as_mut_slice()[2..num_lines - 1].sort_unstable() - } - - assert_eq!( - expected_lines, actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines - ); - }; -} +pub use datafusion_common::assert_batches_eq; +pub use datafusion_common::assert_batches_sorted_eq; /// Scan an empty data source, mainly used in tests pub fn scan_empty( diff --git a/datafusion/core/tests/sql/displayable.rs b/datafusion/core/tests/sql/displayable.rs new file mode 100644 index 0000000000000..b736820009cc9 --- /dev/null +++ b/datafusion/core/tests/sql/displayable.rs @@ -0,0 +1,57 @@ +// 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 object_store::path::Path; + +use datafusion::prelude::*; +use datafusion_physical_plan::displayable; + +#[tokio::test] +async fn teset_displayable() { + // Hard code target_partitions as it appears in the RepartitionExec output + let config = SessionConfig::new().with_target_partitions(3); + let ctx = SessionContext::with_config(config); + + // register the a table + ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()) + .await + .unwrap(); + + // create a plan to run a SQL query + let dataframe = ctx.sql("SELECT a FROM example WHERE a < 5").await.unwrap(); + let physical_plan = dataframe.create_physical_plan().await.unwrap(); + + // Format using display string in verbose mode + let displayable_plan = displayable(physical_plan.as_ref()); + let plan_string = format!("{}", displayable_plan.indent(true)); + + let working_directory = std::env::current_dir().unwrap(); + let normalized = Path::from_filesystem_path(working_directory).unwrap(); + let plan_string = plan_string.replace(normalized.as_ref(), "WORKING_DIR"); + + assert_eq!("CoalesceBatchesExec: target_batch_size=8192\ + \n FilterExec: a@0 < 5\ + \n RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1\ + \n CsvExec: file_groups={1 group: [[WORKING_DIR/tests/data/example.csv]]}, projection=[a], has_header=true", + plan_string.trim()); + + let one_line = format!("{}", displayable_plan.one_line()); + assert_eq!( + "CoalesceBatchesExec: target_batch_size=8192", + one_line.trim() + ); +} diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 173c422942d82..48fb69e51c460 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -83,6 +83,7 @@ pub mod arrow_files; pub mod create_drop; pub mod csv_files; pub mod describe; +pub mod displayable; pub mod explain_analyze; pub mod expr; pub mod group_by; diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml new file mode 100644 index 0000000000000..02f154762760a --- /dev/null +++ b/datafusion/physical-plan/Cargo.toml @@ -0,0 +1,62 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +[package] +name = "datafusion-physical-plan" +description = "Physical (ExecutionPlan) implementations for DataFusion query engine" +keywords = ["arrow", "query", "sql"] +version = { workspace = true } +edition = { workspace = true } +readme = { workspace = true } +homepage = { workspace = true } +repository = { workspace = true } +license = { workspace = true } +authors = { workspace = true } +rust-version = { workspace = true } + +[lib] +name = "datafusion_physical_plan" +path = "src/lib.rs" + +[dependencies] +ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] } +arrow = { workspace = true } +arrow-array = { workspace = true } +arrow-buffer = { workspace = true } +arrow-schema = { workspace = true } +async-trait = "0.1.41" +chrono = { version = "0.4.23", default-features = false } +datafusion-common = { path = "../common", version = "31.0.0" } +datafusion-execution = { path = "../execution", version = "31.0.0" } +datafusion-expr = { path = "../expr", version = "31.0.0" } +datafusion-physical-expr = { path = "../physical-expr", version = "31.0.0" } +futures = "0.3" +half = { version = "2.1", default-features = false } +hashbrown = { version = "0.14", features = ["raw"] } +indexmap = "2.0.0" +itertools = { version = "0.11", features = ["use_std"] } +log = "^0.4" +once_cell = "1.18.0" +parking_lot = "0.12" +pin-project-lite = "^0.2.7" +rand = "0.8" +rstest = "0.18.0" +tempfile = "3" + +#[dev-dependencies] +termtree = "0.4.1" +tokio = { version = "1.28", features = ["macros", "rt", "rt-multi-thread", "sync", "fs", "parking_lot"] } diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs similarity index 100% rename from datafusion/core/src/physical_plan/aggregates/group_values/mod.rs rename to datafusion/physical-plan/src/aggregates/group_values/mod.rs diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/primitive.rs similarity index 99% rename from datafusion/core/src/physical_plan/aggregates/group_values/primitive.rs rename to datafusion/physical-plan/src/aggregates/group_values/primitive.rs index 7a52729d20187..e3ba284797d15 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/primitive.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::physical_plan::aggregates::group_values::GroupValues; +use crate::aggregates::group_values::GroupValues; use ahash::RandomState; use arrow::array::BooleanBufferBuilder; use arrow::buffer::NullBuffer; diff --git a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs similarity index 99% rename from datafusion/core/src/physical_plan/aggregates/group_values/row.rs rename to datafusion/physical-plan/src/aggregates/group_values/row.rs index d711a1619116e..1e8688f9ec206 100644 --- a/datafusion/core/src/physical_plan/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::physical_plan::aggregates::group_values::GroupValues; +use crate::aggregates::group_values::GroupValues; use ahash::RandomState; use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, Rows, SortField}; diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs similarity index 98% rename from datafusion/core/src/physical_plan/aggregates/mod.rs rename to datafusion/physical-plan/src/aggregates/mod.rs index bbc2b949e2ca1..2916786a8072d 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -17,11 +17,11 @@ //! Aggregates functionalities -use crate::physical_plan::aggregates::{ +use crate::aggregates::{ no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, }; -use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::physical_plan::{ +use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use crate::{ DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; @@ -53,7 +53,7 @@ mod row_hash; mod topk; mod topk_stream; -use crate::physical_plan::aggregates::topk_stream::GroupedTopKAggregateStream; +use crate::aggregates::topk_stream::GroupedTopKAggregateStream; pub use datafusion_expr::AggregateFunction; use datafusion_physical_expr::aggregate::is_order_sensitive; pub use datafusion_physical_expr::expressions::create_aggregate_expr; @@ -267,25 +267,25 @@ pub(crate) struct AggregationOrdering { #[derive(Debug)] pub struct AggregateExec { /// Aggregation mode (full, partial) - pub(crate) mode: AggregateMode, + mode: AggregateMode, /// Group by expressions - pub(crate) group_by: PhysicalGroupBy, + group_by: PhysicalGroupBy, /// Aggregate expressions - pub(crate) aggr_expr: Vec>, + aggr_expr: Vec>, /// FILTER (WHERE clause) expression for each aggregate expression - pub(crate) filter_expr: Vec>>, + filter_expr: Vec>>, /// (ORDER BY clause) expression for each aggregate expression - pub(crate) order_by_expr: Vec>, + order_by_expr: Vec>, /// Set if the output of this aggregation is truncated by a upstream sort/limit clause - pub(crate) limit: Option, + limit: Option, /// Input plan, could be a partial aggregate or the input to the aggregate - pub(crate) input: Arc, + pub input: Arc, /// Schema after the aggregate is applied schema: SchemaRef, /// Input schema before any aggregation is applied. For partial aggregate this will be the /// same as input.schema() but for the final aggregate it will be the same as the input /// to the partial aggregate - pub(crate) input_schema: SchemaRef, + pub input_schema: SchemaRef, /// The columns map used to normalize out expressions like Partitioning and PhysicalSortExpr /// The key is the column from the input schema and the values are the columns from the output schema columns_map: HashMap>, @@ -718,6 +718,11 @@ impl AggregateExec { &self.mode } + /// Set the `limit` of this AggExec + pub fn with_limit(mut self, limit: Option) -> Self { + self.limit = limit; + self + } /// Grouping expressions pub fn group_expr(&self) -> &PhysicalGroupBy { &self.group_by @@ -789,6 +794,10 @@ impl AggregateExec { None } } + + pub fn group_by(&self) -> &PhysicalGroupBy { + &self.group_by + } } impl DisplayAs for AggregateExec { @@ -1259,30 +1268,31 @@ pub(crate) fn evaluate_group_by( #[cfg(test)] mod tests { use super::*; - use crate::physical_plan::aggregates::GroupByOrderMode::{ - FullyOrdered, PartiallyOrdered, - }; - use crate::physical_plan::aggregates::{ + use crate::aggregates::GroupByOrderMode::{FullyOrdered, PartiallyOrdered}; + use crate::aggregates::{ get_finest_requirement, get_working_mode, AggregateExec, AggregateMode, PhysicalGroupBy, }; - use crate::physical_plan::coalesce_batches::CoalesceBatchesExec; - use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; - use crate::physical_plan::expressions::{col, Avg}; - use crate::physical_plan::memory::MemoryExec; - use crate::physical_plan::{ + use crate::coalesce_batches::CoalesceBatchesExec; + use crate::coalesce_partitions::CoalescePartitionsExec; + use crate::common; + use crate::expressions::{col, Avg}; + use crate::memory::MemoryExec; + use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; + use crate::test::{assert_is_pending, mem_exec}; + use crate::{ DisplayAs, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; - use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; - use crate::test::{assert_is_pending, csv_exec_sorted}; - use crate::{assert_batches_eq, assert_batches_sorted_eq, physical_plan::common}; use arrow::array::{Float64Array, UInt32Array}; use arrow::compute::{concat_batches, SortOptions}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; - use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; + use datafusion_common::{ + assert_batches_eq, assert_batches_sorted_eq, internal_err, DataFusionError, + Result, ScalarValue, + }; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_physical_expr::expressions::{ lit, ApproxDistinct, Column, Count, FirstValue, LastValue, Median, @@ -1338,7 +1348,8 @@ mod tests { sort_expr("b", &test_schema), sort_expr("c", &test_schema), ]; - let input = csv_exec_sorted(&test_schema, sort_exprs, true); + let input = mem_exec(1).with_sort_information(vec![sort_exprs]); + let input = Arc::new(input) as _; // test cases consists of vector of tuples. Where each tuple represents a single test case. // First field in the tuple is Vec where each element in the vector represents GROUP BY columns @@ -2034,7 +2045,7 @@ mod tests { schema, )?); - let fut = crate::physical_plan::collect(aggregate_exec, task_ctx); + let fut = crate::collect(aggregate_exec, task_ctx); let mut fut = fut.boxed(); assert_is_pending(&mut fut); @@ -2073,7 +2084,7 @@ mod tests { schema, )?); - let fut = crate::physical_plan::collect(aggregate_exec, task_ctx); + let fut = crate::collect(aggregate_exec, task_ctx); let mut fut = fut.boxed(); assert_is_pending(&mut fut); @@ -2191,7 +2202,7 @@ mod tests { schema, )?) as Arc; - let result = crate::physical_plan::collect(aggregate_final, task_ctx).await?; + let result = crate::collect(aggregate_final, task_ctx).await?; if is_first_acc { let expected = [ "+---+----------------+", diff --git a/datafusion/core/src/physical_plan/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs similarity index 97% rename from datafusion/core/src/physical_plan/aggregates/no_grouping.rs rename to datafusion/physical-plan/src/aggregates/no_grouping.rs index 610c24faab1c7..32c0bbc78a5de 100644 --- a/datafusion/core/src/physical_plan/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -17,12 +17,12 @@ //! Aggregate without grouping columns -use crate::physical_plan::aggregates::{ +use crate::aggregates::{ aggregate_expressions, create_accumulators, finalize_aggregation, AccumulatorItem, AggregateMode, }; -use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput}; -use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; +use crate::metrics::{BaselineMetrics, RecordOutput}; +use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::Result; @@ -33,7 +33,7 @@ use std::borrow::Cow; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::physical_plan::filter::batch_filter; +use crate::filter::batch_filter; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use futures::stream::{Stream, StreamExt}; diff --git a/datafusion/core/src/physical_plan/aggregates/order/full.rs b/datafusion/physical-plan/src/aggregates/order/full.rs similarity index 99% rename from datafusion/core/src/physical_plan/aggregates/order/full.rs rename to datafusion/physical-plan/src/aggregates/order/full.rs index 69b308da7c8ca..f46ee687faf16 100644 --- a/datafusion/core/src/physical_plan/aggregates/order/full.rs +++ b/datafusion/physical-plan/src/aggregates/order/full.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::physical_expr::EmitTo; +use datafusion_physical_expr::EmitTo; /// Tracks grouping state when the data is ordered entirely by its /// group keys diff --git a/datafusion/core/src/physical_plan/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs similarity index 100% rename from datafusion/core/src/physical_plan/aggregates/order/mod.rs rename to datafusion/physical-plan/src/aggregates/order/mod.rs diff --git a/datafusion/core/src/physical_plan/aggregates/order/partial.rs b/datafusion/physical-plan/src/aggregates/order/partial.rs similarity index 99% rename from datafusion/core/src/physical_plan/aggregates/order/partial.rs rename to datafusion/physical-plan/src/aggregates/order/partial.rs index 0feac3a5ed52f..ff8a75b9b28be 100644 --- a/datafusion/core/src/physical_plan/aggregates/order/partial.rs +++ b/datafusion/physical-plan/src/aggregates/order/partial.rs @@ -15,12 +15,12 @@ // specific language governing permissions and limitations // under the License. -use crate::physical_expr::EmitTo; use arrow::row::{OwnedRow, RowConverter, Rows, SortField}; use arrow_array::ArrayRef; use arrow_schema::Schema; use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::VecAllocExt; +use datafusion_physical_expr::EmitTo; use datafusion_physical_expr::PhysicalSortExpr; /// Tracks grouping state when the data is ordered by some subset of diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs similarity index 98% rename from datafusion/core/src/physical_plan/aggregates/row_hash.rs rename to datafusion/physical-plan/src/aggregates/row_hash.rs index eef25c1dc2147..d773533ad6a32 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -28,19 +28,19 @@ use std::vec; use futures::ready; use futures::stream::{Stream, StreamExt}; -use crate::physical_plan::aggregates::group_values::{new_group_values, GroupValues}; -use crate::physical_plan::aggregates::order::GroupOrderingFull; -use crate::physical_plan::aggregates::{ +use crate::aggregates::group_values::{new_group_values, GroupValues}; +use crate::aggregates::order::GroupOrderingFull; +use crate::aggregates::{ evaluate_group_by, evaluate_many, evaluate_optional, group_schema, AggregateMode, PhysicalGroupBy, }; -use crate::physical_plan::common::IPCWriter; -use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput}; -use crate::physical_plan::sorts::sort::{read_spill_as_stream, sort_batch}; -use crate::physical_plan::sorts::streaming_merge; -use crate::physical_plan::stream::RecordBatchStreamAdapter; -use crate::physical_plan::{aggregates, PhysicalExpr}; -use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; +use crate::common::IPCWriter; +use crate::metrics::{BaselineMetrics, RecordOutput}; +use crate::sorts::sort::{read_spill_as_stream, sort_batch}; +use crate::sorts::streaming_merge; +use crate::stream::RecordBatchStreamAdapter; +use crate::{aggregates, PhysicalExpr}; +use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use arrow_schema::SortOptions; diff --git a/datafusion/core/src/physical_plan/aggregates/topk/hash_table.rs b/datafusion/physical-plan/src/aggregates/topk/hash_table.rs similarity index 98% rename from datafusion/core/src/physical_plan/aggregates/topk/hash_table.rs rename to datafusion/physical-plan/src/aggregates/topk/hash_table.rs index 8d637ba019822..808a068b28506 100644 --- a/datafusion/core/src/physical_plan/aggregates/topk/hash_table.rs +++ b/datafusion/physical-plan/src/aggregates/topk/hash_table.rs @@ -17,8 +17,8 @@ //! A wrapper around `hashbrown::RawTable` that allows entries to be tracked by index -use crate::physical_plan::aggregates::group_values::primitive::HashValue; -use crate::physical_plan::aggregates::topk::heap::Comparable; +use crate::aggregates::group_values::primitive::HashValue; +use crate::aggregates::topk::heap::Comparable; use ahash::RandomState; use arrow::datatypes::i256; use arrow_array::builder::PrimitiveBuilder; @@ -386,7 +386,7 @@ pub fn new_hash_table(limit: usize, kt: DataType) -> Result Result Result<()> { diff --git a/datafusion/core/src/physical_plan/aggregates/topk/mod.rs b/datafusion/physical-plan/src/aggregates/topk/mod.rs similarity index 100% rename from datafusion/core/src/physical_plan/aggregates/topk/mod.rs rename to datafusion/physical-plan/src/aggregates/topk/mod.rs diff --git a/datafusion/core/src/physical_plan/aggregates/topk/priority_map.rs b/datafusion/physical-plan/src/aggregates/topk/priority_map.rs similarity index 98% rename from datafusion/core/src/physical_plan/aggregates/topk/priority_map.rs rename to datafusion/physical-plan/src/aggregates/topk/priority_map.rs index 8e1886ab76fa4..ee72e4083bf46 100644 --- a/datafusion/core/src/physical_plan/aggregates/topk/priority_map.rs +++ b/datafusion/physical-plan/src/aggregates/topk/priority_map.rs @@ -17,10 +17,8 @@ //! A `Map` / `PriorityQueue` combo that evicts the worst values after reaching `capacity` -use crate::physical_plan::aggregates::topk::hash_table::{ - new_hash_table, ArrowHashTable, -}; -use crate::physical_plan::aggregates::topk::heap::{new_heap, ArrowHeap}; +use crate::aggregates::topk::hash_table::{new_hash_table, ArrowHashTable}; +use crate::aggregates::topk::heap::{new_heap, ArrowHeap}; use arrow_array::ArrayRef; use arrow_schema::DataType; use datafusion_common::Result; @@ -115,12 +113,12 @@ impl PriorityMap { #[cfg(test)] mod tests { use super::*; - use crate::error::Result; use arrow::util::pretty::pretty_format_batches; use arrow_array::{Int64Array, RecordBatch, StringArray}; use arrow_schema::Field; use arrow_schema::Schema; use arrow_schema::{DataType, SchemaRef}; + use datafusion_common::Result; use std::sync::Arc; #[test] diff --git a/datafusion/core/src/physical_plan/aggregates/topk_stream.rs b/datafusion/physical-plan/src/aggregates/topk_stream.rs similarity index 97% rename from datafusion/core/src/physical_plan/aggregates/topk_stream.rs rename to datafusion/physical-plan/src/aggregates/topk_stream.rs index de1c02885d0c0..9f25473cb9b42 100644 --- a/datafusion/core/src/physical_plan/aggregates/topk_stream.rs +++ b/datafusion/physical-plan/src/aggregates/topk_stream.rs @@ -17,12 +17,12 @@ //! A memory-conscious aggregation implementation that limits group buckets to a fixed number -use crate::physical_plan::aggregates::topk::priority_map::PriorityMap; -use crate::physical_plan::aggregates::{ +use crate::aggregates::topk::priority_map::PriorityMap; +use crate::aggregates::{ aggregate_expressions, evaluate_group_by, evaluate_many, AggregateExec, PhysicalGroupBy, }; -use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; +use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::util::pretty::print_batches; use arrow_array::{Array, ArrayRef, RecordBatch}; use arrow_schema::SchemaRef; diff --git a/datafusion/core/src/physical_plan/analyze.rs b/datafusion/physical-plan/src/analyze.rs similarity index 99% rename from datafusion/core/src/physical_plan/analyze.rs rename to datafusion/physical-plan/src/analyze.rs index 98fce19a1dd7d..b7e95c8b1b802 100644 --- a/datafusion/core/src/physical_plan/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use std::{any::Any, time::Instant}; -use crate::physical_plan::{ +use crate::{ display::DisplayableExecutionPlan, DisplayFormatType, ExecutionPlan, Partitioning, Statistics, }; @@ -242,7 +242,7 @@ mod tests { use futures::FutureExt; use crate::{ - physical_plan::collect, + collect, test::{ assert_is_pending, exec::{assert_strong_count_converges_to_zero, BlockingExec}, diff --git a/datafusion/core/src/physical_plan/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs similarity index 95% rename from datafusion/core/src/physical_plan/coalesce_batches.rs rename to datafusion/physical-plan/src/coalesce_batches.rs index cc32d4163b19b..f46a228064fe7 100644 --- a/datafusion/core/src/physical_plan/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -23,7 +23,7 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::physical_plan::{ +use crate::{ DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, }; @@ -309,9 +309,9 @@ pub fn concat_batches( #[cfg(test)] mod tests { use super::*; - use crate::physical_plan::{memory::MemoryExec, repartition::RepartitionExec}; - use crate::test::create_vec_batches; + use crate::{memory::MemoryExec, repartition::RepartitionExec}; use arrow::datatypes::{DataType, Field, Schema}; + use arrow_array::UInt32Array; #[tokio::test(flavor = "multi_thread")] async fn test_concat_batches() -> Result<()> { @@ -365,4 +365,23 @@ mod tests { } Ok(output_partitions) } + + /// Create vector batches + fn create_vec_batches(schema: &Schema, n: usize) -> Vec { + let batch = create_batch(schema); + let mut vec = Vec::with_capacity(n); + for _ in 0..n { + vec.push(batch.clone()); + } + vec + } + + /// Create batch + fn create_batch(schema: &Schema) -> RecordBatch { + RecordBatch::try_new( + Arc::new(schema.clone()), + vec![Arc::new(UInt32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8]))], + ) + .unwrap() + } } diff --git a/datafusion/core/src/physical_plan/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs similarity index 95% rename from datafusion/core/src/physical_plan/coalesce_partitions.rs rename to datafusion/physical-plan/src/coalesce_partitions.rs index 78cb7b201f263..8eddf57ae5515 100644 --- a/datafusion/core/src/physical_plan/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -26,9 +26,7 @@ use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::stream::{ObservedStream, RecordBatchReceiverStream}; use super::{DisplayAs, SendableRecordBatchStream, Statistics}; -use crate::physical_plan::{ - DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, -}; +use crate::{DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning}; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; @@ -171,22 +169,20 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use futures::FutureExt; - use tempfile::TempDir; use super::*; - use crate::physical_plan::{collect, common}; use crate::test::exec::{ assert_strong_count_converges_to_zero, BlockingExec, PanicExec, }; use crate::test::{self, assert_is_pending}; + use crate::{collect, common}; #[tokio::test] async fn merge() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); let num_partitions = 4; - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(num_partitions, tmp_dir.path())?; + let csv = test::scan_partitioned(num_partitions); // input should have 4 partitions assert_eq!(csv.output_partitioning().partition_count(), num_partitions); @@ -201,9 +197,9 @@ mod tests { let batches = common::collect(iter).await?; assert_eq!(batches.len(), num_partitions); - // there should be a total of 100 rows + // there should be a total of 400 rows (100 per each partition) let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum(); - assert_eq!(row_count, 100); + assert_eq!(row_count, 400); Ok(()) } diff --git a/datafusion/core/src/physical_plan/common.rs b/datafusion/physical-plan/src/common.rs similarity index 98% rename from datafusion/core/src/physical_plan/common.rs rename to datafusion/physical-plan/src/common.rs index 787f3eed2673e..c6cfbbfbbac74 100644 --- a/datafusion/core/src/physical_plan/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -18,8 +18,8 @@ //! Defines common code used in execution plans use super::SendableRecordBatchStream; -use crate::physical_plan::stream::RecordBatchReceiverStream; -use crate::physical_plan::{ColumnStatistics, ExecutionPlan, Statistics}; +use crate::stream::RecordBatchReceiverStream; +use crate::{ColumnStatistics, ExecutionPlan, Statistics}; use arrow::datatypes::Schema; use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; use arrow::record_batch::RecordBatch; @@ -375,9 +375,9 @@ mod tests { use std::ops::Not; use super::*; - use crate::physical_plan::memory::MemoryExec; - use crate::physical_plan::sorts::sort::SortExec; - use crate::physical_plan::union::UnionExec; + use crate::memory::MemoryExec; + use crate::sorts::sort::SortExec; + use crate::union::UnionExec; use arrow::compute::SortOptions; use arrow::{ array::{Float32Array, Float64Array}, diff --git a/datafusion/core/src/physical_plan/display.rs b/datafusion/physical-plan/src/display.rs similarity index 98% rename from datafusion/core/src/physical_plan/display.rs rename to datafusion/physical-plan/src/display.rs index 3b345bdf9e3ad..e4a4e113eb07e 100644 --- a/datafusion/core/src/physical_plan/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -16,8 +16,7 @@ // under the License. //! Implementation of physical plan display. See -//! [`crate::physical_plan::displayable`] for examples of how to -//! format +//! [`crate::displayable`] for examples of how to format use std::fmt; @@ -26,7 +25,7 @@ use datafusion_common::display::StringifiedPlan; use datafusion_physical_expr::PhysicalSortExpr; use super::{accept, ExecutionPlan, ExecutionPlanVisitor}; -use datafusion_common::display::GraphvizBuilder; +use datafusion_common::display::{GraphvizBuilder, PlanType}; /// Options for controlling how each [`ExecutionPlan`] should format itself #[derive(Debug, Clone, Copy)] @@ -204,11 +203,7 @@ impl<'a> DisplayableExecutionPlan<'a> { } /// format as a `StringifiedPlan` - pub fn to_stringified( - &self, - verbose: bool, - plan_type: crate::logical_expr::PlanType, - ) -> StringifiedPlan { + pub fn to_stringified(&self, verbose: bool, plan_type: PlanType) -> StringifiedPlan { StringifiedPlan::new(plan_type, self.indent(verbose).to_string()) } } diff --git a/datafusion/core/src/physical_plan/empty.rs b/datafusion/physical-plan/src/empty.rs similarity index 94% rename from datafusion/core/src/physical_plan/empty.rs rename to datafusion/physical-plan/src/empty.rs index b38486991e82f..675dac9ad2656 100644 --- a/datafusion/core/src/physical_plan/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -20,9 +20,7 @@ use std::any::Any; use std::sync::Arc; -use crate::physical_plan::{ - memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning, -}; +use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::array::{ArrayRef, NullArray}; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; @@ -174,13 +172,13 @@ impl ExecutionPlan for EmptyExec { #[cfg(test)] mod tests { use super::*; - use crate::physical_plan::with_new_children_if_necessary; - use crate::{physical_plan::common, test_util}; + use crate::with_new_children_if_necessary; + use crate::{common, test}; #[tokio::test] async fn empty() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); - let schema = test_util::aggr_test_schema(); + let schema = test::aggr_test_schema(); let empty = EmptyExec::new(false, schema.clone()); assert_eq!(empty.schema(), schema); @@ -195,7 +193,7 @@ mod tests { #[test] fn with_new_children() -> Result<()> { - let schema = test_util::aggr_test_schema(); + let schema = test::aggr_test_schema(); let empty = Arc::new(EmptyExec::new(false, schema.clone())); let empty_with_row = Arc::new(EmptyExec::new(true, schema)); @@ -217,7 +215,7 @@ mod tests { #[tokio::test] async fn invalid_execute() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); - let schema = test_util::aggr_test_schema(); + let schema = test::aggr_test_schema(); let empty = EmptyExec::new(false, schema); // ask for the wrong partition @@ -229,7 +227,7 @@ mod tests { #[tokio::test] async fn produce_one_row() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); - let schema = test_util::aggr_test_schema(); + let schema = test::aggr_test_schema(); let empty = EmptyExec::new(true, schema); let iter = empty.execute(0, task_ctx)?; @@ -244,7 +242,7 @@ mod tests { #[tokio::test] async fn produce_one_row_multiple_partition() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); - let schema = test_util::aggr_test_schema(); + let schema = test::aggr_test_schema(); let partitions = 3; let empty = EmptyExec::new(true, schema).with_partitions(partitions); diff --git a/datafusion/core/src/physical_plan/explain.rs b/datafusion/physical-plan/src/explain.rs similarity index 97% rename from datafusion/core/src/physical_plan/explain.rs rename to datafusion/physical-plan/src/explain.rs index 7ee6f268d8151..8d6bf4105f6a1 100644 --- a/datafusion/core/src/physical_plan/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -24,13 +24,13 @@ use datafusion_common::display::StringifiedPlan; use datafusion_common::{internal_err, DataFusionError, Result}; -use crate::physical_plan::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; use arrow::{array::StringBuilder, datatypes::SchemaRef, record_batch::RecordBatch}; use log::trace; use super::DisplayAs; use super::{expressions::PhysicalSortExpr, SendableRecordBatchStream}; -use crate::physical_plan::stream::RecordBatchStreamAdapter; +use crate::stream::RecordBatchStreamAdapter; use datafusion_execution::TaskContext; /// Explain execution plan operator. This operator contains the string diff --git a/datafusion/core/src/physical_plan/filter.rs b/datafusion/physical-plan/src/filter.rs similarity index 93% rename from datafusion/core/src/physical_plan/filter.rs rename to datafusion/physical-plan/src/filter.rs index 0b878814e305a..15208fd0829e2 100644 --- a/datafusion/core/src/physical_plan/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -28,7 +28,7 @@ use super::{ ColumnStatistics, DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use crate::physical_plan::{ +use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, Column, DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, }; @@ -378,77 +378,20 @@ pub type EqualAndNonEqual<'a> = mod tests { use super::*; - use crate::physical_plan::expressions::*; - use crate::physical_plan::ExecutionPlan; - use crate::physical_plan::{collect, with_new_children_if_necessary}; + use crate::expressions::*; use crate::test; use crate::test::exec::StatisticsExec; - use crate::test_util; + use crate::ExecutionPlan; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::utils::DataPtr; use datafusion_common::ColumnStatistics; use datafusion_common::ScalarValue; use datafusion_expr::Operator; use std::iter::Iterator; use std::sync::Arc; - use tempfile::TempDir; - - #[tokio::test] - async fn simple_predicate() -> Result<()> { - let task_ctx = Arc::new(TaskContext::default()); - let schema = test_util::aggr_test_schema(); - - let partitions = 4; - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(partitions, tmp_dir.path())?; - - let predicate: Arc = binary( - binary(col("c2", &schema)?, Operator::Gt, lit(1u32), &schema)?, - Operator::And, - binary(col("c2", &schema)?, Operator::Lt, lit(4u32), &schema)?, - &schema, - )?; - - let filter: Arc = - Arc::new(FilterExec::try_new(predicate, csv)?); - - let results = collect(filter, task_ctx).await?; - - results - .iter() - .for_each(|batch| assert_eq!(13, batch.num_columns())); - let row_count: usize = results.iter().map(|batch| batch.num_rows()).sum(); - assert_eq!(41, row_count); - - Ok(()) - } - - #[tokio::test] - async fn with_new_children() -> Result<()> { - let schema = test_util::aggr_test_schema(); - let partitions = 4; - let tmp_dir = TempDir::new()?; - let input = test::scan_partitioned_csv(partitions, tmp_dir.path())?; - - let predicate: Arc = - binary(col("c2", &schema)?, Operator::Gt, lit(1u32), &schema)?; - - let filter: Arc = - Arc::new(FilterExec::try_new(predicate, input.clone())?); - - let new_filter = filter.clone().with_new_children(vec![input.clone()])?; - assert!(!Arc::data_ptr_eq(&filter, &new_filter)); - - let new_filter2 = - with_new_children_if_necessary(filter.clone(), vec![input])?.into(); - assert!(Arc::data_ptr_eq(&filter, &new_filter2)); - - Ok(()) - } #[tokio::test] async fn collect_columns_predicates() -> Result<()> { - let schema = test_util::aggr_test_schema(); + let schema = test::aggr_test_schema(); let predicate: Arc = binary( binary( binary(col("c2", &schema)?, Operator::GtEq, lit(1u32), &schema)?, diff --git a/datafusion/core/src/physical_plan/insert.rs b/datafusion/physical-plan/src/insert.rs similarity index 99% rename from datafusion/core/src/physical_plan/insert.rs rename to datafusion/physical-plan/src/insert.rs index 8c03fb543f5b1..e60afcbcb0412 100644 --- a/datafusion/core/src/physical_plan/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -35,7 +35,7 @@ use std::any::Any; use std::fmt::Debug; use std::sync::Arc; -use crate::physical_plan::stream::RecordBatchStreamAdapter; +use crate::stream::RecordBatchStreamAdapter; use datafusion_common::{exec_err, internal_err, DataFusionError}; use datafusion_execution::TaskContext; diff --git a/datafusion/core/src/physical_plan/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs similarity index 97% rename from datafusion/core/src/physical_plan/joins/cross_join.rs rename to datafusion/physical-plan/src/joins/cross_join.rs index 256942754350e..4ba29524b3e2f 100644 --- a/datafusion/core/src/physical_plan/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -25,9 +25,9 @@ use std::{any::Any, sync::Arc, task::Poll}; use arrow::datatypes::{Fields, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::physical_plan::DisplayAs; -use crate::physical_plan::{ +use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use crate::DisplayAs; +use crate::{ coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec, ColumnStatistics, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, PhysicalSortExpr, RecordBatchStream, @@ -52,9 +52,9 @@ type JoinLeftData = (RecordBatch, MemoryReservation); #[derive(Debug)] pub struct CrossJoinExec { /// left (build) side which gets loaded in memory - pub(crate) left: Arc, + pub left: Arc, /// right (probe) side which are combined with left side - pub(crate) right: Arc, + pub right: Arc, /// The schema once the join is applied schema: SchemaRef, /// Build-side data @@ -457,10 +457,9 @@ impl CrossJoinStream { #[cfg(test)] mod tests { use super::*; - use crate::assert_batches_sorted_eq; - use crate::common::assert_contains; - use crate::physical_plan::common; - use crate::test::{build_table_scan_i32, columns}; + use crate::common; + use crate::test::build_table_scan_i32; + use datafusion_common::{assert_batches_sorted_eq, assert_contains}; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; async fn join_collect( @@ -678,4 +677,9 @@ mod tests { Ok(()) } + + /// Returns the column names on the schema + fn columns(schema: &Schema) -> Vec { + schema.fields().iter().map(|f| f.name().clone()).collect() + } } diff --git a/datafusion/core/src/physical_plan/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs similarity index 98% rename from datafusion/core/src/physical_plan/joins/hash_join.rs rename to datafusion/physical-plan/src/joins/hash_join.rs index e0e522b2150af..8e204634f3d94 100644 --- a/datafusion/core/src/physical_plan/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -24,13 +24,13 @@ use std::sync::Arc; use std::task::Poll; use std::{any::Any, usize, vec}; -use crate::physical_plan::joins::utils::{ +use crate::joins::utils::{ adjust_indices_by_join_type, apply_join_filter_to_indices, build_batch_from_indices, calculate_join_output_ordering, combine_join_ordering_equivalence_properties, get_final_indices_from_bit_map, need_produce_result_in_final, JoinSide, }; -use crate::physical_plan::DisplayAs; -use crate::physical_plan::{ +use crate::DisplayAs; +use crate::{ coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec, expressions::Column, @@ -85,15 +85,15 @@ type JoinLeftData = (JoinHashMap, RecordBatch, MemoryReservation); #[derive(Debug)] pub struct HashJoinExec { /// left (build) side which gets hashed - pub(crate) left: Arc, + pub left: Arc, /// right (probe) side which are filtered by the hash table - pub(crate) right: Arc, + pub right: Arc, /// Set of common columns used to join on - pub(crate) on: Vec<(Column, Column)>, + pub on: Vec<(Column, Column)>, /// Filters which are applied while finding matching rows - pub(crate) filter: Option, + pub filter: Option, /// How the join is performed - pub(crate) join_type: JoinType, + pub join_type: JoinType, /// The schema once the join is applied schema: SchemaRef, /// Build-side data @@ -103,13 +103,13 @@ pub struct HashJoinExec { /// Output order output_order: Option>, /// Partitioning mode to use - pub(crate) mode: PartitionMode, + pub mode: PartitionMode, /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Information of index and left / right placement of columns column_indices: Vec, /// If null_equals_null is true, null == null else null != null - pub(crate) null_equals_null: bool, + pub null_equals_null: bool, } impl HashJoinExec { @@ -1062,28 +1062,24 @@ mod tests { use arrow::array::{ArrayRef, Date32Array, Int32Array, UInt32Builder, UInt64Builder}; use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::ScalarValue; + use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::Literal; use hashbrown::raw::RawTable; - use crate::execution::context::SessionConfig; - use crate::physical_expr::expressions::BinaryExpr; use crate::{ - assert_batches_sorted_eq, - common::assert_contains, - physical_plan::{ - common, - expressions::Column, - hash_utils::create_hashes, - joins::{hash_join::build_equal_condition_join_indices, utils::JoinSide}, - memory::MemoryExec, - repartition::RepartitionExec, - }, + common, + expressions::Column, + hash_utils::create_hashes, + joins::{hash_join::build_equal_condition_join_indices, utils::JoinSide}, + memory::MemoryExec, + repartition::RepartitionExec, + test::build_table_i32, test::exec::MockExec, - test::{build_table_i32, columns}, }; + use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_physical_expr::expressions::BinaryExpr; use super::*; @@ -2765,7 +2761,7 @@ mod tests { let stream = join.execute(0, task_ctx).unwrap(); // Expect that an error is returned - let result_string = crate::physical_plan::common::collect(stream) + let result_string = crate::common::collect(stream) .await .unwrap_err() .to_string(); @@ -2906,4 +2902,9 @@ mod tests { Ok(()) } + + /// Returns the column names on the schema + fn columns(schema: &Schema) -> Vec { + schema.fields().iter().map(|f| f.name().clone()).collect() + } } diff --git a/datafusion/core/src/physical_plan/joins/hash_join_utils.rs b/datafusion/physical-plan/src/joins/hash_join_utils.rs similarity index 99% rename from datafusion/core/src/physical_plan/joins/hash_join_utils.rs rename to datafusion/physical-plan/src/joins/hash_join_utils.rs index ac0b183818147..bb7976345800d 100644 --- a/datafusion/core/src/physical_plan/joins/hash_join_utils.rs +++ b/datafusion/physical-plan/src/joins/hash_join_utils.rs @@ -24,8 +24,8 @@ use std::ops::IndexMut; use std::sync::Arc; use std::{fmt, usize}; -use crate::physical_plan::joins::utils::{JoinFilter, JoinSide}; -use crate::physical_plan::ExecutionPlan; +use crate::joins::utils::{JoinFilter, JoinSide}; +use crate::ExecutionPlan; use arrow::compute::concat_batches; use arrow::datatypes::{ArrowNativeType, SchemaRef}; @@ -830,7 +830,7 @@ pub fn record_visited_indices( #[cfg(test)] pub mod tests { use super::*; - use crate::physical_plan::{ + use crate::{ expressions::Column, expressions::PhysicalSortExpr, joins::utils::{ColumnIndex, JoinFilter, JoinSide}, diff --git a/datafusion/core/src/physical_plan/joins/mod.rs b/datafusion/physical-plan/src/joins/mod.rs similarity index 100% rename from datafusion/core/src/physical_plan/joins/mod.rs rename to datafusion/physical-plan/src/joins/mod.rs diff --git a/datafusion/core/src/physical_plan/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs similarity index 98% rename from datafusion/core/src/physical_plan/joins/nested_loop_join.rs rename to datafusion/physical-plan/src/joins/nested_loop_join.rs index 618dd66e69a09..c49c16dba3130 100644 --- a/datafusion/core/src/physical_plan/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -19,7 +19,7 @@ //! The nested loop join can execute in parallel by partitions and it is //! determined by the [`JoinType`]. -use crate::physical_plan::joins::utils::{ +use crate::joins::utils::{ append_right_indices, apply_join_filter_to_indices, build_batch_from_indices, build_join_schema, check_join_is_valid, combine_join_equivalence_properties, estimate_join_statistics, get_anti_indices, get_anti_u64_indices, @@ -27,8 +27,8 @@ use crate::physical_plan::joins::utils::{ partitioned_join_output_partitioning, BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinSide, OnceAsync, OnceFut, }; -use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::physical_plan::{ +use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, }; @@ -48,7 +48,7 @@ use std::fmt::Formatter; use std::sync::Arc; use std::task::Poll; -use crate::physical_plan::coalesce_batches::concat_batches; +use crate::coalesce_batches::concat_batches; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; @@ -740,21 +740,17 @@ impl RecordBatchStream for NestedLoopJoinStream { #[cfg(test)] mod tests { use super::*; - use crate::physical_expr::expressions::BinaryExpr; use crate::{ - assert_batches_sorted_eq, - common::assert_contains, - execution::runtime_env::{RuntimeConfig, RuntimeEnv}, - physical_plan::{ - common, expressions::Column, memory::MemoryExec, repartition::RepartitionExec, - }, - test::{build_table_i32, columns}, + common, expressions::Column, memory::MemoryExec, repartition::RepartitionExec, + test::build_table_i32, }; use arrow::datatypes::{DataType, Field}; + use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::BinaryExpr; - use crate::physical_plan::joins::utils::JoinSide; - use datafusion_common::ScalarValue; + use crate::joins::utils::JoinSide; + use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::PhysicalExpr; use std::sync::Arc; @@ -1169,4 +1165,9 @@ mod tests { Ok(()) } + + /// Returns the column names on the schema + fn columns(schema: &Schema) -> Vec { + schema.fields().iter().map(|f| f.name().clone()).collect() + } } diff --git a/datafusion/core/src/physical_plan/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs similarity index 98% rename from datafusion/core/src/physical_plan/joins/sort_merge_join.rs rename to datafusion/physical-plan/src/joins/sort_merge_join.rs index 3de98f5452c57..4de723ab73ea5 100644 --- a/datafusion/core/src/physical_plan/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -30,15 +30,15 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::physical_plan::expressions::Column; -use crate::physical_plan::expressions::PhysicalSortExpr; -use crate::physical_plan::joins::utils::{ +use crate::expressions::Column; +use crate::expressions::PhysicalSortExpr; +use crate::joins::utils::{ build_join_schema, calculate_join_output_ordering, check_join_is_valid, combine_join_equivalence_properties, combine_join_ordering_equivalence_properties, estimate_join_statistics, partitioned_join_output_partitioning, JoinOn, JoinSide, }; -use crate::physical_plan::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; -use crate::physical_plan::{ +use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; +use crate::{ metrics, DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, @@ -63,13 +63,13 @@ use futures::{Stream, StreamExt}; #[derive(Debug)] pub struct SortMergeJoinExec { /// Left sorted joining execution plan - pub(crate) left: Arc, + pub left: Arc, /// Right sorting joining execution plan - pub(crate) right: Arc, + pub right: Arc, /// Set of common columns used to join on - pub(crate) on: JoinOn, + pub on: JoinOn, /// How the join is performed - pub(crate) join_type: JoinType, + pub join_type: JoinType, /// The schema once the join is applied schema: SchemaRef, /// Execution metrics @@ -81,9 +81,9 @@ pub struct SortMergeJoinExec { /// The output ordering output_ordering: Option>, /// Sort options of join columns used in sorting left and right execution plans - pub(crate) sort_options: Vec, + pub sort_options: Vec, /// If null_equals_null is true, null == null else null != null - pub(crate) null_equals_null: bool, + pub null_equals_null: bool, } impl SortMergeJoinExec { @@ -194,6 +194,18 @@ impl SortMergeJoinExec { pub fn on(&self) -> &[(Column, Column)] { &self.on } + + pub fn right(&self) -> &dyn ExecutionPlan { + self.right.as_ref() + } + + pub fn join_type(&self) -> JoinType { + self.join_type + } + + pub fn left(&self) -> &dyn ExecutionPlan { + self.left.as_ref() + } } impl DisplayAs for SortMergeJoinExec { @@ -1392,16 +1404,16 @@ mod tests { use datafusion_execution::config::SessionConfig; use datafusion_execution::TaskContext; - use crate::common::assert_contains; - use crate::physical_plan::expressions::Column; - use crate::physical_plan::joins::utils::JoinOn; - use crate::physical_plan::joins::SortMergeJoinExec; - use crate::physical_plan::memory::MemoryExec; - use crate::physical_plan::{common, ExecutionPlan}; - use crate::test::{build_table_i32, columns}; - use crate::{assert_batches_eq, assert_batches_sorted_eq}; - use datafusion_common::JoinType; + use crate::expressions::Column; + use crate::joins::utils::JoinOn; + use crate::joins::SortMergeJoinExec; + use crate::memory::MemoryExec; + use crate::test::build_table_i32; + use crate::{common, ExecutionPlan}; use datafusion_common::Result; + use datafusion_common::{ + assert_batches_eq, assert_batches_sorted_eq, assert_contains, JoinType, + }; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; fn build_table( @@ -2422,4 +2434,8 @@ mod tests { Ok(()) } + /// Returns the column names on the schema + fn columns(schema: &Schema) -> Vec { + schema.fields().iter().map(|f| f.name().clone()).collect() + } } diff --git a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs similarity index 99% rename from datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs rename to datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 94eefa50ead73..a2fd127112e3d 100644 --- a/datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -32,19 +32,17 @@ use std::task::Poll; use std::vec; use std::{any::Any, usize}; -use crate::physical_plan::common::SharedMemoryReservation; -use crate::physical_plan::joins::hash_join::{ - build_equal_condition_join_indices, update_hash, -}; -use crate::physical_plan::joins::hash_join_utils::{ +use crate::common::SharedMemoryReservation; +use crate::joins::hash_join::{build_equal_condition_join_indices, update_hash}; +use crate::joins::hash_join_utils::{ build_filter_expression_graph, calculate_filter_expr_intervals, combine_two_batches, convert_sort_expr_with_filter_schema, get_pruning_anti_indices, get_pruning_semi_indices, record_visited_indices, IntervalCalculatorInnerState, PruningJoinHashMap, }; -use crate::physical_plan::joins::StreamJoinPartitionMode; -use crate::physical_plan::DisplayAs; -use crate::physical_plan::{ +use crate::joins::StreamJoinPartitionMode; +use crate::DisplayAs; +use crate::{ expressions::Column, expressions::PhysicalSortExpr, joins::{ @@ -1219,9 +1217,9 @@ mod tests { use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{binary, col, Column}; - use crate::physical_plan::joins::hash_join_utils::tests::complicated_filter; + use crate::joins::hash_join_utils::tests::complicated_filter; - use crate::physical_plan::joins::test_utils::{ + use crate::joins::test_utils::{ build_sides_record_batches, compare_batches, create_memory_table, join_expr_tests_fixture_f64, join_expr_tests_fixture_i32, join_expr_tests_fixture_temporal, partitioned_hash_join_with_filter, diff --git a/datafusion/core/src/physical_plan/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs similarity index 98% rename from datafusion/core/src/physical_plan/joins/test_utils.rs rename to datafusion/physical-plan/src/joins/test_utils.rs index dce3cee3a1514..bb4a86199112e 100644 --- a/datafusion/core/src/physical_plan/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -17,13 +17,13 @@ //! This file has test utils for hash joins -use crate::physical_plan::joins::utils::{JoinFilter, JoinOn}; -use crate::physical_plan::joins::{ +use crate::joins::utils::{JoinFilter, JoinOn}; +use crate::joins::{ HashJoinExec, PartitionMode, StreamJoinPartitionMode, SymmetricHashJoinExec, }; -use crate::physical_plan::memory::MemoryExec; -use crate::physical_plan::repartition::RepartitionExec; -use crate::physical_plan::{common, ExecutionPlan, Partitioning}; +use crate::memory::MemoryExec; +use crate::repartition::RepartitionExec; +use crate::{common, ExecutionPlan, Partitioning}; use arrow::util::pretty::pretty_format_batches; use arrow_array::{ ArrayRef, Float64Array, Int32Array, IntervalDayTimeArray, RecordBatch, diff --git a/datafusion/core/src/physical_plan/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs similarity index 99% rename from datafusion/core/src/physical_plan/joins/utils.rs rename to datafusion/physical-plan/src/joins/utils.rs index bd3de1acbf0fd..e33de001df304 100644 --- a/datafusion/core/src/physical_plan/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -25,9 +25,9 @@ use std::sync::Arc; use std::task::{Context, Poll}; use std::usize; -use crate::physical_plan::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder}; -use crate::physical_plan::SchemaRef; -use crate::physical_plan::{ +use crate::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder}; +use crate::SchemaRef; +use crate::{ ColumnStatistics, EquivalenceProperties, ExecutionPlan, Partitioning, Statistics, }; diff --git a/datafusion/core/src/physical_plan/mod.rs b/datafusion/physical-plan/src/lib.rs similarity index 88% rename from datafusion/core/src/physical_plan/mod.rs rename to datafusion/physical-plan/src/lib.rs index 89e3852a3e972..9c116e73ead7e 100644 --- a/datafusion/core/src/physical_plan/mod.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -23,9 +23,9 @@ use self::metrics::MetricsSet; use self::{ coalesce_partitions::CoalescePartitionsExec, display::DisplayableExecutionPlan, }; -use crate::physical_plan::expressions::PhysicalSortExpr; use datafusion_common::Result; pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; +use datafusion_physical_expr::PhysicalSortExpr; pub use visitor::{accept, visit_execution_plan, ExecutionPlanVisitor}; use arrow::datatypes::SchemaRef; @@ -242,45 +242,6 @@ pub fn with_new_children_if_necessary( /// Return a [wrapper](DisplayableExecutionPlan) around an /// [`ExecutionPlan`] which can be displayed in various easier to /// understand ways. -/// -/// ``` -/// use datafusion::prelude::*; -/// use datafusion::physical_plan::displayable; -/// use object_store::path::Path; -/// -/// #[tokio::main] -/// async fn main() { -/// // Hard code target_partitions as it appears in the RepartitionExec output -/// let config = SessionConfig::new() -/// .with_target_partitions(3); -/// let mut ctx = SessionContext::with_config(config); -/// -/// // register the a table -/// ctx.register_csv("example", "tests/data/example.csv", CsvReadOptions::new()).await.unwrap(); -/// -/// // create a plan to run a SQL query -/// let dataframe = ctx.sql("SELECT a FROM example WHERE a < 5").await.unwrap(); -/// let physical_plan = dataframe.create_physical_plan().await.unwrap(); -/// -/// // Format using display string in verbose mode -/// let displayable_plan = displayable(physical_plan.as_ref()); -/// let plan_string = format!("{}", displayable_plan.indent(true)); -/// -/// let working_directory = std::env::current_dir().unwrap(); -/// let normalized = Path::from_filesystem_path(working_directory).unwrap(); -/// let plan_string = plan_string.replace(normalized.as_ref(), "WORKING_DIR"); -/// -/// assert_eq!("CoalesceBatchesExec: target_batch_size=8192\ -/// \n FilterExec: a@0 < 5\ -/// \n RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1\ -/// \n CsvExec: file_groups={1 group: [[WORKING_DIR/tests/data/example.csv]]}, projection=[a], has_header=true", -/// plan_string.trim()); -/// -/// let one_line = format!("{}", displayable_plan.one_line()); -/// assert_eq!("CoalesceBatchesExec: target_batch_size=8192", one_line.trim()); -/// } -/// ``` -/// pub fn displayable(plan: &dyn ExecutionPlan) -> DisplayableExecutionPlan<'_> { DisplayableExecutionPlan::new(plan) } @@ -397,10 +358,13 @@ pub mod unnest; pub mod values; pub mod windows; -use crate::physical_plan::repartition::RepartitionExec; -use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use crate::repartition::RepartitionExec; +use crate::sorts::sort_preserving_merge::SortPreservingMergeExec; pub use datafusion_common::utils::project_schema; use datafusion_execution::TaskContext; pub use datafusion_physical_expr::{ expressions, functions, hash_utils, ordering_equivalence_properties_helper, udf, }; + +#[cfg(test)] +pub mod test; diff --git a/datafusion/core/src/physical_plan/limit.rs b/datafusion/physical-plan/src/limit.rs similarity index 93% rename from datafusion/core/src/physical_plan/limit.rs rename to datafusion/physical-plan/src/limit.rs index 87a07f8d46fec..c6d51b7d9c5d7 100644 --- a/datafusion/core/src/physical_plan/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -22,7 +22,7 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::physical_plan::{ +use crate::{ DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, }; @@ -521,12 +521,12 @@ impl RecordBatchStream for LimitStream { #[cfg(test)] mod tests { + use arrow_schema::Schema; use common::collect; - use tempfile::TempDir; use super::*; - use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; - use crate::physical_plan::common; + use crate::coalesce_partitions::CoalescePartitionsExec; + use crate::common; use crate::test; #[tokio::test] @@ -534,8 +534,7 @@ mod tests { let task_ctx = Arc::new(TaskContext::default()); let num_partitions = 4; - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(num_partitions, tmp_dir.path())?; + let csv = test::scan_partitioned(num_partitions); // input should have 4 partitions assert_eq!(csv.output_partitioning().partition_count(), num_partitions); @@ -619,9 +618,9 @@ mod tests { #[tokio::test] async fn limit_no_column() -> Result<()> { let batches = vec![ - test::make_batch_no_column(6), - test::make_batch_no_column(6), - test::make_batch_no_column(6), + make_batch_no_column(6), + make_batch_no_column(6), + make_batch_no_column(6), ]; let input = test::exec::TestStream::new(batches); @@ -650,9 +649,9 @@ mod tests { async fn skip_and_fetch(skip: usize, fetch: Option) -> Result { let task_ctx = Arc::new(TaskContext::default()); + // 4 partitions @ 100 rows apiece let num_partitions = 4; - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(num_partitions, tmp_dir.path())?; + let csv = test::scan_partitioned(num_partitions); assert_eq!(csv.output_partitioning().partition_count(), num_partitions); @@ -668,7 +667,7 @@ mod tests { #[tokio::test] async fn skip_none_fetch_none() -> Result<()> { let row_count = skip_and_fetch(0, None).await?; - assert_eq!(row_count, 100); + assert_eq!(row_count, 400); Ok(()) } @@ -681,9 +680,9 @@ mod tests { #[tokio::test] async fn skip_3_fetch_none() -> Result<()> { - // there are total of 100 rows, we skipped 3 rows (offset = 3) + // there are total of 400 rows, we skipped 3 rows (offset = 3) let row_count = skip_and_fetch(3, None).await?; - assert_eq!(row_count, 97); + assert_eq!(row_count, 397); Ok(()) } @@ -696,23 +695,24 @@ mod tests { } #[tokio::test] - async fn skip_100_fetch_none() -> Result<()> { - let row_count = skip_and_fetch(100, None).await?; + async fn skip_400_fetch_none() -> Result<()> { + let row_count = skip_and_fetch(400, None).await?; assert_eq!(row_count, 0); Ok(()) } #[tokio::test] - async fn skip_100_fetch_1() -> Result<()> { - let row_count = skip_and_fetch(100, Some(1)).await?; + async fn skip_400_fetch_1() -> Result<()> { + // there are a total of 400 rows + let row_count = skip_and_fetch(400, Some(1)).await?; assert_eq!(row_count, 0); Ok(()) } #[tokio::test] - async fn skip_101_fetch_none() -> Result<()> { - // there are total of 100 rows, we skipped 101 rows (offset = 3) - let row_count = skip_and_fetch(101, None).await?; + async fn skip_401_fetch_none() -> Result<()> { + // there are total of 400 rows, we skipped 401 rows (offset = 3) + let row_count = skip_and_fetch(401, None).await?; assert_eq!(row_count, 0); Ok(()) } @@ -731,7 +731,7 @@ mod tests { #[tokio::test] async fn test_row_number_statistics_for_local_limit() -> Result<()> { let row_count = row_number_statistics_for_local_limit(4, 10).await?; - assert_eq!(row_count, Some(40)); + assert_eq!(row_count, Some(10)); Ok(()) } @@ -741,8 +741,7 @@ mod tests { fetch: Option, ) -> Result> { let num_partitions = 4; - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(num_partitions, tmp_dir.path())?; + let csv = test::scan_partitioned(num_partitions); assert_eq!(csv.output_partitioning().partition_count(), num_partitions); @@ -756,8 +755,7 @@ mod tests { num_partitions: usize, fetch: usize, ) -> Result> { - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(num_partitions, tmp_dir.path())?; + let csv = test::scan_partitioned(num_partitions); assert_eq!(csv.output_partitioning().partition_count(), num_partitions); @@ -765,4 +763,12 @@ mod tests { Ok(offset.statistics().num_rows) } + + /// Return a RecordBatch with a single array with row_count sz + fn make_batch_no_column(sz: usize) -> RecordBatch { + let schema = Arc::new(Schema::empty()); + + let options = RecordBatchOptions::new().with_row_count(Option::from(sz)); + RecordBatch::try_new_with_options(schema, vec![], &options).unwrap() + } } diff --git a/datafusion/core/src/physical_plan/memory.rs b/datafusion/physical-plan/src/memory.rs similarity index 98% rename from datafusion/core/src/physical_plan/memory.rs rename to datafusion/physical-plan/src/memory.rs index 877410c97ca54..d36d93d29edd0 100644 --- a/datafusion/core/src/physical_plan/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -30,7 +30,7 @@ use std::any::Any; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::physical_plan::ordering_equivalence_properties_helper; +use crate::ordering_equivalence_properties_helper; use datafusion_common::DataFusionError; use datafusion_execution::TaskContext; use datafusion_physical_expr::{LexOrdering, OrderingEquivalenceProperties}; @@ -260,8 +260,8 @@ impl RecordBatchStream for MemoryStream { #[cfg(test)] mod tests { - use crate::physical_plan::memory::MemoryExec; - use crate::physical_plan::ExecutionPlan; + use crate::memory::MemoryExec; + use crate::ExecutionPlan; use arrow_schema::{DataType, Field, Schema, SortOptions}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; diff --git a/datafusion/core/src/physical_plan/metrics/baseline.rs b/datafusion/physical-plan/src/metrics/baseline.rs similarity index 98% rename from datafusion/core/src/physical_plan/metrics/baseline.rs rename to datafusion/physical-plan/src/metrics/baseline.rs index 7d72a6a9fae17..dc345cd8cdcd6 100644 --- a/datafusion/core/src/physical_plan/metrics/baseline.rs +++ b/datafusion/physical-plan/src/metrics/baseline.rs @@ -29,7 +29,7 @@ use datafusion_common::Result; /// /// Example: /// ``` -/// use datafusion::physical_plan::metrics::{BaselineMetrics, ExecutionPlanMetricsSet}; +/// use datafusion_physical_plan::metrics::{BaselineMetrics, ExecutionPlanMetricsSet}; /// let metrics = ExecutionPlanMetricsSet::new(); /// /// let partition = 2; diff --git a/datafusion/core/src/physical_plan/metrics/builder.rs b/datafusion/physical-plan/src/metrics/builder.rs similarity index 99% rename from datafusion/core/src/physical_plan/metrics/builder.rs rename to datafusion/physical-plan/src/metrics/builder.rs index 30e9764c64460..beecc13e0029b 100644 --- a/datafusion/core/src/physical_plan/metrics/builder.rs +++ b/datafusion/physical-plan/src/metrics/builder.rs @@ -29,7 +29,7 @@ use super::{ /// case of constant strings /// /// ```rust -/// use datafusion::physical_plan::metrics::*; +/// use datafusion_physical_plan::metrics::*; /// /// let metrics = ExecutionPlanMetricsSet::new(); /// let partition = 1; diff --git a/datafusion/core/src/physical_plan/metrics/mod.rs b/datafusion/physical-plan/src/metrics/mod.rs similarity index 99% rename from datafusion/core/src/physical_plan/metrics/mod.rs rename to datafusion/physical-plan/src/metrics/mod.rs index 652c0af5c2e44..b2e0086f69e9a 100644 --- a/datafusion/core/src/physical_plan/metrics/mod.rs +++ b/datafusion/physical-plan/src/metrics/mod.rs @@ -43,7 +43,7 @@ pub use value::{Count, Gauge, MetricValue, ScopedTimerGuard, Time, Timestamp}; /// [`ExecutionPlanMetricsSet`]. /// /// ``` -/// use datafusion::physical_plan::metrics::*; +/// use datafusion_physical_plan::metrics::*; /// /// let metrics = ExecutionPlanMetricsSet::new(); /// assert!(metrics.clone_inner().output_rows().is_none()); diff --git a/datafusion/core/src/physical_plan/metrics/value.rs b/datafusion/physical-plan/src/metrics/value.rs similarity index 100% rename from datafusion/core/src/physical_plan/metrics/value.rs rename to datafusion/physical-plan/src/metrics/value.rs diff --git a/datafusion/core/src/physical_plan/projection.rs b/datafusion/physical-plan/src/projection.rs similarity index 86% rename from datafusion/core/src/physical_plan/projection.rs rename to datafusion/physical-plan/src/projection.rs index 12c89eee19312..f1ec0a68a6e76 100644 --- a/datafusion/core/src/physical_plan/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -29,7 +29,7 @@ use std::task::{Context, Poll}; use super::expressions::{Column, PhysicalSortExpr}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics}; -use crate::physical_plan::{ +use crate::{ ColumnStatistics, DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, PhysicalExpr, }; @@ -508,110 +508,20 @@ impl RecordBatchStream for ProjectionStream { #[cfg(test)] mod tests { use super::*; - use crate::physical_plan::common::collect; - use crate::physical_plan::expressions::{self, col}; - use crate::test::{self}; - use crate::test_util; + use crate::common::collect; + use crate::expressions; + use crate::test; use arrow_schema::DataType; use datafusion_common::ScalarValue; - use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::binary; - use futures::future; - use tempfile::TempDir; - - // Create a binary expression without coercion. Used here when we do not want to coerce the expressions - // to valid types. Usage can result in an execution (after plan) error. - fn binary_simple( - l: Arc, - op: Operator, - r: Arc, - input_schema: &Schema, - ) -> Arc { - binary(l, op, r, input_schema).unwrap() - } - - #[tokio::test] - async fn project_first_column() -> Result<()> { - let task_ctx = Arc::new(TaskContext::default()); - let schema = test_util::aggr_test_schema(); - - let partitions = 4; - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(partitions, tmp_dir.path())?; - - // pick column c1 and name it column c1 in the output schema - let projection = - ProjectionExec::try_new(vec![(col("c1", &schema)?, "c1".to_string())], csv)?; - - let col_field = projection.schema.field(0); - let col_metadata = col_field.metadata(); - let data: &str = &col_metadata["testing"]; - assert_eq!(data, "test"); - - let mut partition_count = 0; - let mut row_count = 0; - for partition in 0..projection.output_partitioning().partition_count() { - partition_count += 1; - let stream = projection.execute(partition, task_ctx.clone())?; - - row_count += stream - .map(|batch| { - let batch = batch.unwrap(); - assert_eq!(1, batch.num_columns()); - batch.num_rows() - }) - .fold(0, |acc, x| future::ready(acc + x)) - .await; - } - assert_eq!(partitions, partition_count); - assert_eq!(100, row_count); - - Ok(()) - } - - #[tokio::test] - async fn project_input_not_partitioning() -> Result<()> { - let schema = test_util::aggr_test_schema(); - - let partitions = 4; - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(partitions, tmp_dir.path())?; - - // pick column c1 and name it column c1 in the output schema - let projection = - ProjectionExec::try_new(vec![(col("c1", &schema)?, "c1".to_string())], csv)?; - assert!(!projection.benefits_from_input_partitioning()[0]); - Ok(()) - } - - #[tokio::test] - async fn project_input_partitioning() -> Result<()> { - let schema = test_util::aggr_test_schema(); - - let partitions = 4; - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(partitions, tmp_dir.path())?; - - let c1 = col("c2", &schema).unwrap(); - let c2 = col("c9", &schema).unwrap(); - let c1_plus_c2 = binary_simple(c1, Operator::Plus, c2, &schema); - - let projection = - ProjectionExec::try_new(vec![(c1_plus_c2, "c2 + c9".to_string())], csv)?; - - assert!(projection.benefits_from_input_partitioning()[0]); - Ok(()) - } #[tokio::test] async fn project_no_column() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(1, tmp_dir.path())?; - let expected = collect(csv.execute(0, task_ctx.clone())?).await.unwrap(); + let exec = test::scan_partitioned(1); + let expected = collect(exec.execute(0, task_ctx.clone())?).await.unwrap(); - let projection = ProjectionExec::try_new(vec![], csv)?; + let projection = ProjectionExec::try_new(vec![], exec)?; let stream = projection.execute(0, task_ctx.clone())?; let output = collect(stream).await.unwrap(); assert_eq!(output.len(), expected.len()); diff --git a/datafusion/core/src/physical_plan/repartition/distributor_channels.rs b/datafusion/physical-plan/src/repartition/distributor_channels.rs similarity index 100% rename from datafusion/core/src/physical_plan/repartition/distributor_channels.rs rename to datafusion/physical-plan/src/repartition/distributor_channels.rs diff --git a/datafusion/core/src/physical_plan/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs similarity index 96% rename from datafusion/core/src/physical_plan/repartition/mod.rs rename to datafusion/physical-plan/src/repartition/mod.rs index a69f33aa19c1e..14b54dc0614d8 100644 --- a/datafusion/core/src/physical_plan/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -24,14 +24,12 @@ use std::sync::Arc; use std::task::{Context, Poll}; use std::{any::Any, vec}; -use crate::physical_plan::common::transpose; -use crate::physical_plan::hash_utils::create_hashes; -use crate::physical_plan::metrics::BaselineMetrics; -use crate::physical_plan::repartition::distributor_channels::{ - channels, partition_aware_channels, -}; -use crate::physical_plan::sorts::streaming_merge; -use crate::physical_plan::{ +use crate::common::transpose; +use crate::hash_utils::create_hashes; +use crate::metrics::BaselineMetrics; +use crate::repartition::distributor_channels::{channels, partition_aware_channels}; +use crate::sorts::streaming_merge; +use crate::{ DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, Statistics, }; @@ -896,10 +894,7 @@ impl RecordBatchStream for PerPartitionStream { #[cfg(test)] mod tests { use super::*; - use crate::test::create_vec_batches; use crate::{ - assert_batches_sorted_eq, - physical_plan::{collect, expressions::col, memory::MemoryExec}, test::{ assert_is_pending, exec::{ @@ -907,12 +902,14 @@ mod tests { ErrorExec, MockExec, }, }, + {collect, expressions::col, memory::MemoryExec}, }; use arrow::array::{ArrayRef, StringArray}; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; + use arrow_array::UInt32Array; use datafusion_common::cast::as_string_array; - use datafusion_common::exec_err; + use datafusion_common::{assert_batches_sorted_eq, exec_err}; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use futures::FutureExt; use std::collections::HashSet; @@ -922,7 +919,7 @@ mod tests { async fn one_to_many_round_robin() -> Result<()> { // define input partitions let schema = test_schema(); - let partition = create_vec_batches(&schema, 50); + let partition = create_vec_batches(50); let partitions = vec![partition]; // repartition from 1 input to 4 output @@ -942,7 +939,7 @@ mod tests { async fn many_to_one_round_robin() -> Result<()> { // define input partitions let schema = test_schema(); - let partition = create_vec_batches(&schema, 50); + let partition = create_vec_batches(50); let partitions = vec![partition.clone(), partition.clone(), partition.clone()]; // repartition from 3 input to 1 output @@ -959,7 +956,7 @@ mod tests { async fn many_to_many_round_robin() -> Result<()> { // define input partitions let schema = test_schema(); - let partition = create_vec_batches(&schema, 50); + let partition = create_vec_batches(50); let partitions = vec![partition.clone(), partition.clone(), partition.clone()]; // repartition from 3 input to 5 output @@ -980,7 +977,7 @@ mod tests { async fn many_to_many_hash_partition() -> Result<()> { // define input partitions let schema = test_schema(); - let partition = create_vec_batches(&schema, 50); + let partition = create_vec_batches(50); let partitions = vec![partition.clone(), partition.clone(), partition.clone()]; let output_partitions = repartition( @@ -1035,7 +1032,7 @@ mod tests { tokio::spawn(async move { // define input partitions let schema = test_schema(); - let partition = create_vec_batches(&schema, 50); + let partition = create_vec_batches(50); let partitions = vec![partition.clone(), partition.clone(), partition.clone()]; @@ -1075,7 +1072,7 @@ mod tests { let output_stream = exec.execute(0, task_ctx).unwrap(); // Expect that an error is returned - let result_string = crate::physical_plan::common::collect(output_stream) + let result_string = crate::common::collect(output_stream) .await .unwrap_err() .to_string(); @@ -1101,7 +1098,7 @@ mod tests { let output_stream = exec.execute(0, task_ctx).unwrap(); // Expect that an error is returned - let result_string = crate::physical_plan::common::collect(output_stream) + let result_string = crate::common::collect(output_stream) .await .unwrap_err() .to_string(); @@ -1134,7 +1131,7 @@ mod tests { let output_stream = exec.execute(0, task_ctx).unwrap(); // Expect that an error is returned - let result_string = crate::physical_plan::common::collect(output_stream) + let result_string = crate::common::collect(output_stream) .await .unwrap_err() .to_string(); @@ -1182,9 +1179,7 @@ mod tests { assert_batches_sorted_eq!(&expected, &expected_batches); let output_stream = exec.execute(0, task_ctx).unwrap(); - let batches = crate::physical_plan::common::collect(output_stream) - .await - .unwrap(); + let batches = crate::common::collect(output_stream).await.unwrap(); assert_batches_sorted_eq!(&expected, &batches); } @@ -1211,9 +1206,7 @@ mod tests { input.wait().await; // output stream 1 should *not* error and have one of the input batches - let batches = crate::physical_plan::common::collect(output_stream1) - .await - .unwrap(); + let batches = crate::common::collect(output_stream1).await.unwrap(); let expected = vec![ "+------------------+", @@ -1236,7 +1229,7 @@ mod tests { async fn hash_repartition_with_dropping_output_stream() { let task_ctx = Arc::new(TaskContext::default()); let partitioning = Partitioning::Hash( - vec![Arc::new(crate::physical_plan::expressions::Column::new( + vec![Arc::new(crate::expressions::Column::new( "my_awesome_field", 0, ))], @@ -1248,9 +1241,7 @@ mod tests { let exec = RepartitionExec::try_new(input.clone(), partitioning.clone()).unwrap(); let output_stream1 = exec.execute(1, task_ctx.clone()).unwrap(); input.wait().await; - let batches_without_drop = crate::physical_plan::common::collect(output_stream1) - .await - .unwrap(); + let batches_without_drop = crate::common::collect(output_stream1).await.unwrap(); // run some checks on the result let items_vec = str_batches_to_vec(&batches_without_drop); @@ -1272,9 +1263,7 @@ mod tests { // *before* any outputs are produced std::mem::drop(output_stream0); input.wait().await; - let batches_with_drop = crate::physical_plan::common::collect(output_stream1) - .await - .unwrap(); + let batches_with_drop = crate::common::collect(output_stream1).await.unwrap(); assert_eq!(batches_without_drop, batches_with_drop); } @@ -1359,22 +1348,16 @@ mod tests { )]) .unwrap(); let partitioning = Partitioning::Hash( - vec![Arc::new(crate::physical_plan::expressions::Column::new( - "a", 0, - ))], + vec![Arc::new(crate::expressions::Column::new("a", 0))], 2, ); let schema = batch.schema(); let input = MockExec::new(vec![Ok(batch)], schema); let exec = RepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); let output_stream0 = exec.execute(0, task_ctx.clone()).unwrap(); - let batch0 = crate::physical_plan::common::collect(output_stream0) - .await - .unwrap(); + let batch0 = crate::common::collect(output_stream0).await.unwrap(); let output_stream1 = exec.execute(1, task_ctx.clone()).unwrap(); - let batch1 = crate::physical_plan::common::collect(output_stream1) - .await - .unwrap(); + let batch1 = crate::common::collect(output_stream1).await.unwrap(); assert!(batch0.is_empty() || batch1.is_empty()); Ok(()) } @@ -1383,7 +1366,7 @@ mod tests { async fn oom() -> Result<()> { // define input partitions let schema = test_schema(); - let partition = create_vec_batches(&schema, 50); + let partition = create_vec_batches(50); let input_partitions = vec![partition]; let partitioning = Partitioning::RoundRobinBatch(4); @@ -1414,4 +1397,20 @@ mod tests { Ok(()) } + + /// Create vector batches + fn create_vec_batches(n: usize) -> Vec { + let batch = create_batch(); + (0..n).map(|_| batch.clone()).collect() + } + + /// Create batch + fn create_batch() -> RecordBatch { + let schema = test_schema(); + RecordBatch::try_new( + schema, + vec![Arc::new(UInt32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8]))], + ) + .unwrap() + } } diff --git a/datafusion/core/src/physical_plan/sorts/builder.rs b/datafusion/physical-plan/src/sorts/builder.rs similarity index 100% rename from datafusion/core/src/physical_plan/sorts/builder.rs rename to datafusion/physical-plan/src/sorts/builder.rs diff --git a/datafusion/core/src/physical_plan/sorts/cursor.rs b/datafusion/physical-plan/src/sorts/cursor.rs similarity index 99% rename from datafusion/core/src/physical_plan/sorts/cursor.rs rename to datafusion/physical-plan/src/sorts/cursor.rs index c0c791288644b..baa417649fb08 100644 --- a/datafusion/core/src/physical_plan/sorts/cursor.rs +++ b/datafusion/physical-plan/src/sorts/cursor.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::physical_plan::sorts::sort::SortOptions; +use crate::sorts::sort::SortOptions; use arrow::buffer::ScalarBuffer; use arrow::datatypes::ArrowNativeTypeOp; use arrow::row::{Row, Rows}; diff --git a/datafusion/core/src/physical_plan/sorts/index.rs b/datafusion/physical-plan/src/sorts/index.rs similarity index 100% rename from datafusion/core/src/physical_plan/sorts/index.rs rename to datafusion/physical-plan/src/sorts/index.rs diff --git a/datafusion/core/src/physical_plan/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs similarity index 97% rename from datafusion/core/src/physical_plan/sorts/merge.rs rename to datafusion/physical-plan/src/sorts/merge.rs index f8a1457dd62a1..67685509abe5b 100644 --- a/datafusion/core/src/physical_plan/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -18,15 +18,11 @@ //! Merge that deals with an arbitrary size of streaming inputs. //! This is an order-preserving merge. -use crate::physical_plan::metrics::BaselineMetrics; -use crate::physical_plan::sorts::builder::BatchBuilder; -use crate::physical_plan::sorts::cursor::Cursor; -use crate::physical_plan::sorts::stream::{ - FieldCursorStream, PartitionedStream, RowCursorStream, -}; -use crate::physical_plan::{ - PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, -}; +use crate::metrics::BaselineMetrics; +use crate::sorts::builder::BatchBuilder; +use crate::sorts::cursor::Cursor; +use crate::sorts::stream::{FieldCursorStream, PartitionedStream, RowCursorStream}; +use crate::{PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream}; use arrow::datatypes::{DataType, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow_array::*; diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/physical-plan/src/sorts/mod.rs similarity index 100% rename from datafusion/core/src/physical_plan/sorts/mod.rs rename to datafusion/physical-plan/src/sorts/mod.rs diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs similarity index 91% rename from datafusion/core/src/physical_plan/sorts/sort.rs rename to datafusion/physical-plan/src/sorts/sort.rs index 92fb45142ed0c..7d260d42d9cd4 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -19,14 +19,14 @@ //! It will do in-memory sorting if it has enough memory budget //! but spills to disk if needed. -use crate::physical_plan::common::{spawn_buffered, IPCWriter}; -use crate::physical_plan::expressions::PhysicalSortExpr; -use crate::physical_plan::metrics::{ +use crate::common::{spawn_buffered, IPCWriter}; +use crate::expressions::PhysicalSortExpr; +use crate::metrics::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, }; -use crate::physical_plan::sorts::merge::streaming_merge; -use crate::physical_plan::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; -use crate::physical_plan::{ +use crate::sorts::merge::streaming_merge; +use crate::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; +use crate::{ DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; @@ -890,69 +890,41 @@ impl ExecutionPlan for SortExec { #[cfg(test)] mod tests { use super::*; - use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; - use crate::physical_plan::collect; - use crate::physical_plan::expressions::col; - use crate::physical_plan::memory::MemoryExec; + use crate::coalesce_partitions::CoalescePartitionsExec; + use crate::collect; + use crate::expressions::col; + use crate::memory::MemoryExec; use crate::test; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use arrow::array::*; use arrow::compute::SortOptions; use arrow::datatypes::*; - use datafusion_common::cast::{as_primitive_array, as_string_array}; + use datafusion_common::cast::as_primitive_array; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeConfig; use futures::FutureExt; use std::collections::HashMap; - use tempfile::TempDir; #[tokio::test] async fn test_in_mem_sort() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); let partitions = 4; - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(partitions, tmp_dir.path())?; + let csv = test::scan_partitioned(partitions); let schema = csv.schema(); let sort_exec = Arc::new(SortExec::new( - vec![ - // c1 string column - PhysicalSortExpr { - expr: col("c1", &schema)?, - options: SortOptions::default(), - }, - // c2 uin32 column - PhysicalSortExpr { - expr: col("c2", &schema)?, - options: SortOptions::default(), - }, - // c7 uin8 column - PhysicalSortExpr { - expr: col("c7", &schema)?, - options: SortOptions::default(), - }, - ], + vec![PhysicalSortExpr { + expr: col("i", &schema)?, + options: SortOptions::default(), + }], Arc::new(CoalescePartitionsExec::new(csv)), )); let result = collect(sort_exec, task_ctx.clone()).await?; assert_eq!(result.len(), 1); - - let columns = result[0].columns(); - - let c1 = as_string_array(&columns[0])?; - assert_eq!(c1.value(0), "a"); - assert_eq!(c1.value(c1.len() - 1), "e"); - - let c2 = as_primitive_array::(&columns[1])?; - assert_eq!(c2.value(0), 1); - assert_eq!(c2.value(c2.len() - 1), 5,); - - let c7 = as_primitive_array::(&columns[6])?; - assert_eq!(c7.value(0), 15); - assert_eq!(c7.value(c7.len() - 1), 254,); + assert_eq!(result[0].num_rows(), 400); assert_eq!( task_ctx.runtime_env().memory_pool.reserved(), @@ -965,7 +937,7 @@ mod tests { #[tokio::test] async fn test_sort_spill() -> Result<()> { - // trigger spill there will be 4 batches with 5.5KB for each + // trigger spill w/ 100 batches let session_config = SessionConfig::new(); let sort_spill_reservation_bytes = session_config .options() @@ -980,57 +952,35 @@ mod tests { .with_runtime(runtime), ); - let partitions = 4; - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(partitions, tmp_dir.path())?; - let schema = csv.schema(); + let partitions = 100; + let input = test::scan_partitioned(partitions); + let schema = input.schema(); let sort_exec = Arc::new(SortExec::new( - vec![ - // c1 string column - PhysicalSortExpr { - expr: col("c1", &schema)?, - options: SortOptions::default(), - }, - // c2 uin32 column - PhysicalSortExpr { - expr: col("c2", &schema)?, - options: SortOptions::default(), - }, - // c7 uin8 column - PhysicalSortExpr { - expr: col("c7", &schema)?, - options: SortOptions::default(), - }, - ], - Arc::new(CoalescePartitionsExec::new(csv)), + vec![PhysicalSortExpr { + expr: col("i", &schema)?, + options: SortOptions::default(), + }], + Arc::new(CoalescePartitionsExec::new(input)), )); let result = collect(sort_exec.clone(), task_ctx.clone()).await?; - assert_eq!(result.len(), 1); + assert_eq!(result.len(), 2); // Now, validate metrics let metrics = sort_exec.metrics().unwrap(); - assert_eq!(metrics.output_rows().unwrap(), 100); + assert_eq!(metrics.output_rows().unwrap(), 10000); assert!(metrics.elapsed_compute().unwrap() > 0); assert!(metrics.spill_count().unwrap() > 0); assert!(metrics.spilled_bytes().unwrap() > 0); let columns = result[0].columns(); - let c1 = as_string_array(&columns[0])?; - assert_eq!(c1.value(0), "a"); - assert_eq!(c1.value(c1.len() - 1), "e"); - - let c2 = as_primitive_array::(&columns[1])?; - assert_eq!(c2.value(0), 1); - assert_eq!(c2.value(c2.len() - 1), 5,); - - let c7 = as_primitive_array::(&columns[6])?; - assert_eq!(c7.value(0), 15); - assert_eq!(c7.value(c7.len() - 1), 254,); + let i = as_primitive_array::(&columns[0])?; + assert_eq!(i.value(0), 0); + assert_eq!(i.value(i.len() - 1), 81); assert_eq!( task_ctx.runtime_env().memory_pool.reserved(), @@ -1044,7 +994,7 @@ mod tests { #[tokio::test] async fn test_sort_fetch_memory_calculation() -> Result<()> { // This test mirrors down the size from the example above. - let avg_batch_size = 4000; + let avg_batch_size = 400; let partitions = 4; // A tuple of (fetch, expect_spillage) @@ -1075,29 +1025,15 @@ mod tests { .with_session_config(session_config), ); - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(partitions, tmp_dir.path())?; + let csv = test::scan_partitioned(partitions); let schema = csv.schema(); let sort_exec = Arc::new( SortExec::new( - vec![ - // c1 string column - PhysicalSortExpr { - expr: col("c1", &schema)?, - options: SortOptions::default(), - }, - // c2 uin32 column - PhysicalSortExpr { - expr: col("c2", &schema)?, - options: SortOptions::default(), - }, - // c7 uin8 column - PhysicalSortExpr { - expr: col("c7", &schema)?, - options: SortOptions::default(), - }, - ], + vec![PhysicalSortExpr { + expr: col("i", &schema)?, + options: SortOptions::default(), + }], Arc::new(CoalescePartitionsExec::new(csv)), ) .with_fetch(fetch), diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs similarity index 91% rename from datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs rename to datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index fbad5acb016fc..5b485e0b68e41 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -20,13 +20,11 @@ use std::any::Any; use std::sync::Arc; -use crate::physical_plan::common::spawn_buffered; -use crate::physical_plan::expressions::PhysicalSortExpr; -use crate::physical_plan::metrics::{ - BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet, -}; -use crate::physical_plan::sorts::streaming_merge; -use crate::physical_plan::{ +use crate::common::spawn_buffered; +use crate::expressions::PhysicalSortExpr; +use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use crate::sorts::streaming_merge; +use crate::{ DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; @@ -278,19 +276,18 @@ mod tests { use arrow::record_batch::RecordBatch; use datafusion_execution::config::SessionConfig; use futures::{FutureExt, StreamExt}; - use tempfile::TempDir; - - use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; - use crate::physical_plan::expressions::col; - use crate::physical_plan::memory::MemoryExec; - use crate::physical_plan::metrics::{MetricValue, Timestamp}; - use crate::physical_plan::sorts::sort::SortExec; - use crate::physical_plan::stream::RecordBatchReceiverStream; - use crate::physical_plan::{collect, common}; + + use crate::coalesce_partitions::CoalescePartitionsExec; + use crate::expressions::col; + use crate::memory::MemoryExec; + use crate::metrics::{MetricValue, Timestamp}; + use crate::sorts::sort::SortExec; + use crate::stream::RecordBatchReceiverStream; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; - use crate::test::{self, assert_is_pending}; - use crate::{assert_batches_eq, test_util}; + use crate::test::{self, assert_is_pending, make_partition}; + use crate::{collect, common}; use arrow::array::{Int32Array, StringArray, TimestampNanosecondArray}; + use datafusion_common::assert_batches_eq; use super::*; @@ -563,31 +560,16 @@ mod tests { async fn test_partition_sort() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); let partitions = 4; - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(partitions, tmp_dir.path()).unwrap(); + let csv = test::scan_partitioned(partitions); let schema = csv.schema(); - let sort = vec![ - PhysicalSortExpr { - expr: col("c1", &schema).unwrap(), - options: SortOptions { - descending: true, - nulls_first: true, - }, - }, - PhysicalSortExpr { - expr: col("c2", &schema).unwrap(), - options: Default::default(), - }, - PhysicalSortExpr { - expr: col("c7", &schema).unwrap(), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: col("c12", &schema).unwrap(), - options: SortOptions::default(), + let sort = vec![PhysicalSortExpr { + expr: col("i", &schema).unwrap(), + options: SortOptions { + descending: true, + nulls_first: true, }, - ]; + }]; let basic = basic_sort(csv.clone(), sort.clone(), task_ctx.clone()).await; let partition = partition_sort(csv, sort, task_ctx.clone()).await; @@ -636,8 +618,7 @@ mod tests { context: Arc, ) -> Result> { let partitions = 4; - let tmp_dir = TempDir::new()?; - let csv = test::scan_partitioned_csv(partitions, tmp_dir.path()).unwrap(); + let csv = test::scan_partitioned(partitions); let sorted = basic_sort(csv, sort, context).await; let split: Vec<_> = sizes.iter().map(|x| split_batch(&sorted, *x)).collect(); @@ -650,29 +631,11 @@ mod tests { #[tokio::test] async fn test_partition_sort_streaming_input() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); - let schema = test_util::aggr_test_schema(); - let sort = vec![ - // uint8 - PhysicalSortExpr { - expr: col("c7", &schema).unwrap(), - options: Default::default(), - }, - // int16 - PhysicalSortExpr { - expr: col("c4", &schema).unwrap(), - options: Default::default(), - }, - // utf-8 - PhysicalSortExpr { - expr: col("c1", &schema).unwrap(), - options: SortOptions::default(), - }, - // utf-8 - PhysicalSortExpr { - expr: col("c13", &schema).unwrap(), - options: SortOptions::default(), - }, - ]; + let schema = make_partition(11).schema(); + let sort = vec![PhysicalSortExpr { + expr: col("i", &schema).unwrap(), + options: Default::default(), + }]; let input = sorted_partitioned_input(sort.clone(), &[10, 3, 11], task_ctx.clone()) @@ -680,8 +643,8 @@ mod tests { let basic = basic_sort(input.clone(), sort.clone(), task_ctx.clone()).await; let partition = sorted_merge(input, sort, task_ctx.clone()).await; - assert_eq!(basic.num_rows(), 300); - assert_eq!(partition.num_rows(), 300); + assert_eq!(basic.num_rows(), 1200); + assert_eq!(partition.num_rows(), 1200); let basic = arrow::util::pretty::pretty_format_batches(&[basic]) .unwrap() @@ -697,20 +660,11 @@ mod tests { #[tokio::test] async fn test_partition_sort_streaming_input_output() -> Result<()> { - let schema = test_util::aggr_test_schema(); - - let sort = vec![ - // float64 - PhysicalSortExpr { - expr: col("c12", &schema).unwrap(), - options: Default::default(), - }, - // utf-8 - PhysicalSortExpr { - expr: col("c13", &schema).unwrap(), - options: Default::default(), - }, - ]; + let schema = make_partition(11).schema(); + let sort = vec![PhysicalSortExpr { + expr: col("i", &schema).unwrap(), + options: Default::default(), + }]; // Test streaming with default batch size let task_ctx = Arc::new(TaskContext::default()); @@ -727,10 +681,10 @@ mod tests { let merge = Arc::new(SortPreservingMergeExec::new(sort, input)); let merged = collect(merge, task_ctx).await.unwrap(); - assert_eq!(merged.len(), 14); + assert_eq!(merged.len(), 53); - assert_eq!(basic.num_rows(), 300); - assert_eq!(merged.iter().map(|x| x.num_rows()).sum::(), 300); + assert_eq!(basic.num_rows(), 1200); + assert_eq!(merged.iter().map(|x| x.num_rows()).sum::(), 1200); let basic = arrow::util::pretty::pretty_format_batches(&[basic]) .unwrap() @@ -828,9 +782,9 @@ mod tests { #[tokio::test] async fn test_async() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); - let schema = test_util::aggr_test_schema(); + let schema = make_partition(11).schema(); let sort = vec![PhysicalSortExpr { - expr: col("c12", &schema).unwrap(), + expr: col("i", &schema).unwrap(), options: SortOptions::default(), }]; diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/physical-plan/src/sorts/stream.rs similarity index 97% rename from datafusion/core/src/physical_plan/sorts/stream.rs rename to datafusion/physical-plan/src/sorts/stream.rs index 9ef13b7eb25e4..a7f9e7380c473 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/physical-plan/src/sorts/stream.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use crate::physical_plan::sorts::cursor::{FieldArray, FieldCursor, RowCursor}; -use crate::physical_plan::SendableRecordBatchStream; -use crate::physical_plan::{PhysicalExpr, PhysicalSortExpr}; +use crate::sorts::cursor::{FieldArray, FieldCursor, RowCursor}; +use crate::SendableRecordBatchStream; +use crate::{PhysicalExpr, PhysicalSortExpr}; use arrow::array::Array; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; diff --git a/datafusion/core/src/physical_plan/stream.rs b/datafusion/physical-plan/src/stream.rs similarity index 99% rename from datafusion/core/src/physical_plan/stream.rs rename to datafusion/physical-plan/src/stream.rs index 55683a5df3d7b..a3fb856c326d0 100644 --- a/datafusion/core/src/physical_plan/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -22,7 +22,7 @@ use std::sync::Arc; use std::task::Context; use std::task::Poll; -use crate::physical_plan::displayable; +use crate::displayable; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use datafusion_common::DataFusionError; use datafusion_common::Result; diff --git a/datafusion/core/src/physical_plan/streaming.rs b/datafusion/physical-plan/src/streaming.rs similarity index 96% rename from datafusion/core/src/physical_plan/streaming.rs rename to datafusion/physical-plan/src/streaming.rs index 6c33f88a3991e..00809b71e4431 100644 --- a/datafusion/core/src/physical_plan/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -28,9 +28,9 @@ use datafusion_common::{internal_err, plan_err, DataFusionError, Result, Statist use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; use log::debug; -use crate::physical_plan::display::{OutputOrderingDisplay, ProjectSchemaDisplay}; -use crate::physical_plan::stream::RecordBatchStreamAdapter; -use crate::physical_plan::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; +use crate::display::{OutputOrderingDisplay, ProjectSchemaDisplay}; +use crate::stream::RecordBatchStreamAdapter; +use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; use datafusion_execution::TaskContext; use super::{DisplayAs, DisplayFormatType}; diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs new file mode 100644 index 0000000000000..9e6312284c08f --- /dev/null +++ b/datafusion/physical-plan/src/test.rs @@ -0,0 +1,123 @@ +// 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. + +//! Utilities for testing datafusion-physical-plan + +use std::collections::HashMap; +use std::pin::Pin; +use std::sync::Arc; + +use arrow_array::{ArrayRef, Int32Array, RecordBatch}; +use arrow_schema::{DataType, Field, Schema, SchemaRef}; +use futures::{Future, FutureExt}; + +use crate::memory::MemoryExec; +use crate::ExecutionPlan; + +pub mod exec; + +/// Asserts that given future is pending. +pub fn assert_is_pending<'a, T>(fut: &mut Pin + Send + 'a>>) { + let waker = futures::task::noop_waker(); + let mut cx = futures::task::Context::from_waker(&waker); + let poll = fut.poll_unpin(&mut cx); + + assert!(poll.is_pending()); +} + +/// Get the schema for the aggregate_test_* csv files +pub fn aggr_test_schema() -> SchemaRef { + let mut f1 = Field::new("c1", DataType::Utf8, false); + f1.set_metadata(HashMap::from_iter(vec![("testing".into(), "test".into())])); + let schema = Schema::new(vec![ + f1, + Field::new("c2", DataType::UInt32, false), + Field::new("c3", DataType::Int8, false), + Field::new("c4", DataType::Int16, false), + Field::new("c5", DataType::Int32, false), + Field::new("c6", DataType::Int64, false), + Field::new("c7", DataType::UInt8, false), + Field::new("c8", DataType::UInt16, false), + Field::new("c9", DataType::UInt32, false), + Field::new("c10", DataType::UInt64, false), + Field::new("c11", DataType::Float32, false), + Field::new("c12", DataType::Float64, false), + Field::new("c13", DataType::Utf8, false), + ]); + + Arc::new(schema) +} + +/// returns record batch with 3 columns of i32 in memory +pub fn build_table_i32( + a: (&str, &Vec), + b: (&str, &Vec), + c: (&str, &Vec), +) -> RecordBatch { + let schema = Schema::new(vec![ + Field::new(a.0, DataType::Int32, false), + Field::new(b.0, DataType::Int32, false), + Field::new(c.0, DataType::Int32, false), + ]); + + RecordBatch::try_new( + Arc::new(schema), + vec![ + Arc::new(Int32Array::from(a.1.clone())), + Arc::new(Int32Array::from(b.1.clone())), + Arc::new(Int32Array::from(c.1.clone())), + ], + ) + .unwrap() +} + +/// returns memory table scan wrapped around record batch with 3 columns of i32 +pub fn build_table_scan_i32( + a: (&str, &Vec), + b: (&str, &Vec), + c: (&str, &Vec), +) -> Arc { + let batch = build_table_i32(a, b, c); + let schema = batch.schema(); + Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()) +} + +/// Return a RecordBatch with a single Int32 array with values (0..sz) in a field named "i" +pub fn make_partition(sz: i32) -> RecordBatch { + let seq_start = 0; + let seq_end = sz; + let values = (seq_start..seq_end).collect::>(); + let schema = Arc::new(Schema::new(vec![Field::new("i", DataType::Int32, true)])); + let arr = Arc::new(Int32Array::from(values)); + let arr = arr as ArrayRef; + + RecordBatch::try_new(schema, vec![arr]).unwrap() +} + +/// Returns a `MemoryExec` that scans `partitions` of 100 batches each +pub fn scan_partitioned(partitions: usize) -> Arc { + Arc::new(mem_exec(partitions)) +} + +/// Returns a `MemoryExec` that scans `partitions` of 100 batches each +pub fn mem_exec(partitions: usize) -> MemoryExec { + let data: Vec> = (0..partitions).map(|_| vec![make_partition(100)]).collect(); + + let schema = data[0][0].schema(); + let projection = None; + MemoryExec::try_new(&data, schema, projection).unwrap() +} diff --git a/datafusion/core/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs similarity index 98% rename from datafusion/core/src/test/exec.rs rename to datafusion/physical-plan/src/test/exec.rs index 44ce5cf3282b1..a1f40c7ba909c 100644 --- a/datafusion/core/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -31,20 +31,15 @@ use arrow::{ }; use futures::Stream; -use crate::physical_plan::{ - common, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, - SendableRecordBatchStream, Statistics, -}; -use crate::physical_plan::{expressions::PhysicalSortExpr, DisplayAs}; use crate::{ - error::{DataFusionError, Result}, - physical_plan::stream::RecordBatchReceiverStream, -}; -use crate::{ - execution::context::TaskContext, physical_plan::stream::RecordBatchStreamAdapter, + common, stream::RecordBatchReceiverStream, stream::RecordBatchStreamAdapter, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, + SendableRecordBatchStream, Statistics, }; +use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_common::internal_err; +use datafusion_common::{internal_err, DataFusionError, Result}; +use datafusion_execution::TaskContext; /// Index into the data that has been returned so far #[derive(Debug, Default, Clone)] diff --git a/datafusion/core/src/physical_plan/tree_node.rs b/datafusion/physical-plan/src/tree_node.rs similarity index 94% rename from datafusion/core/src/physical_plan/tree_node.rs rename to datafusion/physical-plan/src/tree_node.rs index fad6508fdabef..bce906a00c4d8 100644 --- a/datafusion/core/src/physical_plan/tree_node.rs +++ b/datafusion/physical-plan/src/tree_node.rs @@ -17,7 +17,7 @@ //! This module provides common traits for visiting or rewriting tree nodes easily. -use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; +use crate::{with_new_children_if_necessary, ExecutionPlan}; use datafusion_common::tree_node::{DynTreeNode, Transformed}; use datafusion_common::Result; use std::sync::Arc; diff --git a/datafusion/core/src/physical_plan/udaf.rs b/datafusion/physical-plan/src/udaf.rs similarity index 99% rename from datafusion/core/src/physical_plan/udaf.rs rename to datafusion/physical-plan/src/udaf.rs index 70a43bb397f30..7cc3cc7d59fed 100644 --- a/datafusion/core/src/physical_plan/udaf.rs +++ b/datafusion/physical-plan/src/udaf.rs @@ -27,9 +27,9 @@ use arrow::{ }; use super::{expressions::format_state_name, Accumulator, AggregateExpr}; -use crate::physical_plan::PhysicalExpr; use datafusion_common::{not_impl_err, DataFusionError, Result}; pub use datafusion_expr::AggregateUDF; +use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr::aggregate::utils::down_cast_any_ref; use std::sync::Arc; diff --git a/datafusion/core/src/physical_plan/union.rs b/datafusion/physical-plan/src/union.rs similarity index 98% rename from datafusion/core/src/physical_plan/union.rs rename to datafusion/physical-plan/src/union.rs index 491d24c2897b8..af765e257db2f 100644 --- a/datafusion/core/src/physical_plan/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -41,9 +41,9 @@ use super::{ ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use crate::physical_plan::common::get_meet_of_orderings; -use crate::physical_plan::stream::ObservedStream; -use crate::physical_plan::{expressions, metrics::BaselineMetrics}; +use crate::common::get_meet_of_orderings; +use crate::stream::ObservedStream; +use crate::{expressions, metrics::BaselineMetrics}; use datafusion_common::Result; use datafusion_execution::TaskContext; use tokio::macros::support::thread_rng_n; @@ -604,19 +604,17 @@ mod tests { use super::*; use crate::test; - use crate::{physical_plan::collect, scalar::ScalarValue}; + use crate::collect; use arrow::record_batch::RecordBatch; - use tempfile::TempDir; + use datafusion_common::ScalarValue; #[tokio::test] async fn test_union_partitions() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); - let tmp_dir = TempDir::new()?; - - // Create csv's with different partitioning - let csv = test::scan_partitioned_csv(4, tmp_dir.path())?; - let csv2 = test::scan_partitioned_csv(5, tmp_dir.path())?; + // Create inputs with different partitioning + let csv = test::scan_partitioned(4); + let csv2 = test::scan_partitioned(5); let union_exec = Arc::new(UnionExec::new(vec![csv, csv2])); diff --git a/datafusion/core/src/physical_plan/unnest.rs b/datafusion/physical-plan/src/unnest.rs similarity index 99% rename from datafusion/core/src/physical_plan/unnest.rs rename to datafusion/physical-plan/src/unnest.rs index 40c4edc953309..410ea97887e0c 100644 --- a/datafusion/core/src/physical_plan/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -37,7 +37,7 @@ use log::trace; use std::time::Instant; use std::{any::Any, sync::Arc}; -use crate::physical_plan::{ +use crate::{ expressions::Column, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, PhysicalExpr, PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, @@ -309,7 +309,7 @@ fn build_batch_generic_list Result<()> { - let schema = test_util::aggr_test_schema(); + let schema = test::aggr_test_schema(); let empty = ValuesExec::try_new(schema, vec![]); assert!(empty.is_err()); Ok(()) @@ -210,33 +208,24 @@ mod tests { #[test] fn new_exec_with_batches() { - let schema = Arc::new(Schema::new(vec![Field::new( - "col0", - DataType::UInt32, - false, - )])); - let batches = create_vec_batches(&schema, 10); + let batch = make_partition(7); + let schema = batch.schema(); + let batches = vec![batch.clone(), batch]; + let _exec = ValuesExec::try_new_from_batches(schema, batches).unwrap(); } #[test] fn new_exec_with_batches_empty() { - let schema = Arc::new(Schema::new(vec![Field::new( - "col0", - DataType::UInt32, - false, - )])); + let batch = make_partition(7); + let schema = batch.schema(); let _ = ValuesExec::try_new_from_batches(schema, Vec::new()).unwrap_err(); } #[test] fn new_exec_with_batches_invalid_schema() { - let schema = Arc::new(Schema::new(vec![Field::new( - "col0", - DataType::UInt32, - false, - )])); - let batches = create_vec_batches(&schema, 10); + let batch = make_partition(7); + let batches = vec![batch.clone(), batch]; let invalid_schema = Arc::new(Schema::new(vec![ Field::new("col0", DataType::UInt32, false), diff --git a/datafusion/core/src/physical_plan/visitor.rs b/datafusion/physical-plan/src/visitor.rs similarity index 100% rename from datafusion/core/src/physical_plan/visitor.rs rename to datafusion/physical-plan/src/visitor.rs diff --git a/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs similarity index 99% rename from datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs rename to datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 92e66783dc578..c6211c8061ff4 100644 --- a/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -20,14 +20,12 @@ //! the input data seen so far), which makes it appropriate when processing //! infinite inputs. -use crate::physical_plan::expressions::PhysicalSortExpr; -use crate::physical_plan::metrics::{ - BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet, -}; -use crate::physical_plan::windows::{ +use crate::expressions::PhysicalSortExpr; +use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, window_ordering_equivalence, }; -use crate::physical_plan::{ +use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, WindowExpr, }; diff --git a/datafusion/core/src/physical_plan/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs similarity index 69% rename from datafusion/core/src/physical_plan/windows/mod.rs rename to datafusion/physical-plan/src/windows/mod.rs index 17c21834e121f..2a2f8d6d211bf 100644 --- a/datafusion/core/src/physical_plan/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -17,7 +17,7 @@ //! Physical expressions for window functions -use crate::physical_plan::{ +use crate::{ aggregates, expressions::{ cume_dist, dense_rank, lag, lead, percent_rank, rank, Literal, NthValue, Ntile, @@ -358,31 +358,15 @@ pub(crate) fn window_ordering_equivalence( #[cfg(test)] mod tests { use super::*; - use crate::datasource::physical_plan::CsvExec; - use crate::physical_plan::aggregates::AggregateFunction; - use crate::physical_plan::expressions::col; - use crate::physical_plan::{collect, ExecutionPlan}; + use crate::aggregates::AggregateFunction; + use crate::collect; + use crate::expressions::col; + use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; - use crate::test::{self, assert_is_pending, csv_exec_sorted}; - use arrow::array::*; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, SchemaRef}; - use arrow::record_batch::RecordBatch; - use datafusion_common::cast::as_primitive_array; use datafusion_execution::TaskContext; - use datafusion_expr::{create_udaf, Accumulator, Volatility}; use futures::FutureExt; - use std::path::Path; - use tempfile::TempDir; - - fn create_test_schema( - partitions: usize, - work_dir: &Path, - ) -> Result<(Arc, SchemaRef)> { - let csv = test::scan_partitioned_csv(partitions, work_dir)?; - let schema = csv.schema(); - Ok((csv, schema)) - } fn create_test_schema2() -> Result { let a = Field::new("a", DataType::Int32, true); @@ -394,57 +378,6 @@ mod tests { Ok(schema) } - /// make PhysicalSortExpr with default options - fn sort_expr(name: &str, schema: &Schema) -> PhysicalSortExpr { - sort_expr_options(name, schema, SortOptions::default()) - } - - /// PhysicalSortExpr with specified options - fn sort_expr_options( - name: &str, - schema: &Schema, - options: SortOptions, - ) -> PhysicalSortExpr { - PhysicalSortExpr { - expr: col(name, schema).unwrap(), - options, - } - } - - #[tokio::test] - async fn test_get_partition_by_ordering() -> Result<()> { - let test_schema = create_test_schema2()?; - // Columns a,c are nullable whereas b,d are not nullable. - // Source is sorted by a ASC NULLS FIRST, b ASC NULLS FIRST, c ASC NULLS FIRST, d ASC NULLS FIRST - // Column e is not ordered. - let sort_exprs = vec![ - sort_expr("a", &test_schema), - sort_expr("b", &test_schema), - sort_expr("c", &test_schema), - sort_expr("d", &test_schema), - ]; - // Input is ordered by a,b,c,d - let input = csv_exec_sorted(&test_schema, sort_exprs, true); - let test_data = vec![ - (vec!["a", "b"], vec![0, 1]), - (vec!["b", "a"], vec![1, 0]), - (vec!["b", "a", "c"], vec![1, 0, 2]), - (vec!["d", "b", "a"], vec![2, 1]), - (vec!["d", "e", "a"], vec![2]), - ]; - for (pb_names, expected) in test_data { - let pb_exprs = pb_names - .iter() - .map(|name| col(name, &test_schema)) - .collect::>>()?; - assert_eq!( - get_ordered_partition_by_indices(&pb_exprs, &input), - expected - ); - } - Ok(()) - } - #[tokio::test] async fn test_calc_requirements() -> Result<()> { let schema = create_test_schema2()?; @@ -509,143 +442,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn window_function_with_udaf() -> Result<()> { - #[derive(Debug)] - struct MyCount(i64); - - impl Accumulator for MyCount { - fn state(&self) -> Result> { - Ok(vec![ScalarValue::Int64(Some(self.0))]) - } - - fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - let array = &values[0]; - self.0 += (array.len() - array.null_count()) as i64; - Ok(()) - } - - fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { - let counts: &Int64Array = arrow::array::as_primitive_array(&states[0]); - if let Some(c) = &arrow::compute::sum(counts) { - self.0 += *c; - } - Ok(()) - } - - fn evaluate(&self) -> Result { - Ok(ScalarValue::Int64(Some(self.0))) - } - - fn size(&self) -> usize { - std::mem::size_of_val(self) - } - } - - let my_count = create_udaf( - "my_count", - vec![DataType::Int64], - Arc::new(DataType::Int64), - Volatility::Immutable, - Arc::new(|_| Ok(Box::new(MyCount(0)))), - Arc::new(vec![DataType::Int64]), - ); - - let task_ctx = Arc::new(TaskContext::default()); - let tmp_dir = TempDir::new()?; - let (input, schema) = create_test_schema(1, tmp_dir.path())?; - - let window_exec = Arc::new(WindowAggExec::try_new( - vec![create_window_expr( - &WindowFunction::AggregateUDF(Arc::new(my_count)), - "my_count".to_owned(), - &[col("c3", &schema)?], - &[], - &[], - Arc::new(WindowFrame::new(false)), - schema.as_ref(), - )?], - input, - schema.clone(), - vec![], - )?); - - let result: Vec = collect(window_exec, task_ctx).await?; - assert_eq!(result.len(), 1); - - let n_schema_fields = schema.fields().len(); - let columns = result[0].columns(); - - let count: &Int64Array = as_primitive_array(&columns[n_schema_fields])?; - assert_eq!(count.value(0), 100); - assert_eq!(count.value(99), 100); - Ok(()) - } - - #[tokio::test] - async fn window_function() -> Result<()> { - let task_ctx = Arc::new(TaskContext::default()); - let tmp_dir = TempDir::new()?; - let (input, schema) = create_test_schema(1, tmp_dir.path())?; - - let window_exec = Arc::new(WindowAggExec::try_new( - vec![ - create_window_expr( - &WindowFunction::AggregateFunction(AggregateFunction::Count), - "count".to_owned(), - &[col("c3", &schema)?], - &[], - &[], - Arc::new(WindowFrame::new(false)), - schema.as_ref(), - )?, - create_window_expr( - &WindowFunction::AggregateFunction(AggregateFunction::Max), - "max".to_owned(), - &[col("c3", &schema)?], - &[], - &[], - Arc::new(WindowFrame::new(false)), - schema.as_ref(), - )?, - create_window_expr( - &WindowFunction::AggregateFunction(AggregateFunction::Min), - "min".to_owned(), - &[col("c3", &schema)?], - &[], - &[], - Arc::new(WindowFrame::new(false)), - schema.as_ref(), - )?, - ], - input, - schema.clone(), - vec![], - )?); - - let result: Vec = collect(window_exec, task_ctx).await?; - assert_eq!(result.len(), 1); - - let n_schema_fields = schema.fields().len(); - let columns = result[0].columns(); - - // c3 is small int - - let count: &Int64Array = as_primitive_array(&columns[n_schema_fields])?; - assert_eq!(count.value(0), 100); - assert_eq!(count.value(99), 100); - - let max: &Int8Array = as_primitive_array(&columns[n_schema_fields + 1])?; - assert_eq!(max.value(0), 125); - assert_eq!(max.value(99), 125); - - let min: &Int8Array = as_primitive_array(&columns[n_schema_fields + 2])?; - assert_eq!(min.value(0), -117); - assert_eq!(min.value(99), -117); - - Ok(()) - } - #[tokio::test] async fn test_drop_cancel() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); diff --git a/datafusion/core/src/physical_plan/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs similarity index 98% rename from datafusion/core/src/physical_plan/windows/window_agg_exec.rs rename to datafusion/physical-plan/src/windows/window_agg_exec.rs index 68d1f33fcde48..da43f127f07be 100644 --- a/datafusion/core/src/physical_plan/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -17,15 +17,13 @@ //! Stream and channel implementations for window function expressions. -use crate::physical_plan::common::transpose; -use crate::physical_plan::expressions::PhysicalSortExpr; -use crate::physical_plan::metrics::{ - BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet, -}; -use crate::physical_plan::windows::{ +use crate::common::transpose; +use crate::expressions::PhysicalSortExpr; +use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; +use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, window_ordering_equivalence, }; -use crate::physical_plan::{ +use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, WindowExpr,