diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index d8e310a0dd21..84354c8c0e9a 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1593,7 +1593,6 @@ dependencies = [ "datafusion-execution", "datafusion-expr", "datafusion-expr-common", - "datafusion-functions-aggregate", "datafusion-physical-expr", "datafusion-physical-expr-common", "datafusion-physical-plan", diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index d8aaad801e5c..d90ec3333cb9 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -54,20 +54,15 @@ path = "examples/external_dependency/query-aws-s3.rs" [dev-dependencies] arrow = { workspace = true } +# arrow_schema is required for record_batch! macro :sad: arrow-flight = { workspace = true } arrow-schema = { workspace = true } async-trait = { workspace = true } bytes = { workspace = true } dashmap = { workspace = true } +# note only use main datafusion crate for examples datafusion = { workspace = true, default-features = true, features = ["avro"] } -datafusion-catalog = { workspace = true } -datafusion-common = { workspace = true, default-features = true } -datafusion-expr = { workspace = true } -datafusion-functions-window-common = { workspace = true } -datafusion-optimizer = { workspace = true, default-features = true } -datafusion-physical-expr = { workspace = true, default-features = true } datafusion-proto = { workspace = true } -datafusion-sql = { workspace = true } env_logger = { workspace = true } futures = { workspace = true } log = { workspace = true } diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index 28a3a2f1de09..7e2139370e20 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -15,11 +15,22 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; +use std::collections::{HashMap, HashSet}; +use std::fs::File; +use std::ops::Range; +use std::path::{Path, PathBuf}; +use std::sync::atomic::{AtomicBool, Ordering}; +use std::sync::Arc; + use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; -use arrow_schema::SchemaRef; +use arrow::datatypes::SchemaRef; use async_trait::async_trait; use bytes::Bytes; use datafusion::catalog::Session; +use datafusion::common::{ + internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, +}; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::physical_plan::parquet::{ ParquetAccessPlan, ParquetExecBuilder, @@ -29,6 +40,8 @@ use datafusion::datasource::physical_plan::{ }; use datafusion::datasource::TableProvider; use datafusion::execution::object_store::ObjectStoreUrl; +use datafusion::logical_expr::utils::conjunction; +use datafusion::logical_expr::{TableProviderFilterPushDown, TableType}; use datafusion::parquet::arrow::arrow_reader::{ ArrowReaderOptions, ParquetRecordBatchReaderBuilder, RowSelection, RowSelector, }; @@ -37,27 +50,15 @@ use datafusion::parquet::arrow::ArrowWriter; use datafusion::parquet::file::metadata::ParquetMetaData; use datafusion::parquet::file::properties::{EnabledStatistics, WriterProperties}; use datafusion::parquet::schema::types::ColumnPath; +use datafusion::physical_expr::utils::{Guarantee, LiteralGuarantee}; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_optimizer::pruning::PruningPredicate; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; -use datafusion_common::{ - internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, -}; -use datafusion_expr::utils::conjunction; -use datafusion_expr::{TableProviderFilterPushDown, TableType}; -use datafusion_physical_expr::utils::{Guarantee, LiteralGuarantee}; use futures::future::BoxFuture; use futures::FutureExt; use object_store::ObjectStore; -use std::any::Any; -use std::collections::{HashMap, HashSet}; -use std::fs::File; -use std::ops::Range; -use std::path::{Path, PathBuf}; -use std::sync::atomic::{AtomicBool, Ordering}; -use std::sync::Arc; use tempfile::TempDir; use url::Url; @@ -282,7 +283,7 @@ impl IndexTableProvider { .transpose()? // if there are no filters, use a literal true to have a predicate // that always evaluates to true we can pass to the index - .unwrap_or_else(|| datafusion_physical_expr::expressions::lit(true)); + .unwrap_or_else(|| datafusion::physical_expr::expressions::lit(true)); Ok(predicate) } diff --git a/datafusion-examples/examples/advanced_udaf.rs b/datafusion-examples/examples/advanced_udaf.rs index a914cea4a928..fd65c3352bbc 100644 --- a/datafusion-examples/examples/advanced_udaf.rs +++ b/datafusion-examples/examples/advanced_udaf.rs @@ -15,27 +15,25 @@ // specific language governing permissions and limitations // under the License. -use arrow_schema::{Field, Schema}; +use arrow::datatypes::{Field, Schema}; +use datafusion::physical_expr::NullState; use datafusion::{arrow::datatypes::DataType, logical_expr::Volatility}; -use datafusion_physical_expr::NullState; use std::{any::Any, sync::Arc}; -use arrow::{ - array::{ - ArrayRef, AsArray, Float32Array, PrimitiveArray, PrimitiveBuilder, UInt32Array, - }, - datatypes::{ArrowNativeTypeOp, ArrowPrimitiveType, Float64Type, UInt32Type}, - record_batch::RecordBatch, +use arrow::array::{ + ArrayRef, AsArray, Float32Array, PrimitiveArray, PrimitiveBuilder, UInt32Array, }; +use arrow::datatypes::{ArrowNativeTypeOp, ArrowPrimitiveType, Float64Type, UInt32Type}; +use arrow::record_batch::RecordBatch; +use datafusion::common::{cast::as_float64_array, ScalarValue}; use datafusion::error::Result; -use datafusion::prelude::*; -use datafusion_common::{cast::as_float64_array, ScalarValue}; -use datafusion_expr::{ +use datafusion::logical_expr::{ expr::AggregateFunction, function::{AccumulatorArgs, AggregateFunctionSimplification, StateFieldsArgs}, simplify::SimplifyInfo, - Accumulator, AggregateUDF, AggregateUDFImpl, GroupsAccumulator, Signature, + Accumulator, AggregateUDF, AggregateUDFImpl, EmitTo, GroupsAccumulator, Signature, }; +use datafusion::prelude::*; /// This example shows how to use the full AggregateUDFImpl API to implement a user /// defined aggregate function. As in the `simple_udaf.rs` example, this struct implements @@ -308,7 +306,7 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { } /// Generate output, as specified by `emit_to` and update the intermediate state - fn evaluate(&mut self, emit_to: datafusion_expr::EmitTo) -> Result { + fn evaluate(&mut self, emit_to: EmitTo) -> Result { let counts = emit_to.take_needed(&mut self.counts); let prods = emit_to.take_needed(&mut self.prods); let nulls = self.null_state.build(emit_to); @@ -344,7 +342,7 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { } // return arrays for counts and prods - fn state(&mut self, emit_to: datafusion_expr::EmitTo) -> Result> { + fn state(&mut self, emit_to: EmitTo) -> Result> { let nulls = self.null_state.build(emit_to); let nulls = Some(nulls); diff --git a/datafusion-examples/examples/advanced_udf.rs b/datafusion-examples/examples/advanced_udf.rs index ae35cff6facf..290d1c53334b 100644 --- a/datafusion-examples/examples/advanced_udf.rs +++ b/datafusion-examples/examples/advanced_udf.rs @@ -24,14 +24,14 @@ use arrow::array::{ use arrow::compute; use arrow::datatypes::{DataType, Float64Type}; use arrow::record_batch::RecordBatch; +use datafusion::common::{exec_err, internal_err, ScalarValue}; use datafusion::error::Result; +use datafusion::logical_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion::logical_expr::Volatility; -use datafusion::prelude::*; -use datafusion_common::{exec_err, internal_err, ScalarValue}; -use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; -use datafusion_expr::{ +use datafusion::logical_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, }; +use datafusion::prelude::*; /// This example shows how to use the full ScalarUDFImpl API to implement a user /// defined function. As in the `simple_udf.rs` example, this struct implements diff --git a/datafusion-examples/examples/advanced_udwf.rs b/datafusion-examples/examples/advanced_udwf.rs index 49e890467d21..ac326be9cb04 100644 --- a/datafusion-examples/examples/advanced_udwf.rs +++ b/datafusion-examples/examples/advanced_udwf.rs @@ -18,22 +18,24 @@ use datafusion::{arrow::datatypes::DataType, logical_expr::Volatility}; use std::any::Any; +use arrow::datatypes::Field; use arrow::{ array::{ArrayRef, AsArray, Float64Array}, datatypes::Float64Type, }; -use arrow_schema::Field; +use datafusion::common::ScalarValue; use datafusion::error::Result; use datafusion::functions_aggregate::average::avg_udaf; -use datafusion::prelude::*; -use datafusion_common::ScalarValue; -use datafusion_expr::expr::WindowFunction; -use datafusion_expr::function::{WindowFunctionSimplification, WindowUDFFieldArgs}; -use datafusion_expr::simplify::SimplifyInfo; -use datafusion_expr::{ - Expr, PartitionEvaluator, Signature, WindowFrame, WindowUDF, WindowUDFImpl, +use datafusion::logical_expr::expr::WindowFunction; +use datafusion::logical_expr::function::{ + PartitionEvaluatorArgs, WindowFunctionSimplification, WindowUDFFieldArgs, +}; +use datafusion::logical_expr::simplify::SimplifyInfo; +use datafusion::logical_expr::{ + Expr, PartitionEvaluator, Signature, WindowFrame, WindowFunctionDefinition, + WindowUDF, WindowUDFImpl, }; -use datafusion_functions_window_common::partition::PartitionEvaluatorArgs; +use datafusion::prelude::*; /// This example shows how to use the full WindowUDFImpl API to implement a user /// defined window function. As in the `simple_udwf.rs` example, this struct implements @@ -189,7 +191,7 @@ impl WindowUDFImpl for SimplifySmoothItUdf { fn simplify(&self) -> Option { let simplify = |window_function: WindowFunction, _: &dyn SimplifyInfo| { Ok(Expr::WindowFunction(WindowFunction { - fun: datafusion_expr::WindowFunctionDefinition::AggregateUDF(avg_udaf()), + fun: WindowFunctionDefinition::AggregateUDF(avg_udaf()), args: window_function.args, partition_by: window_function.partition_by, order_by: window_function.order_by, diff --git a/datafusion-examples/examples/analyzer_rule.rs b/datafusion-examples/examples/analyzer_rule.rs index aded64ed4105..cb81cd167a88 100644 --- a/datafusion-examples/examples/analyzer_rule.rs +++ b/datafusion-examples/examples/analyzer_rule.rs @@ -16,12 +16,12 @@ // under the License. use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; +use datafusion::common::config::ConfigOptions; +use datafusion::common::tree_node::{Transformed, TreeNode}; +use datafusion::common::Result; +use datafusion::logical_expr::{col, lit, Expr, LogicalPlan, LogicalPlanBuilder}; +use datafusion::optimizer::analyzer::AnalyzerRule; use datafusion::prelude::SessionContext; -use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::Result; -use datafusion_expr::{col, lit, Expr, LogicalPlan, LogicalPlanBuilder}; -use datafusion_optimizer::analyzer::AnalyzerRule; use std::sync::{Arc, Mutex}; /// This example demonstrates how to add your own [`AnalyzerRule`] to diff --git a/datafusion-examples/examples/composed_extension_codec.rs b/datafusion-examples/examples/composed_extension_codec.rs index 5c34eccf26e1..4baefcae507f 100644 --- a/datafusion-examples/examples/composed_extension_codec.rs +++ b/datafusion-examples/examples/composed_extension_codec.rs @@ -36,11 +36,11 @@ use std::ops::Deref; use std::sync::Arc; use datafusion::common::Result; +use datafusion::common::{internal_err, DataFusionError}; +use datafusion::logical_expr::registry::FunctionRegistry; +use datafusion::logical_expr::{AggregateUDF, ScalarUDF}; use datafusion::physical_plan::{DisplayAs, ExecutionPlan}; use datafusion::prelude::SessionContext; -use datafusion_common::{internal_err, DataFusionError}; -use datafusion_expr::registry::FunctionRegistry; -use datafusion_expr::{AggregateUDF, ScalarUDF}; use datafusion_proto::physical_plan::{AsExecutionPlan, PhysicalExtensionCodec}; use datafusion_proto::protobuf; diff --git a/datafusion-examples/examples/csv_json_opener.rs b/datafusion-examples/examples/csv_json_opener.rs index 334e4c83404f..0b71432f621c 100644 --- a/datafusion-examples/examples/csv_json_opener.rs +++ b/datafusion-examples/examples/csv_json_opener.rs @@ -17,7 +17,7 @@ use std::sync::Arc; -use arrow_schema::{DataType, Field, Schema}; +use arrow::datatypes::{DataType, Field, Schema}; use datafusion::{ assert_batches_eq, datasource::{ diff --git a/datafusion-examples/examples/custom_file_format.rs b/datafusion-examples/examples/custom_file_format.rs index 95168597ebaa..5493fa21968e 100644 --- a/datafusion-examples/examples/custom_file_format.rs +++ b/datafusion-examples/examples/custom_file_format.rs @@ -19,11 +19,12 @@ use std::{any::Any, sync::Arc}; use arrow::{ array::{AsArray, RecordBatch, StringArray, UInt8Array}, - datatypes::UInt64Type, + datatypes::{DataType, Field, Schema, SchemaRef, UInt64Type}, }; -use arrow_schema::{DataType, Field, Schema, SchemaRef}; +use datafusion::common::{GetExt, Statistics}; use datafusion::execution::session_state::SessionStateBuilder; use datafusion::physical_expr::LexRequirement; +use datafusion::physical_expr::PhysicalExpr; use datafusion::{ datasource::{ file_format::{ @@ -38,8 +39,6 @@ use datafusion::{ physical_plan::ExecutionPlan, prelude::SessionContext, }; -use datafusion_common::{GetExt, Statistics}; -use datafusion_physical_expr::PhysicalExpr; use object_store::{ObjectMeta, ObjectStore}; use tempfile::tempdir; diff --git a/datafusion-examples/examples/dataframe.rs b/datafusion-examples/examples/dataframe.rs index c93d74dd85b0..6f61c164f41d 100644 --- a/datafusion-examples/examples/dataframe.rs +++ b/datafusion-examples/examples/dataframe.rs @@ -17,15 +17,15 @@ use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; use datafusion::arrow::datatypes::{DataType, Field, Schema}; +use datafusion::common::config::CsvOptions; +use datafusion::common::parsers::CompressionTypeVariant; +use datafusion::common::DataFusionError; +use datafusion::common::ScalarValue; use datafusion::dataframe::DataFrameWriteOptions; use datafusion::error::Result; use datafusion::functions_aggregate::average::avg; use datafusion::functions_aggregate::min_max::max; use datafusion::prelude::*; -use datafusion_common::config::CsvOptions; -use datafusion_common::parsers::CompressionTypeVariant; -use datafusion_common::DataFusionError; -use datafusion_common::ScalarValue; use std::fs::File; use std::io::Write; use std::sync::Arc; diff --git a/datafusion-examples/examples/date_time_functions.rs b/datafusion-examples/examples/date_time_functions.rs index d9e53e61d1ff..dbe9970439df 100644 --- a/datafusion-examples/examples/date_time_functions.rs +++ b/datafusion-examples/examples/date_time_functions.rs @@ -22,9 +22,9 @@ use datafusion::arrow::array::StringArray; use datafusion::arrow::datatypes::{DataType, Field, Schema}; use datafusion::arrow::record_batch::RecordBatch; use datafusion::assert_batches_eq; +use datafusion::common::assert_contains; use datafusion::error::Result; use datafusion::prelude::*; -use datafusion_common::assert_contains; #[tokio::main] async fn main() -> Result<()> { diff --git a/datafusion-examples/examples/deserialize_to_struct.rs b/datafusion-examples/examples/deserialize_to_struct.rs index 5ac3ee6187d1..d6655b3b654f 100644 --- a/datafusion-examples/examples/deserialize_to_struct.rs +++ b/datafusion-examples/examples/deserialize_to_struct.rs @@ -17,9 +17,9 @@ use arrow::array::{AsArray, PrimitiveArray}; use arrow::datatypes::{Float64Type, Int32Type}; +use datafusion::common::assert_batches_eq; use datafusion::error::Result; use datafusion::prelude::*; -use datafusion_common::assert_batches_eq; use futures::StreamExt; /// This example shows how to convert query results into Rust structs by using diff --git a/datafusion-examples/examples/expr_api.rs b/datafusion-examples/examples/expr_api.rs index 943e5d5e027c..6bfde2ebbf52 100644 --- a/datafusion-examples/examples/expr_api.rs +++ b/datafusion-examples/examples/expr_api.rs @@ -22,20 +22,20 @@ use arrow::array::{BooleanArray, Int32Array, Int8Array}; use arrow::record_batch::RecordBatch; use datafusion::arrow::datatypes::{DataType, Field, Schema, TimeUnit}; +use datafusion::common::tree_node::{Transformed, TreeNode}; use datafusion::common::DFSchema; +use datafusion::common::{ScalarValue, ToDFSchema}; use datafusion::error::Result; use datafusion::functions_aggregate::first_last::first_value_udaf; +use datafusion::logical_expr::execution_props::ExecutionProps; +use datafusion::logical_expr::expr::BinaryExpr; +use datafusion::logical_expr::interval_arithmetic::Interval; +use datafusion::logical_expr::simplify::SimplifyContext; +use datafusion::logical_expr::{ColumnarValue, ExprFunctionExt, ExprSchemable, Operator}; +use datafusion::optimizer::analyzer::type_coercion::TypeCoercionRewriter; use datafusion::optimizer::simplify_expressions::ExprSimplifier; use datafusion::physical_expr::{analyze, AnalysisContext, ExprBoundaries}; use datafusion::prelude::*; -use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{ScalarValue, ToDFSchema}; -use datafusion_expr::execution_props::ExecutionProps; -use datafusion_expr::expr::BinaryExpr; -use datafusion_expr::interval_arithmetic::Interval; -use datafusion_expr::simplify::SimplifyContext; -use datafusion_expr::{ColumnarValue, ExprFunctionExt, ExprSchemable, Operator}; -use datafusion_optimizer::analyzer::type_coercion::TypeCoercionRewriter; /// This example demonstrates the DataFusion [`Expr`] API. /// @@ -357,7 +357,7 @@ fn type_coercion_demo() -> Result<()> { // Evaluation with an expression that has not been type coerced cannot succeed. let props = ExecutionProps::default(); let physical_expr = - datafusion_physical_expr::create_physical_expr(&expr, &df_schema, &props)?; + datafusion::physical_expr::create_physical_expr(&expr, &df_schema, &props)?; let e = physical_expr.evaluate(&batch).unwrap_err(); assert!(e .find_root() @@ -373,7 +373,7 @@ fn type_coercion_demo() -> Result<()> { let context = SimplifyContext::new(&props).with_schema(Arc::new(df_schema.clone())); let simplifier = ExprSimplifier::new(context); let coerced_expr = simplifier.coerce(expr.clone(), &df_schema)?; - let physical_expr = datafusion_physical_expr::create_physical_expr( + let physical_expr = datafusion::physical_expr::create_physical_expr( &coerced_expr, &df_schema, &props, @@ -385,7 +385,7 @@ fn type_coercion_demo() -> Result<()> { .clone() .rewrite(&mut TypeCoercionRewriter::new(&df_schema))? .data; - let physical_expr = datafusion_physical_expr::create_physical_expr( + let physical_expr = datafusion::physical_expr::create_physical_expr( &coerced_expr, &df_schema, &props, @@ -413,7 +413,7 @@ fn type_coercion_demo() -> Result<()> { } })? .data; - let physical_expr = datafusion_physical_expr::create_physical_expr( + let physical_expr = datafusion::physical_expr::create_physical_expr( &coerced_expr, &df_schema, &props, diff --git a/datafusion-examples/examples/file_stream_provider.rs b/datafusion-examples/examples/file_stream_provider.rs index e4fd937fd373..e6c59d57e98d 100644 --- a/datafusion-examples/examples/file_stream_provider.rs +++ b/datafusion-examples/examples/file_stream_provider.rs @@ -18,7 +18,7 @@ #[cfg(not(target_os = "windows"))] mod non_windows { use datafusion::assert_batches_eq; - use datafusion_common::instant::Instant; + use datafusion::common::instant::Instant; use std::fs::{File, OpenOptions}; use std::io::Write; use std::path::PathBuf; @@ -27,19 +27,18 @@ mod non_windows { use std::thread; use std::time::Duration; - use arrow::datatypes::{DataType, Field, Schema}; - use arrow_schema::SchemaRef; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use futures::StreamExt; use nix::sys::stat; use nix::unistd; use tempfile::TempDir; use tokio::task::JoinSet; + use datafusion::common::{exec_err, Result}; use datafusion::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; use datafusion::datasource::TableProvider; + use datafusion::logical_expr::SortExpr; use datafusion::prelude::{SessionConfig, SessionContext}; - use datafusion_common::{exec_err, Result}; - use datafusion_expr::SortExpr; // Number of lines written to FIFO const TEST_BATCH_SIZE: usize = 5; @@ -157,7 +156,7 @@ mod non_windows { ])); // Specify the ordering: - let order = vec![vec![datafusion_expr::col("a1").sort(true, false)]]; + let order = vec![vec![datafusion::logical_expr::col("a1").sort(true, false)]]; let provider = fifo_table(schema.clone(), fifo_path, order.clone()); ctx.register_table("fifo", provider)?; @@ -189,7 +188,7 @@ mod non_windows { } #[tokio::main] -async fn main() -> datafusion_common::Result<()> { +async fn main() -> datafusion::error::Result<()> { #[cfg(target_os = "windows")] { println!("file_stream_provider example does not work on windows"); diff --git a/datafusion-examples/examples/flight/flight_sql_server.rs b/datafusion-examples/examples/flight/flight_sql_server.rs index 2e46daf7cb4e..54e8de7177cb 100644 --- a/datafusion-examples/examples/flight/flight_sql_server.rs +++ b/datafusion-examples/examples/flight/flight_sql_server.rs @@ -16,6 +16,7 @@ // under the License. use arrow::array::{ArrayRef, StringArray}; +use arrow::datatypes::{DataType, Field, Schema}; use arrow::ipc::writer::IpcWriteOptions; use arrow::record_batch::RecordBatch; use arrow_flight::encode::FlightDataEncoderBuilder; @@ -32,7 +33,6 @@ use arrow_flight::{ Action, FlightDescriptor, FlightEndpoint, FlightInfo, HandshakeRequest, HandshakeResponse, IpcMessage, SchemaAsIpc, Ticket, }; -use arrow_schema::{DataType, Field, Schema}; use dashmap::DashMap; use datafusion::logical_expr::LogicalPlan; use datafusion::prelude::{DataFrame, ParquetReadOptions, SessionConfig, SessionContext}; diff --git a/datafusion-examples/examples/function_factory.rs b/datafusion-examples/examples/function_factory.rs index 58ffa060ebaa..06367f5c09e3 100644 --- a/datafusion-examples/examples/function_factory.rs +++ b/datafusion-examples/examples/function_factory.rs @@ -15,20 +15,21 @@ // specific language governing permissions and limitations // under the License. -use std::result::Result as RResult; -use std::sync::Arc; - +use arrow::datatypes::DataType; +use datafusion::common::tree_node::{Transformed, TreeNode}; +use datafusion::common::{exec_err, internal_err, DataFusionError}; use datafusion::error::Result; use datafusion::execution::context::{ FunctionFactory, RegisterFunction, SessionContext, SessionState, }; -use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{exec_err, internal_err, DataFusionError}; -use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; -use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; -use datafusion_expr::{ - CreateFunction, Expr, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, +use datafusion::logical_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; +use datafusion::logical_expr::sort_properties::{ExprProperties, SortProperties}; +use datafusion::logical_expr::{ + ColumnarValue, CreateFunction, Expr, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, + Signature, Volatility, }; +use std::result::Result as RResult; +use std::sync::Arc; /// This example shows how to utilize [FunctionFactory] to implement simple /// SQL-macro like functions using a `CREATE FUNCTION` statement. The same @@ -111,7 +112,7 @@ struct ScalarFunctionWrapper { name: String, expr: Expr, signature: Signature, - return_type: arrow_schema::DataType, + return_type: DataType, } impl ScalarUDFImpl for ScalarFunctionWrapper { @@ -127,17 +128,11 @@ impl ScalarUDFImpl for ScalarFunctionWrapper { &self.signature } - fn return_type( - &self, - _arg_types: &[arrow_schema::DataType], - ) -> Result { + fn return_type(&self, _arg_types: &[DataType]) -> Result { Ok(self.return_type.clone()) } - fn invoke_with_args( - &self, - _args: ScalarFunctionArgs, - ) -> Result { + fn invoke_with_args(&self, _args: ScalarFunctionArgs) -> Result { // Since this function is always simplified to another expression, it // should never actually be invoked internal_err!("This function should not get invoked!") @@ -226,10 +221,7 @@ impl TryFrom for ScalarFunctionWrapper { .into_iter() .map(|a| a.data_type) .collect(), - definition - .params - .behavior - .unwrap_or(datafusion_expr::Volatility::Volatile), + definition.params.behavior.unwrap_or(Volatility::Volatile), ), }) } diff --git a/datafusion-examples/examples/optimizer_rule.rs b/datafusion-examples/examples/optimizer_rule.rs index e8a272f28318..0206c7cd157e 100644 --- a/datafusion-examples/examples/optimizer_rule.rs +++ b/datafusion-examples/examples/optimizer_rule.rs @@ -16,16 +16,16 @@ // under the License. use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; -use arrow_schema::DataType; -use datafusion::prelude::SessionContext; -use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{assert_batches_eq, Result, ScalarValue}; -use datafusion_expr::{ +use arrow::datatypes::DataType; +use datafusion::common::tree_node::{Transformed, TreeNode}; +use datafusion::common::{assert_batches_eq, Result, ScalarValue}; +use datafusion::logical_expr::{ BinaryExpr, ColumnarValue, Expr, LogicalPlan, Operator, ScalarUDF, ScalarUDFImpl, Signature, Volatility, }; -use datafusion_optimizer::optimizer::ApplyOrder; -use datafusion_optimizer::{OptimizerConfig, OptimizerRule}; +use datafusion::optimizer::ApplyOrder; +use datafusion::optimizer::{OptimizerConfig, OptimizerRule}; +use datafusion::prelude::SessionContext; use std::any::Any; use std::sync::Arc; diff --git a/datafusion-examples/examples/parquet_exec_visitor.rs b/datafusion-examples/examples/parquet_exec_visitor.rs index eeb288beb0df..2f1b6333373e 100644 --- a/datafusion-examples/examples/parquet_exec_visitor.rs +++ b/datafusion-examples/examples/parquet_exec_visitor.rs @@ -20,6 +20,7 @@ use std::sync::Arc; use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::{ListingOptions, PartitionedFile}; use datafusion::datasource::physical_plan::ParquetExec; +use datafusion::error::DataFusionError; use datafusion::execution::context::SessionContext; use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::{ @@ -88,7 +89,7 @@ struct ParquetExecVisitor { } impl ExecutionPlanVisitor for ParquetExecVisitor { - type Error = datafusion_common::DataFusionError; + type Error = DataFusionError; /// This function is called once for every node in the tree. /// Based on your needs implement either `pre_visit` (visit each node before its children/inputs) diff --git a/datafusion-examples/examples/parquet_index.rs b/datafusion-examples/examples/parquet_index.rs index d6e17764442d..67fa038e2d31 100644 --- a/datafusion-examples/examples/parquet_index.rs +++ b/datafusion-examples/examples/parquet_index.rs @@ -19,37 +19,37 @@ use arrow::array::{ Array, ArrayRef, AsArray, BooleanArray, Int32Array, RecordBatch, StringArray, UInt64Array, }; -use arrow::datatypes::Int32Type; +use arrow::datatypes::{Int32Type, SchemaRef}; use arrow::util::pretty::pretty_format_batches; -use arrow_schema::SchemaRef; use async_trait::async_trait; use datafusion::catalog::Session; +use datafusion::common::{ + internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, +}; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion::datasource::TableProvider; use datafusion::execution::object_store::ObjectStoreUrl; +use datafusion::logical_expr::{ + utils::conjunction, TableProviderFilterPushDown, TableType, +}; use datafusion::parquet::arrow::arrow_reader::statistics::StatisticsConverter; use datafusion::parquet::arrow::{ arrow_reader::ParquetRecordBatchReaderBuilder, ArrowWriter, }; +use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; -use datafusion_common::{ - internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, -}; -use datafusion_expr::{utils::conjunction, TableProviderFilterPushDown, TableType}; -use datafusion_physical_expr::PhysicalExpr; use std::any::Any; use std::collections::HashSet; use std::fmt::Display; -use std::fs::{self, DirEntry, File}; +use std::fs; +use std::fs::{DirEntry, File}; use std::ops::Range; use std::path::{Path, PathBuf}; -use std::sync::{ - atomic::{AtomicUsize, Ordering}, - Arc, -}; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::Arc; use tempfile::TempDir; use url::Url; @@ -233,7 +233,7 @@ impl TableProvider for IndexTableProvider { .transpose()? // if there are no filters, use a literal true to have a predicate // that always evaluates to true we can pass to the index - .unwrap_or_else(|| datafusion_physical_expr::expressions::lit(true)); + .unwrap_or_else(|| datafusion::physical_expr::expressions::lit(true)); // Use the index to find the files that might have data that matches the // predicate. Any file that can not have data that matches the predicate diff --git a/datafusion-examples/examples/parse_sql_expr.rs b/datafusion-examples/examples/parse_sql_expr.rs index d8f0778e19e3..5387e7c4a05d 100644 --- a/datafusion-examples/examples/parse_sql_expr.rs +++ b/datafusion-examples/examples/parse_sql_expr.rs @@ -16,14 +16,14 @@ // under the License. use arrow::datatypes::{DataType, Field, Schema}; +use datafusion::common::DFSchema; +use datafusion::logical_expr::{col, lit}; +use datafusion::sql::unparser::Unparser; use datafusion::{ assert_batches_eq, error::Result, prelude::{ParquetReadOptions, SessionContext}, }; -use datafusion_common::DFSchema; -use datafusion_expr::{col, lit}; -use datafusion_sql::unparser::Unparser; /// This example demonstrates the programmatic parsing of SQL expressions using /// the DataFusion [`SessionContext::parse_sql_expr`] API or the [`DataFrame::parse_sql_expr`] API. diff --git a/datafusion-examples/examples/plan_to_sql.rs b/datafusion-examples/examples/plan_to_sql.rs index cf1202498416..54483b143a16 100644 --- a/datafusion-examples/examples/plan_to_sql.rs +++ b/datafusion-examples/examples/plan_to_sql.rs @@ -15,24 +15,24 @@ // specific language governing permissions and limitations // under the License. +use datafusion::common::DFSchemaRef; use datafusion::error::Result; use datafusion::logical_expr::sqlparser::ast::Statement; -use datafusion::prelude::*; -use datafusion::sql::unparser::expr_to_sql; -use datafusion_common::DFSchemaRef; -use datafusion_expr::{ +use datafusion::logical_expr::{ Extension, LogicalPlan, LogicalPlanBuilder, UserDefinedLogicalNode, UserDefinedLogicalNodeCore, }; -use datafusion_sql::unparser::ast::{ +use datafusion::prelude::*; +use datafusion::sql::unparser::ast::{ DerivedRelationBuilder, QueryBuilder, RelationBuilder, SelectBuilder, }; -use datafusion_sql::unparser::dialect::CustomDialectBuilder; -use datafusion_sql::unparser::extension_unparser::UserDefinedLogicalNodeUnparser; -use datafusion_sql::unparser::extension_unparser::{ +use datafusion::sql::unparser::dialect::CustomDialectBuilder; +use datafusion::sql::unparser::expr_to_sql; +use datafusion::sql::unparser::extension_unparser::UserDefinedLogicalNodeUnparser; +use datafusion::sql::unparser::extension_unparser::{ UnparseToStatementResult, UnparseWithinStatementResult, }; -use datafusion_sql::unparser::{plan_to_sql, Unparser}; +use datafusion::sql::unparser::{plan_to_sql, Unparser}; use std::fmt; use std::sync::Arc; diff --git a/datafusion-examples/examples/planner_api.rs b/datafusion-examples/examples/planner_api.rs index 35cf766ba1af..e52f0d78682f 100644 --- a/datafusion-examples/examples/planner_api.rs +++ b/datafusion-examples/examples/planner_api.rs @@ -16,10 +16,10 @@ // under the License. use datafusion::error::Result; +use datafusion::logical_expr::{LogicalPlan, PlanType}; use datafusion::physical_plan::displayable; use datafusion::physical_planner::DefaultPhysicalPlanner; use datafusion::prelude::*; -use datafusion_expr::{LogicalPlan, PlanType}; /// This example demonstrates the process of converting logical plan /// into physical execution plans using DataFusion. diff --git a/datafusion-examples/examples/pruning.rs b/datafusion-examples/examples/pruning.rs index c090cd2bcca9..4c802bcdbda0 100644 --- a/datafusion-examples/examples/pruning.rs +++ b/datafusion-examples/examples/pruning.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashSet; +use std::sync::Arc; + use arrow::array::{ArrayRef, BooleanArray, Int32Array}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion::common::{DFSchema, ScalarValue}; @@ -22,8 +25,6 @@ use datafusion::execution::context::ExecutionProps; use datafusion::physical_expr::create_physical_expr; use datafusion::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; use datafusion::prelude::*; -use std::collections::HashSet; -use std::sync::Arc; /// This example shows how to use DataFusion's `PruningPredicate` to prove /// filter expressions can never be true based on statistics such as min/max diff --git a/datafusion-examples/examples/regexp.rs b/datafusion-examples/examples/regexp.rs index 5419efd2faea..12d115b9b502 100644 --- a/datafusion-examples/examples/regexp.rs +++ b/datafusion-examples/examples/regexp.rs @@ -16,9 +16,9 @@ // specific language governing permissions and limitations // under the License. +use datafusion::common::{assert_batches_eq, assert_contains}; use datafusion::error::Result; use datafusion::prelude::*; -use datafusion_common::{assert_batches_eq, assert_contains}; /// This example demonstrates how to use the regexp_* functions /// diff --git a/datafusion-examples/examples/remote_catalog.rs b/datafusion-examples/examples/remote_catalog.rs index 38629328d71c..e44b0c9569f9 100644 --- a/datafusion-examples/examples/remote_catalog.rs +++ b/datafusion-examples/examples/remote_catalog.rs @@ -30,18 +30,18 @@ /// [Unity]: https://github.com/unitycatalog/unitycatalog /// [Hive]: https://hive.apache.org/ use arrow::array::record_batch; -use arrow_schema::{Field, Fields, Schema, SchemaRef}; +use arrow::datatypes::{Field, Fields, Schema, SchemaRef}; use async_trait::async_trait; use datafusion::catalog::TableProvider; +use datafusion::catalog::{AsyncSchemaProvider, Session}; use datafusion::common::Result; +use datafusion::common::{assert_batches_eq, internal_datafusion_err, plan_err}; use datafusion::execution::SendableRecordBatchStream; +use datafusion::logical_expr::{Expr, TableType}; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::{DataFrame, SessionContext}; -use datafusion_catalog::{AsyncSchemaProvider, Session}; -use datafusion_common::{assert_batches_eq, internal_datafusion_err, plan_err}; -use datafusion_expr::{Expr, TableType}; use futures::TryStreamExt; use std::any::Any; use std::sync::Arc; diff --git a/datafusion-examples/examples/simple_udaf.rs b/datafusion-examples/examples/simple_udaf.rs index ef97bf9763b0..82bde7c034a5 100644 --- a/datafusion-examples/examples/simple_udaf.rs +++ b/datafusion-examples/examples/simple_udaf.rs @@ -20,9 +20,9 @@ use datafusion::arrow::{ array::ArrayRef, array::Float32Array, datatypes::DataType, record_batch::RecordBatch, }; +use datafusion::common::cast::as_float64_array; use datafusion::{error::Result, physical_plan::Accumulator}; use datafusion::{logical_expr::Volatility, prelude::*, scalar::ScalarValue}; -use datafusion_common::cast::as_float64_array; use std::sync::Arc; // create local session context with an in-memory table diff --git a/datafusion-examples/examples/simple_udf.rs b/datafusion-examples/examples/simple_udf.rs index 6879a17f34be..5612e0939f70 100644 --- a/datafusion-examples/examples/simple_udf.rs +++ b/datafusion-examples/examples/simple_udf.rs @@ -24,10 +24,10 @@ use datafusion::{ logical_expr::Volatility, }; +use datafusion::common::cast::as_float64_array; use datafusion::error::Result; +use datafusion::logical_expr::ColumnarValue; use datafusion::prelude::*; -use datafusion_common::cast::as_float64_array; -use datafusion_expr::ColumnarValue; use std::sync::Arc; /// create local execution context with an in-memory table: diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs index 7cf1ce87690e..75c7645c18a5 100644 --- a/datafusion-examples/examples/simple_udtf.rs +++ b/datafusion-examples/examples/simple_udtf.rs @@ -21,17 +21,17 @@ use async_trait::async_trait; use datafusion::arrow::datatypes::SchemaRef; use datafusion::arrow::record_batch::RecordBatch; use datafusion::catalog::Session; +use datafusion::catalog::TableFunctionImpl; +use datafusion::common::{plan_err, ScalarValue}; use datafusion::datasource::TableProvider; use datafusion::error::Result; use datafusion::execution::context::ExecutionProps; +use datafusion::logical_expr::simplify::SimplifyContext; +use datafusion::logical_expr::{Expr, TableType}; +use datafusion::optimizer::simplify_expressions::ExprSimplifier; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::ExecutionPlan; -use datafusion::prelude::SessionContext; -use datafusion_catalog::TableFunctionImpl; -use datafusion_common::{plan_err, ScalarValue}; -use datafusion_expr::simplify::SimplifyContext; -use datafusion_expr::{Expr, TableType}; -use datafusion_optimizer::simplify_expressions::ExprSimplifier; +use datafusion::prelude::*; use std::fs::File; use std::io::Seek; use std::path::Path; diff --git a/datafusion-examples/examples/simple_udwf.rs b/datafusion-examples/examples/simple_udwf.rs index 22dfbbbf0c3a..1736ff00bd70 100644 --- a/datafusion-examples/examples/simple_udwf.rs +++ b/datafusion-examples/examples/simple_udwf.rs @@ -19,14 +19,13 @@ use std::sync::Arc; use arrow::{ array::{ArrayRef, AsArray, Float64Array}, - datatypes::Float64Type, + datatypes::{DataType, Float64Type}, }; -use arrow_schema::DataType; +use datafusion::common::ScalarValue; use datafusion::error::Result; +use datafusion::logical_expr::{PartitionEvaluator, Volatility, WindowFrame}; use datafusion::prelude::*; -use datafusion_common::ScalarValue; -use datafusion_expr::{PartitionEvaluator, Volatility, WindowFrame}; // create local execution context with `cars.csv` registered as a table named `cars` async fn create_context() -> Result { diff --git a/datafusion-examples/examples/sql_analysis.rs b/datafusion-examples/examples/sql_analysis.rs index 2158b8e4b016..d3826026a972 100644 --- a/datafusion-examples/examples/sql_analysis.rs +++ b/datafusion-examples/examples/sql_analysis.rs @@ -23,13 +23,13 @@ use std::sync::Arc; +use datafusion::common::tree_node::{TreeNode, TreeNodeRecursion}; use datafusion::common::Result; +use datafusion::logical_expr::LogicalPlan; use datafusion::{ datasource::MemTable, execution::context::{SessionConfig, SessionContext}, }; -use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion}; -use datafusion_expr::LogicalPlan; use test_utils::tpcds::tpcds_schemas; /// Counts the total number of joins in a plan diff --git a/datafusion-examples/examples/sql_dialect.rs b/datafusion-examples/examples/sql_dialect.rs index 259f38216b80..16aa5be02635 100644 --- a/datafusion-examples/examples/sql_dialect.rs +++ b/datafusion-examples/examples/sql_dialect.rs @@ -18,7 +18,7 @@ use std::fmt::Display; use datafusion::error::Result; -use datafusion_sql::{ +use datafusion::sql::{ parser::{CopyToSource, CopyToStatement, DFParser, Statement}, sqlparser::{keywords::Keyword, parser::ParserError, tokenizer::Token}, }; diff --git a/datafusion-examples/examples/sql_frontend.rs b/datafusion-examples/examples/sql_frontend.rs index 839ee95eb181..c7afb79084bf 100644 --- a/datafusion-examples/examples/sql_frontend.rs +++ b/datafusion-examples/examples/sql_frontend.rs @@ -16,19 +16,20 @@ // under the License. use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::config::ConfigOptions; -use datafusion_common::{plan_err, Result}; -use datafusion_expr::{ +use datafusion::common::plan_err; +use datafusion::config::ConfigOptions; +use datafusion::error::Result; +use datafusion::logical_expr::{ AggregateUDF, Expr, LogicalPlan, ScalarUDF, TableProviderFilterPushDown, TableSource, WindowUDF, }; -use datafusion_optimizer::{ +use datafusion::optimizer::{ Analyzer, AnalyzerRule, Optimizer, OptimizerConfig, OptimizerContext, OptimizerRule, }; -use datafusion_sql::planner::{ContextProvider, SqlToRel}; -use datafusion_sql::sqlparser::dialect::PostgreSqlDialect; -use datafusion_sql::sqlparser::parser::Parser; -use datafusion_sql::TableReference; +use datafusion::sql::planner::{ContextProvider, SqlToRel}; +use datafusion::sql::sqlparser::dialect::PostgreSqlDialect; +use datafusion::sql::sqlparser::parser::Parser; +use datafusion::sql::TableReference; use std::any::Any; use std::sync::Arc; diff --git a/datafusion-examples/examples/sql_query.rs b/datafusion-examples/examples/sql_query.rs index a6e7fe91dda5..0ac203cfb7e7 100644 --- a/datafusion-examples/examples/sql_query.rs +++ b/datafusion-examples/examples/sql_query.rs @@ -18,12 +18,12 @@ use datafusion::arrow::array::{UInt64Array, UInt8Array}; use datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion::arrow::record_batch::RecordBatch; +use datafusion::common::{assert_batches_eq, exec_datafusion_err}; use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::ListingOptions; use datafusion::datasource::MemTable; use datafusion::error::{DataFusionError, Result}; -use datafusion::prelude::SessionContext; -use datafusion_common::{assert_batches_eq, exec_datafusion_err}; +use datafusion::prelude::*; use object_store::local::LocalFileSystem; use std::path::Path; use std::sync::Arc; diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 085f44191b8a..25bbe86e8b5c 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -17,6 +17,15 @@ //! [`ParquetExec`] Execution plan for reading Parquet files +mod access_plan; +mod metrics; +mod opener; +mod page_filter; +mod reader; +mod row_filter; +mod row_group_filter; +mod writer; + use std::any::Any; use std::fmt::Debug; use std::sync::Arc; @@ -27,11 +36,13 @@ use crate::datasource::physical_plan::{ parquet::page_filter::PagePruningAccessPlanFilter, DisplayAs, FileGroupPartitioner, FileScanConfig, }; +use crate::datasource::schema_adapter::{ + DefaultSchemaAdapterFactory, SchemaAdapterFactory, +}; use crate::{ config::{ConfigOptions, TableParquetOptions}, error::Result, execution::context::TaskContext, - physical_optimizer::pruning::PruningPredicate, physical_plan::{ metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, @@ -39,33 +50,21 @@ use crate::{ }, }; +pub use access_plan::{ParquetAccessPlan, RowGroupAccess}; use arrow::datatypes::SchemaRef; use datafusion_common::Constraints; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; +use datafusion_physical_optimizer::pruning::PruningPredicate; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; - -use itertools::Itertools; -use log::debug; - -mod access_plan; -mod metrics; -mod opener; -mod page_filter; -mod reader; -mod row_filter; -mod row_group_filter; -mod writer; - -use crate::datasource::schema_adapter::{ - DefaultSchemaAdapterFactory, SchemaAdapterFactory, -}; -pub use access_plan::{ParquetAccessPlan, RowGroupAccess}; pub use metrics::ParquetFileMetrics; use opener::ParquetOpener; pub use reader::{DefaultParquetFileReaderFactory, ParquetFileReaderFactory}; pub use row_filter::can_expr_be_pushed_down_with_schemas; pub use writer::plan_to_parquet; +use itertools::Itertools; +use log::debug; + /// Execution plan for reading one or more Parquet files. /// /// ```text diff --git a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs b/datafusion/core/src/datasource/physical_plan/parquet/opener.rs index 883f296f3b95..a1f8f0172ce4 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/opener.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/opener.rs @@ -17,6 +17,8 @@ //! [`ParquetOpener`] for opening Parquet files +use std::sync::Arc; + use crate::datasource::file_format::{ coerce_file_schema_to_string_type, coerce_file_schema_to_view_type, }; @@ -29,17 +31,18 @@ use crate::datasource::physical_plan::{ FileMeta, FileOpenFuture, FileOpener, ParquetFileMetrics, ParquetFileReaderFactory, }; use crate::datasource::schema_adapter::SchemaAdapterFactory; -use crate::physical_optimizer::pruning::PruningPredicate; + use arrow_schema::{ArrowError, SchemaRef}; use datafusion_common::{exec_err, Result}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_optimizer::pruning::PruningPredicate; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; + use futures::{StreamExt, TryStreamExt}; use log::debug; use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask}; -use std::sync::Arc; /// Implements [`FileOpener`] for a parquet file pub(super) struct ParquetOpener { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs index 4d0a8451a0d4..dcc4b0bc8150 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs @@ -17,14 +17,19 @@ //! Contains code to filter entire pages +use std::collections::HashSet; +use std::sync::Arc; + use super::metrics::ParquetFileMetrics; use crate::datasource::physical_plan::parquet::ParquetAccessPlan; -use crate::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; + use arrow::array::BooleanArray; use arrow::{array::ArrayRef, datatypes::SchemaRef}; use arrow_schema::Schema; use datafusion_common::ScalarValue; use datafusion_physical_expr::{split_conjunction, PhysicalExpr}; +use datafusion_physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; + use log::{debug, trace}; use parquet::arrow::arrow_reader::statistics::StatisticsConverter; use parquet::file::metadata::{ParquetColumnIndex, ParquetOffsetIndex}; @@ -34,8 +39,6 @@ use parquet::{ arrow::arrow_reader::{RowSelection, RowSelector}, file::metadata::{ParquetMetaData, RowGroupMetaData}, }; -use std::collections::HashSet; -use std::sync::Arc; /// Filters a [`ParquetAccessPlan`] based on the [Parquet PageIndex], if present /// diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs index 39d86fe857f7..b008157a8324 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs @@ -15,11 +15,17 @@ // specific language governing permissions and limitations // under the License. +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; + +use super::{ParquetAccessPlan, ParquetFileMetrics}; use crate::datasource::listing::FileRange; -use crate::physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; + use arrow::{array::ArrayRef, datatypes::Schema}; use arrow_array::BooleanArray; use datafusion_common::{Column, Result, ScalarValue}; +use datafusion_physical_optimizer::pruning::{PruningPredicate, PruningStatistics}; + use parquet::arrow::arrow_reader::statistics::StatisticsConverter; use parquet::arrow::parquet_column; use parquet::basic::Type; @@ -30,10 +36,6 @@ use parquet::{ bloom_filter::Sbbf, file::metadata::RowGroupMetaData, }; -use std::collections::{HashMap, HashSet}; -use std::sync::Arc; - -use super::{ParquetAccessPlan, ParquetFileMetrics}; /// Reduces the [`ParquetAccessPlan`] based on row group level metadata. /// diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index c5874deb6ed5..54d505e1b4b9 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -17,6 +17,12 @@ //! [`SessionState`]: information required to run queries in a session +use std::any::Any; +use std::collections::hash_map::Entry; +use std::collections::{HashMap, HashSet}; +use std::fmt::Debug; +use std::sync::Arc; + use crate::catalog::{CatalogProviderList, SchemaProvider, TableProviderFactory}; use crate::catalog_common::information_schema::{ InformationSchemaProvider, INFORMATION_SCHEMA, @@ -27,11 +33,9 @@ use crate::datasource::file_format::{format_as_file_type, FileFormatFactory}; use crate::datasource::provider_as_source; use crate::execution::context::{EmptySerializerRegistry, FunctionFactory, QueryPlanner}; use crate::execution::SessionStateDefaults; -use crate::physical_optimizer::optimizer::PhysicalOptimizer; use crate::physical_planner::{DefaultPhysicalPlanner, PhysicalPlanner}; + use arrow_schema::{DataType, SchemaRef}; -use async_trait::async_trait; -use chrono::{DateTime, Utc}; use datafusion_catalog::{Session, TableFunction, TableFunctionImpl}; use datafusion_common::alias::AliasGenerator; use datafusion_common::config::{ConfigExtension, ConfigOptions, TableOptions}; @@ -61,20 +65,19 @@ use datafusion_optimizer::{ }; use datafusion_physical_expr::create_physical_expr; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_optimizer::optimizer::PhysicalOptimizer; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::ExecutionPlan; use datafusion_sql::parser::{DFParser, Statement}; use datafusion_sql::planner::{ContextProvider, ParserOptions, PlannerContext, SqlToRel}; + +use async_trait::async_trait; +use chrono::{DateTime, Utc}; use itertools::Itertools; use log::{debug, info}; use object_store::ObjectStore; use sqlparser::ast::{Expr as SQLExpr, ExprWithAlias as SQLExprWithAlias}; use sqlparser::dialect::dialect_from_str; -use std::any::Any; -use std::collections::hash_map::Entry; -use std::collections::{HashMap, HashSet}; -use std::fmt::Debug; -use std::sync::Arc; use url::Url; use uuid::Uuid; diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index e9501bd37a8a..5d917e1673f1 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -207,7 +207,7 @@ //! [`QueryPlanner`]: execution::context::QueryPlanner //! [`OptimizerRule`]: datafusion_optimizer::optimizer::OptimizerRule //! [`AnalyzerRule`]: datafusion_optimizer::analyzer::AnalyzerRule -//! [`PhysicalOptimizerRule`]: crate::physical_optimizer::PhysicalOptimizerRule +//! [`PhysicalOptimizerRule`]: datafusion_physical_optimizer::PhysicalOptimizerRule //! //! ## Query Planning and Execution Overview //! @@ -349,7 +349,7 @@ //! filtering can never be `true` using additional statistical information. //! //! [cp_solver]: crate::physical_expr::intervals::cp_solver -//! [`PruningPredicate`]: crate::physical_optimizer::pruning::PruningPredicate +//! [`PruningPredicate`]: datafusion_physical_optimizer::pruning::PruningPredicate //! [`PhysicalExpr`]: crate::physical_plan::PhysicalExpr //! //! ## Execution @@ -624,19 +624,41 @@ //! //! ## Crate Organization //! -//! DataFusion is organized into multiple crates to enforce modularity -//! and improve compilation times. The crates are: +//! Most users interact with DataFusion via this crate (`datafusion`), which re-exports +//! all functionality needed to build and execute queries. +//! +//! There are three other crates that provide additional functionality that +//! must be used directly: +//! * [`datafusion_proto`]: Plan serialization and deserialization +//! * [`datafusion_substrait`]: Support for the substrait plan serialization format +//! * [`datafusion_sqllogictest`] : The DataFusion SQL logic test runner +//! +//! [`datafusion_proto`]: https://crates.io/crates/datafusion-proto +//! [`datafusion_substrait`]: https://crates.io/crates/datafusion-substrait +//! [`datafusion_sqllogictest`]: https://crates.io/crates/datafusion-sqllogictest +//! +//! DataFusion is internally split into multiple sub crates to +//! enforce modularity and improve compilation times. See the +//! [list of modules](#modules) for all available sub-crates. Major ones are //! //! * [datafusion_common]: Common traits and types +//! * [datafusion_catalog]: Catalog APIs such as [`SchemaProvider`] and [`CatalogProvider`] //! * [datafusion_execution]: State and structures needed for execution -//! * [datafusion_expr]: [`LogicalPlan`], [`Expr`] and related logical planning structure +//! * [datafusion_expr]: [`LogicalPlan`], [`Expr`] and related logical planning structure //! * [datafusion_functions]: Scalar function packages +//! * [datafusion_functions_aggregate]: Aggregate functions such as `MIN`, `MAX`, `SUM`, etc //! * [datafusion_functions_nested]: Scalar function packages for `ARRAY`s, `MAP`s and `STRUCT`s +//! * [datafusion_functions_table]: Table Functions such as `GENERATE_SERIES` +//! * [datafusion_functions_window]: Window functions such as `ROW_NUMBER`, `RANK`, etc //! * [datafusion_optimizer]: [`OptimizerRule`]s and [`AnalyzerRule`]s //! * [datafusion_physical_expr]: [`PhysicalExpr`] and related expressions //! * [datafusion_physical_plan]: [`ExecutionPlan`] and related expressions +//! * [datafusion_physical_optimizer]: [`ExecutionPlan`] and related expressions //! * [datafusion_sql]: SQL planner ([`SqlToRel`]) //! +//! [`SchemaProvider`]: datafusion_catalog::SchemaProvider +//! [`CatalogProvider`]: datafusion_catalog::CatalogProvider +//! //! ## Citing DataFusion in Academic Papers //! //! You can use the following citation to reference DataFusion in academic papers: @@ -659,7 +681,7 @@ //! [`OptimizerRule`]: optimizer::optimizer::OptimizerRule //! [`ExecutionPlan`]: physical_plan::ExecutionPlan //! [`PhysicalPlanner`]: physical_planner::PhysicalPlanner -//! [`PhysicalOptimizerRule`]: datafusion::physical_optimizer::optimizer::PhysicalOptimizerRule +//! [`PhysicalOptimizerRule`]: datafusion_physical_optimizer::PhysicalOptimizerRule //! [`Schema`]: arrow::datatypes::Schema //! [`PhysicalExpr`]: physical_plan::PhysicalExpr //! [`RecordBatch`]: arrow::record_batch::RecordBatch @@ -677,7 +699,6 @@ pub mod dataframe; pub mod datasource; pub mod error; pub mod execution; -pub mod physical_optimizer; pub mod physical_planner; pub mod prelude; pub mod scalar; @@ -721,6 +742,11 @@ pub mod optimizer { pub use datafusion_optimizer::*; } +/// re-export of [`datafusion_physical_optimizer`] crate +pub mod physical_optimizer { + pub use datafusion_physical_optimizer::*; +} + /// re-export of [`datafusion_physical_expr`] crate pub mod physical_expr_common { pub use datafusion_physical_expr_common::*; diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs deleted file mode 100644 index e6aa15a4c09d..000000000000 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ /dev/null @@ -1,28 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Optimizer that rewrites [`ExecutionPlan`]s. -//! -//! These rules take advantage of physical plan properties , such as -//! "Repartition" or "Sortedness" -//! -//! [`ExecutionPlan`]: crate::physical_plan::ExecutionPlan - -pub mod optimizer; -pub mod projection_pushdown; - -pub use datafusion_physical_optimizer::*; diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs deleted file mode 100644 index 7a6f991121ef..000000000000 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ /dev/null @@ -1,124 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Physical optimizer traits - -use datafusion_physical_optimizer::PhysicalOptimizerRule; -use std::sync::Arc; - -use super::projection_pushdown::ProjectionPushdown; -use super::update_aggr_exprs::OptimizeAggregateOrder; -use crate::physical_optimizer::aggregate_statistics::AggregateStatistics; -use crate::physical_optimizer::coalesce_batches::CoalesceBatches; -use crate::physical_optimizer::combine_partial_final_agg::CombinePartialFinalAggregate; -use crate::physical_optimizer::enforce_distribution::EnforceDistribution; -use crate::physical_optimizer::enforce_sorting::EnforceSorting; -use crate::physical_optimizer::join_selection::JoinSelection; -use crate::physical_optimizer::limit_pushdown::LimitPushdown; -use crate::physical_optimizer::limited_distinct_aggregation::LimitedDistinctAggregation; -use crate::physical_optimizer::output_requirements::OutputRequirements; -use crate::physical_optimizer::sanity_checker::SanityCheckPlan; -use crate::physical_optimizer::topk_aggregation::TopKAggregation; - -/// A rule-based physical optimizer. -#[derive(Clone, Debug)] -pub struct PhysicalOptimizer { - /// All rules to apply - pub rules: Vec>, -} - -impl Default for PhysicalOptimizer { - fn default() -> Self { - Self::new() - } -} - -impl PhysicalOptimizer { - /// Create a new optimizer using the recommended list of rules - pub fn new() -> Self { - let rules: Vec> = vec![ - // If there is a output requirement of the query, make sure that - // this information is not lost across different rules during optimization. - Arc::new(OutputRequirements::new_add_mode()), - Arc::new(AggregateStatistics::new()), - // Statistics-based join selection will change the Auto mode to a real join implementation, - // like collect left, or hash join, or future sort merge join, which will influence the - // EnforceDistribution and EnforceSorting rules as they decide whether to add additional - // repartitioning and local sorting steps to meet distribution and ordering requirements. - // Therefore, it should run before EnforceDistribution and EnforceSorting. - Arc::new(JoinSelection::new()), - // The LimitedDistinctAggregation rule should be applied before the EnforceDistribution rule, - // as that rule may inject other operations in between the different AggregateExecs. - // Applying the rule early means only directly-connected AggregateExecs must be examined. - Arc::new(LimitedDistinctAggregation::new()), - // The EnforceDistribution rule is for adding essential repartitioning to satisfy distribution - // requirements. Please make sure that the whole plan tree is determined before this rule. - // This rule increases parallelism if doing so is beneficial to the physical plan; i.e. at - // least one of the operators in the plan benefits from increased parallelism. - Arc::new(EnforceDistribution::new()), - // The CombinePartialFinalAggregate rule should be applied after the EnforceDistribution rule - Arc::new(CombinePartialFinalAggregate::new()), - // The EnforceSorting rule is for adding essential local sorting to satisfy the required - // ordering. Please make sure that the whole plan tree is determined before this rule. - // Note that one should always run this rule after running the EnforceDistribution rule - // as the latter may break local sorting requirements. - Arc::new(EnforceSorting::new()), - // Run once after the local sorting requirement is changed - Arc::new(OptimizeAggregateOrder::new()), - // TODO: `try_embed_to_hash_join` in the ProjectionPushdown rule would be block by the CoalesceBatches, so add it before CoalesceBatches. Maybe optimize it in the future. - Arc::new(ProjectionPushdown::new()), - // The CoalesceBatches rule will not influence the distribution and ordering of the - // whole plan tree. Therefore, to avoid influencing other rules, it should run last. - Arc::new(CoalesceBatches::new()), - // Remove the ancillary output requirement operator since we are done with the planning - // phase. - Arc::new(OutputRequirements::new_remove_mode()), - // The aggregation limiter will try to find situations where the accumulator count - // is not tied to the cardinality, i.e. when the output of the aggregation is passed - // into an `order by max(x) limit y`. In this case it will copy the limit value down - // to the aggregation, allowing it to use only y number of accumulators. - Arc::new(TopKAggregation::new()), - // The ProjectionPushdown rule tries to push projections towards - // the sources in the execution plan. As a result of this process, - // a projection can disappear if it reaches the source providers, and - // sequential projections can merge into one. Even if these two cases - // are not present, the load of executors such as join or union will be - // reduced by narrowing their input tables. - Arc::new(ProjectionPushdown::new()), - // The LimitPushdown rule tries to push limits down as far as possible, - // replacing operators with fetching variants, or adding limits - // past operators that support limit pushdown. - Arc::new(LimitPushdown::new()), - // The SanityCheckPlan rule checks whether the order and - // distribution requirements of each node in the plan - // is satisfied. It will also reject non-runnable query - // plans that use pipeline-breaking operators on infinite - // input(s). The rule generates a diagnostic error - // message for invalid plans. It makes no changes to the - // given query plan; i.e. it only acts as a final - // gatekeeping rule. - Arc::new(SanityCheckPlan::new()), - ]; - - Self::with_rules(rules) - } - - /// Create a new optimizer with the given rules - pub fn with_rules(rules: Vec>) -> Self { - Self { rules } - } -} diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs deleted file mode 100644 index 46fddbd3c936..000000000000 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ /dev/null @@ -1,1454 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! This file implements the `ProjectionPushdown` physical optimization rule. -//! The function [`remove_unnecessary_projections`] tries to push down all -//! projections one by one if the operator below is amenable to this. If a -//! projection reaches a source, it can even disappear from the plan entirely. - -use std::sync::Arc; - -use crate::error::Result; -use crate::physical_plan::ExecutionPlan; - -use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{TransformedResult, TreeNode}; - -use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_plan::projection::remove_unnecessary_projections; - -/// This rule inspects `ProjectionExec`'s in the given physical plan and tries to -/// remove or swap with its child. -#[derive(Default, Debug)] -pub struct ProjectionPushdown {} - -impl ProjectionPushdown { - #[allow(missing_docs)] - pub fn new() -> Self { - Self {} - } -} - -impl PhysicalOptimizerRule for ProjectionPushdown { - fn optimize( - &self, - plan: Arc, - _config: &ConfigOptions, - ) -> Result> { - plan.transform_down(remove_unnecessary_projections).data() - } - - fn name(&self) -> &str { - "ProjectionPushdown" - } - - fn schema_check(&self) -> bool { - true - } -} - -#[cfg(test)] -mod tests { - use std::any::Any; - - use super::*; - use crate::datasource::file_format::file_compression_type::FileCompressionType; - use crate::datasource::listing::PartitionedFile; - use crate::datasource::physical_plan::CsvExec; - use crate::datasource::physical_plan::FileScanConfig; - use crate::physical_plan::get_plan_string; - use crate::physical_plan::joins::{ - HashJoinExec, NestedLoopJoinExec, StreamJoinPartitionMode, SymmetricHashJoinExec, - }; - use crate::physical_plan::memory::MemoryExec; - use crate::physical_plan::repartition::RepartitionExec; - use crate::physical_plan::sorts::sort::SortExec; - use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; - - use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; - use datafusion_common::{JoinSide, JoinType, ScalarValue}; - use datafusion_execution::object_store::ObjectStoreUrl; - use datafusion_execution::{SendableRecordBatchStream, TaskContext}; - use datafusion_expr::{ - ColumnarValue, Operator, ScalarUDF, ScalarUDFImpl, Signature, Volatility, - }; - use datafusion_physical_expr::expressions::{ - binary, col, BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, - }; - use datafusion_physical_expr::ScalarFunctionExpr; - use datafusion_physical_expr::{ - Distribution, Partitioning, PhysicalExpr, PhysicalSortExpr, - PhysicalSortRequirement, - }; - use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; - use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; - use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; - use datafusion_physical_plan::filter::FilterExec; - use datafusion_physical_plan::joins::utils::{ColumnIndex, JoinFilter}; - use datafusion_physical_plan::joins::PartitionMode; - use datafusion_physical_plan::projection::{update_expr, ProjectionExec}; - use datafusion_physical_plan::streaming::PartitionStream; - use datafusion_physical_plan::streaming::StreamingTableExec; - use datafusion_physical_plan::union::UnionExec; - - use itertools::Itertools; - - /// Mocked UDF - #[derive(Debug)] - struct DummyUDF { - signature: Signature, - } - - impl DummyUDF { - fn new() -> Self { - Self { - signature: Signature::variadic_any(Volatility::Immutable), - } - } - } - - impl ScalarUDFImpl for DummyUDF { - fn as_any(&self) -> &dyn Any { - self - } - - fn name(&self) -> &str { - "dummy_udf" - } - - fn signature(&self) -> &Signature { - &self.signature - } - - fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(DataType::Int32) - } - - fn invoke_batch( - &self, - _args: &[ColumnarValue], - _number_rows: usize, - ) -> Result { - unimplemented!("DummyUDF::invoke") - } - } - - #[test] - fn test_update_matching_exprs() -> Result<()> { - let exprs: Vec> = vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 3)), - Operator::Divide, - Arc::new(Column::new("e", 5)), - )), - Arc::new(CastExpr::new( - Arc::new(Column::new("a", 3)), - DataType::Float32, - None, - )), - Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), - Arc::new(ScalarFunctionExpr::new( - "scalar_expr", - Arc::new(ScalarUDF::new_from_impl(DummyUDF::new())), - vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 1)), - Operator::Divide, - Arc::new(Column::new("c", 0)), - )), - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Divide, - Arc::new(Column::new("b", 1)), - )), - ], - DataType::Int32, - )), - Arc::new(CaseExpr::try_new( - Some(Arc::new(Column::new("d", 2))), - vec![ - ( - Arc::new(Column::new("a", 3)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d", 2)), - Operator::Plus, - Arc::new(Column::new("e", 5)), - )) as Arc, - ), - ( - Arc::new(Column::new("a", 3)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("e", 5)), - Operator::Plus, - Arc::new(Column::new("d", 2)), - )) as Arc, - ), - ], - Some(Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 3)), - Operator::Modulo, - Arc::new(Column::new("e", 5)), - ))), - )?), - ]; - let child: Vec<(Arc, String)> = vec![ - (Arc::new(Column::new("c", 2)), "c".to_owned()), - (Arc::new(Column::new("b", 1)), "b".to_owned()), - (Arc::new(Column::new("d", 3)), "d".to_owned()), - (Arc::new(Column::new("a", 0)), "a".to_owned()), - (Arc::new(Column::new("f", 5)), "f".to_owned()), - (Arc::new(Column::new("e", 4)), "e".to_owned()), - ]; - - let expected_exprs: Vec> = vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Divide, - Arc::new(Column::new("e", 4)), - )), - Arc::new(CastExpr::new( - Arc::new(Column::new("a", 0)), - DataType::Float32, - None, - )), - Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 5)))), - Arc::new(ScalarFunctionExpr::new( - "scalar_expr", - Arc::new(ScalarUDF::new_from_impl(DummyUDF::new())), - vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 1)), - Operator::Divide, - Arc::new(Column::new("c", 2)), - )), - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Divide, - Arc::new(Column::new("b", 1)), - )), - ], - DataType::Int32, - )), - Arc::new(CaseExpr::try_new( - Some(Arc::new(Column::new("d", 3))), - vec![ - ( - Arc::new(Column::new("a", 0)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d", 3)), - Operator::Plus, - Arc::new(Column::new("e", 4)), - )) as Arc, - ), - ( - Arc::new(Column::new("a", 0)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("e", 4)), - Operator::Plus, - Arc::new(Column::new("d", 3)), - )) as Arc, - ), - ], - Some(Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Modulo, - Arc::new(Column::new("e", 4)), - ))), - )?), - ]; - - for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { - assert!(update_expr(&expr, &child, true)? - .unwrap() - .eq(&expected_expr)); - } - - Ok(()) - } - - #[test] - fn test_update_projected_exprs() -> Result<()> { - let exprs: Vec> = vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 3)), - Operator::Divide, - Arc::new(Column::new("e", 5)), - )), - Arc::new(CastExpr::new( - Arc::new(Column::new("a", 3)), - DataType::Float32, - None, - )), - Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), - Arc::new(ScalarFunctionExpr::new( - "scalar_expr", - Arc::new(ScalarUDF::new_from_impl(DummyUDF::new())), - vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 1)), - Operator::Divide, - Arc::new(Column::new("c", 0)), - )), - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Divide, - Arc::new(Column::new("b", 1)), - )), - ], - DataType::Int32, - )), - Arc::new(CaseExpr::try_new( - Some(Arc::new(Column::new("d", 2))), - vec![ - ( - Arc::new(Column::new("a", 3)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d", 2)), - Operator::Plus, - Arc::new(Column::new("e", 5)), - )) as Arc, - ), - ( - Arc::new(Column::new("a", 3)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("e", 5)), - Operator::Plus, - Arc::new(Column::new("d", 2)), - )) as Arc, - ), - ], - Some(Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 3)), - Operator::Modulo, - Arc::new(Column::new("e", 5)), - ))), - )?), - ]; - let projected_exprs: Vec<(Arc, String)> = vec![ - (Arc::new(Column::new("a", 3)), "a".to_owned()), - (Arc::new(Column::new("b", 1)), "b_new".to_owned()), - (Arc::new(Column::new("c", 0)), "c".to_owned()), - (Arc::new(Column::new("d", 2)), "d_new".to_owned()), - (Arc::new(Column::new("e", 5)), "e".to_owned()), - (Arc::new(Column::new("f", 4)), "f_new".to_owned()), - ]; - - let expected_exprs: Vec> = vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Divide, - Arc::new(Column::new("e", 4)), - )), - Arc::new(CastExpr::new( - Arc::new(Column::new("a", 0)), - DataType::Float32, - None, - )), - Arc::new(NegativeExpr::new(Arc::new(Column::new("f_new", 5)))), - Arc::new(ScalarFunctionExpr::new( - "scalar_expr", - Arc::new(ScalarUDF::new_from_impl(DummyUDF::new())), - vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b_new", 1)), - Operator::Divide, - Arc::new(Column::new("c", 2)), - )), - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Divide, - Arc::new(Column::new("b_new", 1)), - )), - ], - DataType::Int32, - )), - Arc::new(CaseExpr::try_new( - Some(Arc::new(Column::new("d_new", 3))), - vec![ - ( - Arc::new(Column::new("a", 0)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d_new", 3)), - Operator::Plus, - Arc::new(Column::new("e", 4)), - )) as Arc, - ), - ( - Arc::new(Column::new("a", 0)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("e", 4)), - Operator::Plus, - Arc::new(Column::new("d_new", 3)), - )) as Arc, - ), - ], - Some(Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Modulo, - Arc::new(Column::new("e", 4)), - ))), - )?), - ]; - - for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { - assert!(update_expr(&expr, &projected_exprs, false)? - .unwrap() - .eq(&expected_expr)); - } - - Ok(()) - } - - fn create_simple_csv_exec() -> Arc { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])); - Arc::new( - CsvExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_projection(Some(vec![0, 1, 2, 3, 4])), - ) - .with_has_header(false) - .with_delimeter(0) - .with_quote(0) - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), - ) - } - - fn create_projecting_csv_exec() -> Arc { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - ])); - Arc::new( - CsvExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_projection(Some(vec![3, 2, 1])), - ) - .with_has_header(false) - .with_delimeter(0) - .with_quote(0) - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), - ) - } - - fn create_projecting_memory_exec() -> Arc { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])); - - Arc::new(MemoryExec::try_new(&[], schema, Some(vec![2, 0, 3, 4])).unwrap()) - } - - #[test] - fn test_csv_after_projection() -> Result<()> { - let csv = create_projecting_csv_exec(); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("b", 2)), "b".to_string()), - (Arc::new(Column::new("d", 0)), "d".to_string()), - ], - csv.clone(), - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[b@2 as b, d@0 as d]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[d, c, b], has_header=false", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "CsvExec: file_groups={1 group: [[x]]}, projection=[b, d], has_header=false", - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_memory_after_projection() -> Result<()> { - let memory = create_projecting_memory_exec(); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("d", 2)), "d".to_string()), - (Arc::new(Column::new("e", 3)), "e".to_string()), - (Arc::new(Column::new("a", 1)), "a".to_string()), - ], - memory.clone(), - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[d@2 as d, e@3 as e, a@1 as a]", - " MemoryExec: partitions=0, partition_sizes=[]", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = ["MemoryExec: partitions=0, partition_sizes=[]"]; - assert_eq!(get_plan_string(&after_optimize), expected); - assert_eq!( - after_optimize - .clone() - .as_any() - .downcast_ref::() - .unwrap() - .projection() - .clone() - .unwrap(), - vec![3, 4, 0] - ); - - Ok(()) - } - - #[test] - fn test_streaming_table_after_projection() -> Result<()> { - #[derive(Debug)] - struct DummyStreamPartition { - schema: SchemaRef, - } - impl PartitionStream for DummyStreamPartition { - fn schema(&self) -> &SchemaRef { - &self.schema - } - fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { - unreachable!() - } - } - - let streaming_table = StreamingTableExec::try_new( - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])), - vec![Arc::new(DummyStreamPartition { - schema: Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])), - }) as _], - Some(&vec![0_usize, 2, 4, 3]), - vec![ - LexOrdering::new(vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("e", 2)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), - }, - ]), - LexOrdering::new(vec![PhysicalSortExpr { - expr: Arc::new(Column::new("d", 3)), - options: SortOptions::default(), - }]), - ] - .into_iter(), - true, - None, - )?; - let projection = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("d", 3)), "d".to_string()), - (Arc::new(Column::new("e", 2)), "e".to_string()), - (Arc::new(Column::new("a", 0)), "a".to_string()), - ], - Arc::new(streaming_table) as _, - )?) as _; - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let result = after_optimize - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!( - result.partition_schema(), - &Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])) - ); - assert_eq!( - result.projection().clone().unwrap().to_vec(), - vec![3_usize, 4, 0] - ); - assert_eq!( - result.projected_schema(), - &Schema::new(vec![ - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - Field::new("a", DataType::Int32, true), - ]) - ); - assert_eq!( - result.projected_output_ordering().into_iter().collect_vec(), - vec![ - LexOrdering::new(vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("e", 1)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 2)), - options: SortOptions::default(), - }, - ]), - LexOrdering::new(vec![PhysicalSortExpr { - expr: Arc::new(Column::new("d", 0)), - options: SortOptions::default(), - }]), - ] - ); - assert!(result.is_infinite()); - - Ok(()) - } - - #[test] - fn test_projection_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let child_projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("e", 4)), "new_e".to_string()), - (Arc::new(Column::new("a", 0)), "a".to_string()), - (Arc::new(Column::new("b", 1)), "new_b".to_string()), - ], - csv.clone(), - )?); - let top_projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("new_b", 3)), "new_b".to_string()), - ( - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Plus, - Arc::new(Column::new("new_e", 1)), - )), - "binary".to_string(), - ), - (Arc::new(Column::new("new_b", 3)), "newest_b".to_string()), - ], - child_projection.clone(), - )?); - - let initial = get_plan_string(&top_projection); - let expected_initial = [ - "ProjectionExec: expr=[new_b@3 as new_b, c@0 + new_e@1 as binary, new_b@3 as newest_b]", - " ProjectionExec: expr=[c@2 as c, e@4 as new_e, a@0 as a, b@1 as new_b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(top_projection, &ConfigOptions::new())?; - - let expected = [ - "ProjectionExec: expr=[b@1 as new_b, c@2 + e@4 as binary, b@1 as newest_b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_output_req_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let sort_req: Arc = Arc::new(OutputRequirementExec::new( - csv.clone(), - Some(LexRequirement::new(vec![ - PhysicalSortRequirement { - expr: Arc::new(Column::new("b", 1)), - options: Some(SortOptions::default()), - }, - PhysicalSortRequirement { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: Some(SortOptions::default()), - }, - ])), - Distribution::HashPartitioned(vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - ]), - )); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("a", 0)), "new_a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - ], - sort_req.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " OutputRequirementExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected: [&str; 3] = [ - "OutputRequirementExec", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - - assert_eq!(get_plan_string(&after_optimize), expected); - let expected_reqs = LexRequirement::new(vec![ - PhysicalSortRequirement { - expr: Arc::new(Column::new("b", 2)), - options: Some(SortOptions::default()), - }, - PhysicalSortRequirement { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Plus, - Arc::new(Column::new("new_a", 1)), - )), - options: Some(SortOptions::default()), - }, - ]); - assert_eq!( - after_optimize - .as_any() - .downcast_ref::() - .unwrap() - .required_input_ordering()[0] - .clone() - .unwrap(), - expected_reqs - ); - let expected_distribution: Vec> = vec![ - Arc::new(Column::new("new_a", 1)), - Arc::new(Column::new("b", 2)), - ]; - if let Distribution::HashPartitioned(vec) = after_optimize - .as_any() - .downcast_ref::() - .unwrap() - .required_input_distribution()[0] - .clone() - { - assert!(vec - .iter() - .zip(expected_distribution) - .all(|(actual, expected)| actual.eq(&expected))); - } else { - panic!("Expected HashPartitioned distribution!"); - }; - - Ok(()) - } - - #[test] - fn test_coalesce_partitions_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let coalesce_partitions: Arc = - Arc::new(CoalescePartitionsExec::new(csv)); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("b", 1)), "b".to_string()), - (Arc::new(Column::new("a", 0)), "a_new".to_string()), - (Arc::new(Column::new("d", 3)), "d".to_string()), - ], - coalesce_partitions, - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", - " CoalescePartitionsExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "CoalescePartitionsExec", - " ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_filter_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let predicate = Arc::new(BinaryExpr::new( - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 1)), - Operator::Minus, - Arc::new(Column::new("a", 0)), - )), - Operator::Gt, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d", 3)), - Operator::Minus, - Arc::new(Column::new("a", 0)), - )), - )); - let filter: Arc = - Arc::new(FilterExec::try_new(predicate, csv)?); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("a", 0)), "a_new".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - (Arc::new(Column::new("d", 3)), "d".to_string()), - ], - filter.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", - " FilterExec: b@1 - a@0 > d@3 - a@0", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "FilterExec: b@1 - a_new@0 > d@2 - a_new@0", - " ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_join_after_projection() -> Result<()> { - let left_csv = create_simple_csv_exec(); - let right_csv = create_simple_csv_exec(); - - let join: Arc = Arc::new(SymmetricHashJoinExec::try_new( - left_csv, - right_csv, - vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], - // b_left-(1+a_right)<=a_right+c_left - Some(JoinFilter::new( - Arc::new(BinaryExpr::new( - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b_left_inter", 0)), - Operator::Minus, - Arc::new(BinaryExpr::new( - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - Operator::Plus, - Arc::new(Column::new("a_right_inter", 1)), - )), - )), - Operator::LtEq, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a_right_inter", 1)), - Operator::Plus, - Arc::new(Column::new("c_left_inter", 2)), - )), - )), - vec![ - ColumnIndex { - index: 1, - side: JoinSide::Left, - }, - ColumnIndex { - index: 0, - side: JoinSide::Right, - }, - ColumnIndex { - index: 2, - side: JoinSide::Left, - }, - ], - Arc::new(Schema::new(vec![ - Field::new("b_left_inter", DataType::Int32, true), - Field::new("a_right_inter", DataType::Int32, true), - Field::new("c_left_inter", DataType::Int32, true), - ])), - )), - &JoinType::Inner, - true, - None, - None, - StreamJoinPartitionMode::SinglePartition, - )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c_from_left".to_string()), - (Arc::new(Column::new("b", 1)), "b_from_left".to_string()), - (Arc::new(Column::new("a", 0)), "a_from_left".to_string()), - (Arc::new(Column::new("a", 5)), "a_from_right".to_string()), - (Arc::new(Column::new("c", 7)), "c_from_right".to_string()), - ], - join, - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, a@5 as a_from_right, c@7 as c_from_right]", - " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b_from_left@1, c_from_right@1)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " ProjectionExec: expr=[a@0 as a_from_right, c@2 as c_from_right]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - let expected_filter_col_ind = vec![ - ColumnIndex { - index: 1, - side: JoinSide::Left, - }, - ColumnIndex { - index: 0, - side: JoinSide::Right, - }, - ColumnIndex { - index: 0, - side: JoinSide::Left, - }, - ]; - - assert_eq!( - expected_filter_col_ind, - after_optimize - .as_any() - .downcast_ref::() - .unwrap() - .filter() - .unwrap() - .column_indices() - ); - - Ok(()) - } - - #[test] - fn test_join_after_required_projection() -> Result<()> { - let left_csv = create_simple_csv_exec(); - let right_csv = create_simple_csv_exec(); - - let join: Arc = Arc::new(SymmetricHashJoinExec::try_new( - left_csv, - right_csv, - vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], - // b_left-(1+a_right)<=a_right+c_left - Some(JoinFilter::new( - Arc::new(BinaryExpr::new( - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b_left_inter", 0)), - Operator::Minus, - Arc::new(BinaryExpr::new( - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - Operator::Plus, - Arc::new(Column::new("a_right_inter", 1)), - )), - )), - Operator::LtEq, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a_right_inter", 1)), - Operator::Plus, - Arc::new(Column::new("c_left_inter", 2)), - )), - )), - vec![ - ColumnIndex { - index: 1, - side: JoinSide::Left, - }, - ColumnIndex { - index: 0, - side: JoinSide::Right, - }, - ColumnIndex { - index: 2, - side: JoinSide::Left, - }, - ], - Arc::new(Schema::new(vec![ - Field::new("b_left_inter", DataType::Int32, true), - Field::new("a_right_inter", DataType::Int32, true), - Field::new("c_left_inter", DataType::Int32, true), - ])), - )), - &JoinType::Inner, - true, - None, - None, - StreamJoinPartitionMode::SinglePartition, - )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("a", 5)), "a".to_string()), - (Arc::new(Column::new("b", 6)), "b".to_string()), - (Arc::new(Column::new("c", 7)), "c".to_string()), - (Arc::new(Column::new("d", 8)), "d".to_string()), - (Arc::new(Column::new("e", 9)), "e".to_string()), - (Arc::new(Column::new("a", 0)), "a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("d", 3)), "d".to_string()), - (Arc::new(Column::new("e", 4)), "e".to_string()), - ], - join, - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[a@5 as a, b@6 as b, c@7 as c, d@8 as d, e@9 as e, a@0 as a, b@1 as b, c@2 as c, d@3 as d, e@4 as e]", - " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "ProjectionExec: expr=[a@5 as a, b@6 as b, c@7 as c, d@8 as d, e@9 as e, a@0 as a, b@1 as b, c@2 as c, d@3 as d, e@4 as e]", - " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - Ok(()) - } - - #[test] - fn test_nested_loop_join_after_projection() -> Result<()> { - let left_csv = create_simple_csv_exec(); - let right_csv = create_simple_csv_exec(); - - let col_left_a = col("a", &left_csv.schema())?; - let col_right_b = col("b", &right_csv.schema())?; - let col_left_c = col("c", &left_csv.schema())?; - // left_a < right_b - let filter_expr = - binary(col_left_a, Operator::Lt, col_right_b, &Schema::empty())?; - let filter_column_indices = vec![ - ColumnIndex { - index: 0, - side: JoinSide::Left, - }, - ColumnIndex { - index: 1, - side: JoinSide::Right, - }, - ColumnIndex { - index: 2, - side: JoinSide::Right, - }, - ]; - let filter_schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ]); - - let join: Arc = Arc::new(NestedLoopJoinExec::try_new( - left_csv, - right_csv, - Some(JoinFilter::new( - filter_expr, - filter_column_indices, - Arc::new(filter_schema), - )), - &JoinType::Inner, - None, - )?); - - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![(col_left_c, "c".to_string())], - Arc::clone(&join), - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c]", - " NestedLoopJoinExec: join_type=Inner, filter=a@0 < b@1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - let expected = [ - "NestedLoopJoinExec: join_type=Inner, filter=a@0 < b@1, projection=[c@2]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - Ok(()) - } - - #[test] - fn test_hash_join_after_projection() -> Result<()> { - // sql like - // SELECT t1.c as c_from_left, t1.b as b_from_left, t1.a as a_from_left, t2.c as c_from_right FROM t1 JOIN t2 ON t1.b = t2.c WHERE t1.b - (1 + t2.a) <= t2.a + t1.c - let left_csv = create_simple_csv_exec(); - let right_csv = create_simple_csv_exec(); - - let join: Arc = Arc::new(HashJoinExec::try_new( - left_csv, - right_csv, - vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], - // b_left-(1+a_right)<=a_right+c_left - Some(JoinFilter::new( - Arc::new(BinaryExpr::new( - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b_left_inter", 0)), - Operator::Minus, - Arc::new(BinaryExpr::new( - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - Operator::Plus, - Arc::new(Column::new("a_right_inter", 1)), - )), - )), - Operator::LtEq, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a_right_inter", 1)), - Operator::Plus, - Arc::new(Column::new("c_left_inter", 2)), - )), - )), - vec![ - ColumnIndex { - index: 1, - side: JoinSide::Left, - }, - ColumnIndex { - index: 0, - side: JoinSide::Right, - }, - ColumnIndex { - index: 2, - side: JoinSide::Left, - }, - ], - Arc::new(Schema::new(vec![ - Field::new("b_left_inter", DataType::Int32, true), - Field::new("a_right_inter", DataType::Int32, true), - Field::new("c_left_inter", DataType::Int32, true), - ])), - )), - &JoinType::Inner, - None, - PartitionMode::Auto, - true, - )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c_from_left".to_string()), - (Arc::new(Column::new("b", 1)), "b_from_left".to_string()), - (Arc::new(Column::new("a", 0)), "a_from_left".to_string()), - (Arc::new(Column::new("c", 7)), "c_from_right".to_string()), - ], - join.clone(), - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@7 as c_from_right]", " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - // HashJoinExec only returns result after projection. Because there are some alias columns in the projection, the ProjectionExec is not removed. - let expected = ["ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@3 as c_from_right]", " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[a@0, b@1, c@2, c@7]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false"]; - assert_eq!(get_plan_string(&after_optimize), expected); - - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("a", 0)), "a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("c", 7)), "c".to_string()), - ], - join.clone(), - )?); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - // Comparing to the previous result, this projection don't have alias columns either change the order of output fields. So the ProjectionExec is removed. - let expected = ["HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[a@0, b@1, c@2, c@7]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false"]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_repartition_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let repartition: Arc = Arc::new(RepartitionExec::try_new( - csv, - Partitioning::Hash( - vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("d", 3)), - ], - 6, - ), - )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("b", 1)), "b_new".to_string()), - (Arc::new(Column::new("a", 0)), "a".to_string()), - (Arc::new(Column::new("d", 3)), "d_new".to_string()), - ], - repartition, - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", - " RepartitionExec: partitioning=Hash([a@0, b@1, d@3], 6), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "RepartitionExec: partitioning=Hash([a@1, b_new@0, d_new@2], 6), input_partitions=1", - " ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - assert_eq!( - after_optimize - .as_any() - .downcast_ref::() - .unwrap() - .partitioning() - .clone(), - Partitioning::Hash( - vec![ - Arc::new(Column::new("a", 1)), - Arc::new(Column::new("b_new", 0)), - Arc::new(Column::new("d_new", 2)), - ], - 6, - ), - ); - - Ok(()) - } - - #[test] - fn test_sort_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let sort_req: Arc = Arc::new(SortExec::new( - LexOrdering::new(vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: SortOptions::default(), - }, - ]), - csv.clone(), - )); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("a", 0)), "new_a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - ], - sort_req.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " SortExec: expr=[b@1 ASC, c@2 + a@0 ASC], preserve_partitioning=[false]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "SortExec: expr=[b@2 ASC, c@0 + new_a@1 ASC], preserve_partitioning=[false]", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_sort_preserving_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let sort_req: Arc = Arc::new(SortPreservingMergeExec::new( - LexOrdering::new(vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: SortOptions::default(), - }, - ]), - csv.clone(), - )); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("a", 0)), "new_a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - ], - sort_req.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " SortPreservingMergeExec: [b@1 ASC, c@2 + a@0 ASC]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "SortPreservingMergeExec: [b@2 ASC, c@0 + new_a@1 ASC]", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_union_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let union: Arc = - Arc::new(UnionExec::new(vec![csv.clone(), csv.clone(), csv])); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("a", 0)), "new_a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - ], - union.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " UnionExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "UnionExec", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } -} diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index b6f2f8e9ac4a..77e4b491da6d 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -16,45 +16,45 @@ // under the License. //! This module contains tests for limiting memory at runtime in DataFusion + +use std::any::Any; +use std::num::NonZeroUsize; +use std::sync::{Arc, LazyLock}; + #[cfg(feature = "extended_tests")] mod memory_limit_validation; - use arrow::datatypes::{Int32Type, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow_array::{ArrayRef, DictionaryArray}; use arrow_schema::SortOptions; -use async_trait::async_trait; use datafusion::assert_batches_eq; -use datafusion::physical_optimizer::PhysicalOptimizerRule; -use datafusion::physical_plan::memory::MemoryExec; -use datafusion::physical_plan::streaming::PartitionStream; -use datafusion_execution::memory_pool::{ - GreedyMemoryPool, MemoryPool, TrackConsumersPool, -}; -use datafusion_expr::{Expr, TableType}; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; -use datafusion_physical_plan::spill::get_record_batch_memory_size; -use futures::StreamExt; -use std::any::Any; -use std::num::NonZeroUsize; -use std::sync::{Arc, LazyLock}; -use tokio::fs::File; - use datafusion::datasource::streaming::StreamingTable; use datafusion::datasource::{MemTable, TableProvider}; use datafusion::execution::disk_manager::DiskManagerConfig; use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::execution::session_state::SessionStateBuilder; -use datafusion::physical_optimizer::join_selection::JoinSelection; +use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; +use datafusion::physical_plan::streaming::PartitionStream; use datafusion::physical_plan::{ExecutionPlan, SendableRecordBatchStream}; -use datafusion_common::{assert_contains, Result}; - use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_catalog::Session; +use datafusion_common::{assert_contains, Result}; +use datafusion_execution::memory_pool::{ + GreedyMemoryPool, MemoryPool, TrackConsumersPool, +}; use datafusion_execution::TaskContext; +use datafusion_expr::{Expr, TableType}; +use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_optimizer::join_selection::JoinSelection; +use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::spill::get_record_batch_memory_size; use test_utils::AccessLogGenerator; +use async_trait::async_trait; +use futures::StreamExt; +use tokio::fs::File; + #[cfg(test)] #[ctor::ctor] fn init() { diff --git a/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs b/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs index 95da1767a3ef..f5ecd41ab11e 100644 --- a/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs @@ -15,31 +15,30 @@ // specific language governing permissions and limitations // under the License. -use datafusion_common::config::ConfigOptions; -use datafusion_execution::TaskContext; -use datafusion_physical_optimizer::aggregate_statistics::AggregateStatistics; -use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_plan::aggregates::AggregateExec; -use datafusion_physical_plan::projection::ProjectionExec; -use datafusion_physical_plan::ExecutionPlan; use std::sync::Arc; -use datafusion_common::Result; - -use datafusion_physical_plan::aggregates::PhysicalGroupBy; -use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::common; -use datafusion_physical_plan::filter::FilterExec; -use datafusion_physical_plan::memory::MemoryExec; +use crate::physical_optimizer::test_utils::TestAggregate; use arrow::array::Int32Array; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; use datafusion_common::cast::as_int64_array; +use datafusion_common::config::ConfigOptions; +use datafusion_common::Result; +use datafusion_execution::TaskContext; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{self, cast}; -use datafusion_physical_optimizer::test_utils::TestAggregate; +use datafusion_physical_optimizer::aggregate_statistics::AggregateStatistics; +use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::aggregates::AggregateMode; +use datafusion_physical_plan::aggregates::PhysicalGroupBy; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::common; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::memory::MemoryExec; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::ExecutionPlan; /// Mock data using a MemoryExec which has an exact count statistic fn mock_data() -> Result> { diff --git a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs index 85efebf2386a..f0588e45cc6a 100644 --- a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs @@ -22,11 +22,9 @@ use std::sync::Arc; -use crate::physical_optimizer::parquet_exec; +use crate::physical_optimizer::test_utils::{parquet_exec, trim_plan_display}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion::physical_optimizer::combine_partial_final_agg::CombinePartialFinalAggregate; -use datafusion::physical_optimizer::test_utils::trim_plan_display; use datafusion_common::config::ConfigOptions; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::sum::sum_udaf; @@ -34,6 +32,7 @@ use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctio use datafusion_physical_expr::expressions::{col, lit}; use datafusion_physical_expr::Partitioning; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_optimizer::combine_partial_final_agg::CombinePartialFinalAggregate; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index ea75e7d0e5f4..856f7dc8e8a9 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -19,7 +19,11 @@ use std::fmt::Debug; use std::ops::Deref; use std::sync::Arc; -use crate::physical_optimizer::parquet_exec_with_sort; +use crate::physical_optimizer::test_utils::{ + check_integrity, coalesce_partitions_exec, repartition_exec, schema, + sort_merge_join_exec, sort_preserving_merge_exec, +}; +use crate::physical_optimizer::test_utils::{parquet_exec_with_sort, trim_plan_display}; use arrow::compute::SortOptions; use datafusion::config::ConfigOptions; @@ -40,11 +44,6 @@ use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_optimizer::enforce_distribution::*; use datafusion_physical_optimizer::enforce_sorting::EnforceSorting; use datafusion_physical_optimizer::output_requirements::OutputRequirements; -use datafusion_physical_optimizer::test_utils::trim_plan_display; -use datafusion_physical_optimizer::test_utils::{ - check_integrity, coalesce_partitions_exec, repartition_exec, schema, - sort_merge_join_exec, sort_preserving_merge_exec, -}; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, @@ -293,7 +292,7 @@ fn hash_join_exec( join_on: &JoinOn, join_type: &JoinType, ) -> Arc { - datafusion_physical_optimizer::test_utils::hash_join_exec( + crate::physical_optimizer::test_utils::hash_join_exec( left, right, join_on.clone(), diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 4ca8381f2672..66bd1c37d3a0 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -17,7 +17,14 @@ use std::sync::Arc; -use crate::physical_optimizer::parquet_exec; +use crate::physical_optimizer::test_utils::{ + aggregate_exec, bounded_window_exec, check_integrity, coalesce_batches_exec, + coalesce_partitions_exec, create_test_schema, create_test_schema2, + create_test_schema3, filter_exec, global_limit_exec, hash_join_exec, limit_exec, + local_limit_exec, memory_exec, parquet_exec, repartition_exec, sort_exec, sort_expr, + sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, + spr_repartition_exec, stream_exec_ordered, union_exec, RequirementsTestExec, +}; use datafusion_physical_plan::displayable; use arrow::compute::SortOptions; @@ -37,15 +44,14 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{TreeNode, TransformedResult}; -use datafusion_physical_optimizer::test_utils::{check_integrity, bounded_window_exec, coalesce_partitions_exec, create_test_schema, create_test_schema2, create_test_schema3, filter_exec, global_limit_exec, hash_join_exec, limit_exec, local_limit_exec, memory_exec, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, spr_repartition_exec, stream_exec_ordered, union_exec, coalesce_batches_exec, aggregate_exec, RequirementsTestExec}; use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::file_format::file_compression_type::FileCompressionType; - use datafusion_physical_optimizer::enforce_distribution::EnforceDistribution; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::sorts::sort::SortExec; + use rstest::rstest; /// Create a csv exec for tests diff --git a/datafusion/core/tests/physical_optimizer/join_selection.rs b/datafusion/core/tests/physical_optimizer/join_selection.rs new file mode 100644 index 000000000000..ae7adacadb19 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/join_selection.rs @@ -0,0 +1,1494 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; +use std::{ + any::Any, + pin::Pin, + task::{Context, Poll}, +}; + +use arrow::datatypes::{DataType, Field, Schema}; +use arrow::record_batch::RecordBatch; +use arrow_schema::SchemaRef; +use datafusion_common::config::ConfigOptions; +use datafusion_common::JoinSide; +use datafusion_common::{stats::Precision, ColumnStatistics, JoinType, ScalarValue}; +use datafusion_common::{Result, Statistics}; +use datafusion_execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; +use datafusion_expr::Operator; +use datafusion_physical_expr::expressions::col; +use datafusion_physical_expr::expressions::{BinaryExpr, Column, NegativeExpr}; +use datafusion_physical_expr::intervals::utils::check_support; +use datafusion_physical_expr::PhysicalExprRef; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalExpr}; +use datafusion_physical_optimizer::join_selection::{ + hash_join_swap_subrule, JoinSelection, +}; +use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::displayable; +use datafusion_physical_plan::joins::utils::ColumnIndex; +use datafusion_physical_plan::joins::utils::JoinFilter; +use datafusion_physical_plan::joins::{HashJoinExec, NestedLoopJoinExec, PartitionMode}; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::ExecutionPlanProperties; +use datafusion_physical_plan::{ + execution_plan::{Boundedness, EmissionType}, + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, +}; + +use futures::Stream; +use rstest::rstest; + +/// Return statistics for empty table +fn empty_statistics() -> Statistics { + Statistics { + num_rows: Precision::Absent, + total_byte_size: Precision::Absent, + column_statistics: vec![ColumnStatistics::new_unknown()], + } +} + +/// Get table thresholds: (num_rows, byte_size) +fn get_thresholds() -> (usize, usize) { + let optimizer_options = ConfigOptions::new().optimizer; + ( + optimizer_options.hash_join_single_partition_threshold_rows, + optimizer_options.hash_join_single_partition_threshold, + ) +} + +/// Return statistics for small table +fn small_statistics() -> Statistics { + let (threshold_num_rows, threshold_byte_size) = get_thresholds(); + Statistics { + num_rows: Precision::Inexact(threshold_num_rows / 128), + total_byte_size: Precision::Inexact(threshold_byte_size / 128), + column_statistics: vec![ColumnStatistics::new_unknown()], + } +} + +/// Return statistics for big table +fn big_statistics() -> Statistics { + let (threshold_num_rows, threshold_byte_size) = get_thresholds(); + Statistics { + num_rows: Precision::Inexact(threshold_num_rows * 2), + total_byte_size: Precision::Inexact(threshold_byte_size * 2), + column_statistics: vec![ColumnStatistics::new_unknown()], + } +} + +/// Return statistics for big table +fn bigger_statistics() -> Statistics { + let (threshold_num_rows, threshold_byte_size) = get_thresholds(); + Statistics { + num_rows: Precision::Inexact(threshold_num_rows * 4), + total_byte_size: Precision::Inexact(threshold_byte_size * 4), + column_statistics: vec![ColumnStatistics::new_unknown()], + } +} + +fn create_big_and_small() -> (Arc, Arc) { + let big = Arc::new(StatisticsExec::new( + big_statistics(), + Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), + )); + + let small = Arc::new(StatisticsExec::new( + small_statistics(), + Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), + )); + (big, small) +} + +/// Create a column statistics vector for a single column +/// that has the given min/max/distinct_count properties. +/// +/// Given min/max will be mapped to a [`ScalarValue`] if +/// they are not `None`. +fn create_column_stats( + min: Option, + max: Option, + distinct_count: Option, +) -> Vec { + vec![ColumnStatistics { + distinct_count: distinct_count + .map(Precision::Inexact) + .unwrap_or(Precision::Absent), + min_value: min + .map(|size| Precision::Inexact(ScalarValue::UInt64(Some(size)))) + .unwrap_or(Precision::Absent), + max_value: max + .map(|size| Precision::Inexact(ScalarValue::UInt64(Some(size)))) + .unwrap_or(Precision::Absent), + ..Default::default() + }] +} + +/// Create join filter for NLJoinExec with expression `big_col > small_col` +/// where both columns are 0-indexed and come from left and right inputs respectively +fn nl_join_filter() -> Option { + let column_indices = vec![ + ColumnIndex { + index: 0, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ]; + let intermediate_schema = Schema::new(vec![ + Field::new("big_col", DataType::Int32, false), + Field::new("small_col", DataType::Int32, false), + ]); + let expression = Arc::new(BinaryExpr::new( + Arc::new(Column::new_with_schema("big_col", &intermediate_schema).unwrap()), + Operator::Gt, + Arc::new(Column::new_with_schema("small_col", &intermediate_schema).unwrap()), + )) as _; + Some(JoinFilter::new( + expression, + column_indices, + Arc::new(intermediate_schema), + )) +} + +/// Returns three plans with statistics of (min, max, distinct_count) +/// * big 100K rows @ (0, 50k, 50k) +/// * medium 10K rows @ (1k, 5k, 1k) +/// * small 1K rows @ (0, 100k, 1k) +fn create_nested_with_min_max() -> ( + Arc, + Arc, + Arc, +) { + let big = Arc::new(StatisticsExec::new( + Statistics { + num_rows: Precision::Inexact(100_000), + column_statistics: create_column_stats(Some(0), Some(50_000), Some(50_000)), + total_byte_size: Precision::Absent, + }, + Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), + )); + + let medium = Arc::new(StatisticsExec::new( + Statistics { + num_rows: Precision::Inexact(10_000), + column_statistics: create_column_stats(Some(1000), Some(5000), Some(1000)), + total_byte_size: Precision::Absent, + }, + Schema::new(vec![Field::new("medium_col", DataType::Int32, false)]), + )); + + let small = Arc::new(StatisticsExec::new( + Statistics { + num_rows: Precision::Inexact(1000), + column_statistics: create_column_stats(Some(0), Some(100_000), Some(1000)), + total_byte_size: Precision::Absent, + }, + Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), + )); + + (big, medium, small) +} + +#[tokio::test] +async fn test_join_with_swap() { + let (big, small) = create_big_and_small(); + + let join = Arc::new( + HashJoinExec::try_new( + Arc::clone(&big), + Arc::clone(&small), + vec![( + Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()), + Arc::new(Column::new_with_schema("small_col", &small.schema()).unwrap()), + )], + None, + &JoinType::Left, + None, + PartitionMode::CollectLeft, + false, + ) + .unwrap(), + ); + + let optimized_join = JoinSelection::new() + .optimize(join, &ConfigOptions::new()) + .unwrap(); + + let swapping_projection = optimized_join + .as_any() + .downcast_ref::() + .expect("A proj is required to swap columns back to their original order"); + + assert_eq!(swapping_projection.expr().len(), 2); + let (col, name) = &swapping_projection.expr()[0]; + assert_eq!(name, "big_col"); + assert_col_expr(col, "big_col", 1); + let (col, name) = &swapping_projection.expr()[1]; + assert_eq!(name, "small_col"); + assert_col_expr(col, "small_col", 0); + + let swapped_join = swapping_projection + .input() + .as_any() + .downcast_ref::() + .expect("The type of the plan should not be changed"); + + assert_eq!( + swapped_join.left().statistics().unwrap().total_byte_size, + Precision::Inexact(8192) + ); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, + Precision::Inexact(2097152) + ); +} + +#[tokio::test] +async fn test_left_join_no_swap() { + let (big, small) = create_big_and_small(); + + let join = Arc::new( + HashJoinExec::try_new( + Arc::clone(&small), + Arc::clone(&big), + vec![( + Arc::new(Column::new_with_schema("small_col", &small.schema()).unwrap()), + Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()), + )], + None, + &JoinType::Left, + None, + PartitionMode::CollectLeft, + false, + ) + .unwrap(), + ); + + let optimized_join = JoinSelection::new() + .optimize(join, &ConfigOptions::new()) + .unwrap(); + + let swapped_join = optimized_join + .as_any() + .downcast_ref::() + .expect("The type of the plan should not be changed"); + + assert_eq!( + swapped_join.left().statistics().unwrap().total_byte_size, + Precision::Inexact(8192) + ); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, + Precision::Inexact(2097152) + ); +} + +#[tokio::test] +async fn test_join_with_swap_semi() { + let join_types = [JoinType::LeftSemi, JoinType::LeftAnti]; + for join_type in join_types { + let (big, small) = create_big_and_small(); + + let join = HashJoinExec::try_new( + Arc::clone(&big), + Arc::clone(&small), + vec![( + Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()), + Arc::new(Column::new_with_schema("small_col", &small.schema()).unwrap()), + )], + None, + &join_type, + None, + PartitionMode::Partitioned, + false, + ) + .unwrap(); + + let original_schema = join.schema(); + + let optimized_join = JoinSelection::new() + .optimize(Arc::new(join), &ConfigOptions::new()) + .unwrap(); + + let swapped_join = optimized_join + .as_any() + .downcast_ref::() + .expect( + "A proj is not required to swap columns back to their original order", + ); + + assert_eq!(swapped_join.schema().fields().len(), 1); + assert_eq!( + swapped_join.left().statistics().unwrap().total_byte_size, + Precision::Inexact(8192) + ); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, + Precision::Inexact(2097152) + ); + assert_eq!(original_schema, swapped_join.schema()); + } +} + +/// Compare the input plan with the plan after running the probe order optimizer. +macro_rules! assert_optimized { + ($EXPECTED_LINES: expr, $PLAN: expr) => { + let expected_lines = $EXPECTED_LINES.iter().map(|s| *s).collect::>(); + + let plan = Arc::new($PLAN); + let optimized = JoinSelection::new() + .optimize(plan.clone(), &ConfigOptions::new()) + .unwrap(); + + let plan_string = displayable(optimized.as_ref()).indent(true).to_string(); + let actual_lines = plan_string.split("\n").collect::>(); + + assert_eq!( + &expected_lines, &actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + }; +} + +#[tokio::test] +async fn test_nested_join_swap() { + let (big, medium, small) = create_nested_with_min_max(); + + // Form the inner join: big JOIN small + let child_join = HashJoinExec::try_new( + Arc::clone(&big), + Arc::clone(&small), + vec![( + col("big_col", &big.schema()).unwrap(), + col("small_col", &small.schema()).unwrap(), + )], + None, + &JoinType::Inner, + None, + PartitionMode::CollectLeft, + false, + ) + .unwrap(); + let child_schema = child_join.schema(); + + // Form join tree `medium LEFT JOIN (big JOIN small)` + let join = HashJoinExec::try_new( + Arc::clone(&medium), + Arc::new(child_join), + vec![( + col("medium_col", &medium.schema()).unwrap(), + col("small_col", &child_schema).unwrap(), + )], + None, + &JoinType::Left, + None, + PartitionMode::CollectLeft, + false, + ) + .unwrap(); + + // Hash join uses the left side to build the hash table, and right side to probe it. We want + // to keep left as small as possible, so if we can estimate (with a reasonable margin of error) + // that the left side is smaller than the right side, we should swap the sides. + // + // The first hash join's left is 'small' table (with 1000 rows), and the second hash join's + // left is the F(small IJ big) which has an estimated cardinality of 2000 rows (vs medium which + // has an exact cardinality of 10_000 rows). + let expected = [ + "ProjectionExec: expr=[medium_col@2 as medium_col, big_col@0 as big_col, small_col@1 as small_col]", + " HashJoinExec: mode=CollectLeft, join_type=Right, on=[(small_col@1, medium_col@0)]", + " ProjectionExec: expr=[big_col@1 as big_col, small_col@0 as small_col]", + " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(small_col@0, big_col@0)]", + " StatisticsExec: col_count=1, row_count=Inexact(1000)", + " StatisticsExec: col_count=1, row_count=Inexact(100000)", + " StatisticsExec: col_count=1, row_count=Inexact(10000)", + "", + ]; + assert_optimized!(expected, join); +} + +#[tokio::test] +async fn test_join_no_swap() { + let (big, small) = create_big_and_small(); + let join = Arc::new( + HashJoinExec::try_new( + Arc::clone(&small), + Arc::clone(&big), + vec![( + Arc::new(Column::new_with_schema("small_col", &small.schema()).unwrap()), + Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()), + )], + None, + &JoinType::Inner, + None, + PartitionMode::CollectLeft, + false, + ) + .unwrap(), + ); + + let optimized_join = JoinSelection::new() + .optimize(join, &ConfigOptions::new()) + .unwrap(); + + let swapped_join = optimized_join + .as_any() + .downcast_ref::() + .expect("The type of the plan should not be changed"); + + assert_eq!( + swapped_join.left().statistics().unwrap().total_byte_size, + Precision::Inexact(8192) + ); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, + Precision::Inexact(2097152) + ); +} + +#[rstest( + join_type, + case::inner(JoinType::Inner), + case::left(JoinType::Left), + case::right(JoinType::Right), + case::full(JoinType::Full) +)] +#[tokio::test] +async fn test_nl_join_with_swap(join_type: JoinType) { + let (big, small) = create_big_and_small(); + + let join = Arc::new( + NestedLoopJoinExec::try_new( + Arc::clone(&big), + Arc::clone(&small), + nl_join_filter(), + &join_type, + None, + ) + .unwrap(), + ); + + let optimized_join = JoinSelection::new() + .optimize(join, &ConfigOptions::new()) + .unwrap(); + + let swapping_projection = optimized_join + .as_any() + .downcast_ref::() + .expect("A proj is required to swap columns back to their original order"); + + assert_eq!(swapping_projection.expr().len(), 2); + let (col, name) = &swapping_projection.expr()[0]; + assert_eq!(name, "big_col"); + assert_col_expr(col, "big_col", 1); + let (col, name) = &swapping_projection.expr()[1]; + assert_eq!(name, "small_col"); + assert_col_expr(col, "small_col", 0); + + let swapped_join = swapping_projection + .input() + .as_any() + .downcast_ref::() + .expect("The type of the plan should not be changed"); + + // Assert join side of big_col swapped in filter expression + let swapped_filter = swapped_join.filter().unwrap(); + let swapped_big_col_idx = swapped_filter.schema().index_of("big_col").unwrap(); + let swapped_big_col_side = swapped_filter + .column_indices() + .get(swapped_big_col_idx) + .unwrap() + .side; + assert_eq!( + swapped_big_col_side, + JoinSide::Right, + "Filter column side should be swapped" + ); + + assert_eq!( + swapped_join.left().statistics().unwrap().total_byte_size, + Precision::Inexact(8192) + ); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, + Precision::Inexact(2097152) + ); +} + +#[rstest( + join_type, + case::left_semi(JoinType::LeftSemi), + case::left_anti(JoinType::LeftAnti), + case::right_semi(JoinType::RightSemi), + case::right_anti(JoinType::RightAnti) +)] +#[tokio::test] +async fn test_nl_join_with_swap_no_proj(join_type: JoinType) { + let (big, small) = create_big_and_small(); + + let join = Arc::new( + NestedLoopJoinExec::try_new( + Arc::clone(&big), + Arc::clone(&small), + nl_join_filter(), + &join_type, + None, + ) + .unwrap(), + ); + + let optimized_join = JoinSelection::new() + .optimize( + Arc::::clone(&join), + &ConfigOptions::new(), + ) + .unwrap(); + + let swapped_join = optimized_join + .as_any() + .downcast_ref::() + .expect("The type of the plan should not be changed"); + + // Assert before/after schemas are equal + assert_eq!( + join.schema(), + swapped_join.schema(), + "Join schema should not be modified while optimization" + ); + + // Assert join side of big_col swapped in filter expression + let swapped_filter = swapped_join.filter().unwrap(); + let swapped_big_col_idx = swapped_filter.schema().index_of("big_col").unwrap(); + let swapped_big_col_side = swapped_filter + .column_indices() + .get(swapped_big_col_idx) + .unwrap() + .side; + assert_eq!( + swapped_big_col_side, + JoinSide::Right, + "Filter column side should be swapped" + ); + + assert_eq!( + swapped_join.left().statistics().unwrap().total_byte_size, + Precision::Inexact(8192) + ); + assert_eq!( + swapped_join.right().statistics().unwrap().total_byte_size, + Precision::Inexact(2097152) + ); +} + +#[rstest( + join_type, projection, small_on_right, + case::inner(JoinType::Inner, vec![1], true), + case::left(JoinType::Left, vec![1], true), + case::right(JoinType::Right, vec![1], true), + case::full(JoinType::Full, vec![1], true), + case::left_anti(JoinType::LeftAnti, vec![0], false), + case::left_semi(JoinType::LeftSemi, vec![0], false), + case::right_anti(JoinType::RightAnti, vec![0], true), + case::right_semi(JoinType::RightSemi, vec![0], true), + )] +#[tokio::test] +async fn test_hash_join_swap_on_joins_with_projections( + join_type: JoinType, + projection: Vec, + small_on_right: bool, +) -> Result<()> { + let (big, small) = create_big_and_small(); + + let left = if small_on_right { &big } else { &small }; + let right = if small_on_right { &small } else { &big }; + + let left_on = if small_on_right { + "big_col" + } else { + "small_col" + }; + let right_on = if small_on_right { + "small_col" + } else { + "big_col" + }; + + let join = Arc::new(HashJoinExec::try_new( + Arc::clone(left), + Arc::clone(right), + vec![( + Arc::new(Column::new_with_schema(left_on, &left.schema())?), + Arc::new(Column::new_with_schema(right_on, &right.schema())?), + )], + None, + &join_type, + Some(projection), + PartitionMode::Partitioned, + false, + )?); + + let swapped = join + .swap_inputs(PartitionMode::Partitioned) + .expect("swap_hash_join must support joins with projections"); + let swapped_join = swapped.as_any().downcast_ref::().expect( + "ProjectionExec won't be added above if HashJoinExec contains embedded projection", + ); + + assert_eq!(swapped_join.projection, Some(vec![0_usize])); + assert_eq!(swapped.schema().fields.len(), 1); + assert_eq!(swapped.schema().fields[0].name(), "small_col"); + Ok(()) +} + +fn assert_col_expr(expr: &Arc, name: &str, index: usize) { + let col = expr + .as_any() + .downcast_ref::() + .expect("Projection items should be Column expression"); + assert_eq!(col.name(), name); + assert_eq!(col.index(), index); +} + +#[tokio::test] +async fn test_join_selection_collect_left() { + let big = Arc::new(StatisticsExec::new( + big_statistics(), + Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), + )); + + let small = Arc::new(StatisticsExec::new( + small_statistics(), + Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), + )); + + let empty = Arc::new(StatisticsExec::new( + empty_statistics(), + Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), + )); + + let join_on = vec![( + col("small_col", &small.schema()).unwrap(), + col("big_col", &big.schema()).unwrap(), + )]; + check_join_partition_mode( + Arc::::clone(&small), + Arc::::clone(&big), + join_on, + false, + PartitionMode::CollectLeft, + ); + + let join_on = vec![( + col("big_col", &big.schema()).unwrap(), + col("small_col", &small.schema()).unwrap(), + )]; + check_join_partition_mode( + big, + Arc::::clone(&small), + join_on, + true, + PartitionMode::CollectLeft, + ); + + let join_on = vec![( + col("small_col", &small.schema()).unwrap(), + col("empty_col", &empty.schema()).unwrap(), + )]; + check_join_partition_mode( + Arc::::clone(&small), + Arc::::clone(&empty), + join_on, + false, + PartitionMode::CollectLeft, + ); + + let join_on = vec![( + col("empty_col", &empty.schema()).unwrap(), + col("small_col", &small.schema()).unwrap(), + )]; + check_join_partition_mode(empty, small, join_on, true, PartitionMode::CollectLeft); +} + +#[tokio::test] +async fn test_join_selection_partitioned() { + let bigger = Arc::new(StatisticsExec::new( + bigger_statistics(), + Schema::new(vec![Field::new("bigger_col", DataType::Int32, false)]), + )); + + let big = Arc::new(StatisticsExec::new( + big_statistics(), + Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), + )); + + let empty = Arc::new(StatisticsExec::new( + empty_statistics(), + Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), + )); + + let join_on = vec![( + Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("bigger_col", &bigger.schema()).unwrap()) as _, + )]; + check_join_partition_mode( + Arc::::clone(&big), + Arc::::clone(&bigger), + join_on, + false, + PartitionMode::Partitioned, + ); + + let join_on = vec![( + Arc::new(Column::new_with_schema("bigger_col", &bigger.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()) as _, + )]; + check_join_partition_mode( + bigger, + Arc::::clone(&big), + join_on, + true, + PartitionMode::Partitioned, + ); + + let join_on = vec![( + Arc::new(Column::new_with_schema("empty_col", &empty.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()) as _, + )]; + check_join_partition_mode( + Arc::::clone(&empty), + Arc::::clone(&big), + join_on, + false, + PartitionMode::Partitioned, + ); + + let join_on = vec![( + Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("empty_col", &empty.schema()).unwrap()) as _, + )]; + check_join_partition_mode(big, empty, join_on, false, PartitionMode::Partitioned); +} + +fn check_join_partition_mode( + left: Arc, + right: Arc, + on: Vec<(PhysicalExprRef, PhysicalExprRef)>, + is_swapped: bool, + expected_mode: PartitionMode, +) { + let join = Arc::new( + HashJoinExec::try_new( + left, + right, + on, + None, + &JoinType::Inner, + None, + PartitionMode::Auto, + false, + ) + .unwrap(), + ); + + let optimized_join = JoinSelection::new() + .optimize(join, &ConfigOptions::new()) + .unwrap(); + + if !is_swapped { + let swapped_join = optimized_join + .as_any() + .downcast_ref::() + .expect("The type of the plan should not be changed"); + assert_eq!(*swapped_join.partition_mode(), expected_mode); + } else { + let swapping_projection = optimized_join + .as_any() + .downcast_ref::() + .expect("A proj is required to swap columns back to their original order"); + let swapped_join = swapping_projection + .input() + .as_any() + .downcast_ref::() + .expect("The type of the plan should not be changed"); + + assert_eq!(*swapped_join.partition_mode(), expected_mode); + } +} + +#[derive(Debug)] +struct UnboundedStream { + batch_produce: Option, + count: usize, + batch: RecordBatch, +} + +impl Stream for UnboundedStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + _cx: &mut Context<'_>, + ) -> Poll> { + if let Some(val) = self.batch_produce { + if val <= self.count { + return Poll::Ready(None); + } + } + self.count += 1; + Poll::Ready(Some(Ok(self.batch.clone()))) + } +} + +impl RecordBatchStream for UnboundedStream { + fn schema(&self) -> SchemaRef { + self.batch.schema() + } +} + +/// A mock execution plan that simply returns the provided data source characteristic +#[derive(Debug, Clone)] +pub struct UnboundedExec { + batch_produce: Option, + batch: RecordBatch, + cache: PlanProperties, +} + +impl UnboundedExec { + /// Create new exec that clones the given record batch to its output. + /// + /// Set `batch_produce` to `Some(n)` to emit exactly `n` batches per partition. + pub fn new( + batch_produce: Option, + batch: RecordBatch, + partitions: usize, + ) -> Self { + let cache = Self::compute_properties(batch.schema(), batch_produce, partitions); + Self { + batch_produce, + batch, + cache, + } + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + batch_produce: Option, + n_partitions: usize, + ) -> PlanProperties { + let boundedness = if batch_produce.is_none() { + Boundedness::Unbounded { + requires_infinite_memory: false, + } + } else { + Boundedness::Bounded + }; + PlanProperties::new( + EquivalenceProperties::new(schema), + Partitioning::UnknownPartitioning(n_partitions), + EmissionType::Incremental, + boundedness, + ) + } +} + +impl DisplayAs for UnboundedExec { + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!( + f, + "UnboundedExec: unbounded={}", + self.batch_produce.is_none(), + ) + } + } + } +} + +impl ExecutionPlan for UnboundedExec { + fn name(&self) -> &'static str { + Self::static_name() + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn children(&self) -> Vec<&Arc> { + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> Result { + Ok(Box::pin(UnboundedStream { + batch_produce: self.batch_produce, + count: 0, + batch: self.batch.clone(), + })) + } +} + +#[derive(Eq, PartialEq, Debug)] +pub enum SourceType { + Unbounded, + Bounded, +} + +/// A mock execution plan that simply returns the provided statistics +#[derive(Debug, Clone)] +pub struct StatisticsExec { + stats: Statistics, + schema: Arc, + cache: PlanProperties, +} + +impl StatisticsExec { + pub fn new(stats: Statistics, schema: Schema) -> Self { + assert_eq!( + stats.column_statistics.len(), schema.fields().len(), + "if defined, the column statistics vector length should be the number of fields" + ); + let cache = Self::compute_properties(Arc::new(schema.clone())); + Self { + stats, + schema: Arc::new(schema), + cache, + } + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(schema: SchemaRef) -> PlanProperties { + PlanProperties::new( + EquivalenceProperties::new(schema), + Partitioning::UnknownPartitioning(2), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } +} + +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 name(&self) -> &'static str { + Self::static_name() + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn children(&self) -> Vec<&Arc> { + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> Result { + unimplemented!("This plan only serves for testing statistics") + } + + fn statistics(&self) -> Result { + Ok(self.stats.clone()) + } +} + +#[test] +fn check_expr_supported() { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Utf8, false), + ])); + let supported_expr = Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )) as Arc; + assert!(check_support(&supported_expr, &schema)); + let supported_expr_2 = Arc::new(Column::new("a", 0)) as Arc; + assert!(check_support(&supported_expr_2, &schema)); + let unsupported_expr = Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Or, + Arc::new(Column::new("a", 0)), + )) as Arc; + assert!(!check_support(&unsupported_expr, &schema)); + let unsupported_expr_2 = Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Or, + Arc::new(NegativeExpr::new(Arc::new(Column::new("a", 0)))), + )) as Arc; + assert!(!check_support(&unsupported_expr_2, &schema)); +} + +struct TestCase { + case: String, + initial_sources_unbounded: (SourceType, SourceType), + initial_join_type: JoinType, + initial_mode: PartitionMode, + expected_sources_unbounded: (SourceType, SourceType), + expected_join_type: JoinType, + expected_mode: PartitionMode, + expecting_swap: bool, +} + +#[tokio::test] +async fn test_join_with_swap_full() -> Result<()> { + // NOTE: Currently, some initial conditions are not viable after join order selection. + // For example, full join always comes in partitioned mode. See the warning in + // function "swap". If this changes in the future, we should update these tests. + let cases = vec![ + TestCase { + case: "Bounded - Unbounded 1".to_string(), + initial_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + initial_join_type: JoinType::Full, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + expected_join_type: JoinType::Full, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }, + TestCase { + case: "Unbounded - Bounded 2".to_string(), + initial_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), + initial_join_type: JoinType::Full, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), + expected_join_type: JoinType::Full, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }, + TestCase { + case: "Bounded - Bounded 3".to_string(), + initial_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), + initial_join_type: JoinType::Full, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), + expected_join_type: JoinType::Full, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }, + TestCase { + case: "Unbounded - Unbounded 4".to_string(), + initial_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), + initial_join_type: JoinType::Full, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), + expected_join_type: JoinType::Full, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }, + ]; + for case in cases.into_iter() { + test_join_with_maybe_swap_unbounded_case(case).await? + } + Ok(()) +} + +#[tokio::test] +async fn test_cases_without_collect_left_check() -> Result<()> { + let mut cases = vec![]; + let join_types = vec![JoinType::LeftSemi, JoinType::Inner]; + for join_type in join_types { + cases.push(TestCase { + case: "Unbounded - Bounded / CollectLeft".to_string(), + initial_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), + initial_join_type: join_type, + initial_mode: PartitionMode::CollectLeft, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + expected_join_type: join_type.swap(), + expected_mode: PartitionMode::CollectLeft, + expecting_swap: true, + }); + cases.push(TestCase { + case: "Bounded - Unbounded / CollectLeft".to_string(), + initial_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + initial_join_type: join_type, + initial_mode: PartitionMode::CollectLeft, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + expected_join_type: join_type, + expected_mode: PartitionMode::CollectLeft, + expecting_swap: false, + }); + cases.push(TestCase { + case: "Unbounded - Unbounded / CollectLeft".to_string(), + initial_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), + initial_join_type: join_type, + initial_mode: PartitionMode::CollectLeft, + expected_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), + expected_join_type: join_type, + expected_mode: PartitionMode::CollectLeft, + expecting_swap: false, + }); + cases.push(TestCase { + case: "Bounded - Bounded / CollectLeft".to_string(), + initial_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), + initial_join_type: join_type, + initial_mode: PartitionMode::CollectLeft, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), + expected_join_type: join_type, + expected_mode: PartitionMode::CollectLeft, + expecting_swap: false, + }); + cases.push(TestCase { + case: "Unbounded - Bounded / Partitioned".to_string(), + initial_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), + initial_join_type: join_type, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + expected_join_type: join_type.swap(), + expected_mode: PartitionMode::Partitioned, + expecting_swap: true, + }); + cases.push(TestCase { + case: "Bounded - Unbounded / Partitioned".to_string(), + initial_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + initial_join_type: join_type, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + expected_join_type: join_type, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }); + cases.push(TestCase { + case: "Bounded - Bounded / Partitioned".to_string(), + initial_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), + initial_join_type: join_type, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), + expected_join_type: join_type, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }); + cases.push(TestCase { + case: "Unbounded - Unbounded / Partitioned".to_string(), + initial_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), + initial_join_type: join_type, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), + expected_join_type: join_type, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }); + } + + for case in cases.into_iter() { + test_join_with_maybe_swap_unbounded_case(case).await? + } + Ok(()) +} + +#[tokio::test] +async fn test_not_support_collect_left() -> Result<()> { + let mut cases = vec![]; + // After [JoinSelection] optimization, these join types cannot run in CollectLeft mode except + // [JoinType::LeftSemi] + let the_ones_not_support_collect_left = vec![JoinType::Left, JoinType::LeftAnti]; + for join_type in the_ones_not_support_collect_left { + cases.push(TestCase { + case: "Unbounded - Bounded".to_string(), + initial_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), + initial_join_type: join_type, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + expected_join_type: join_type.swap(), + expected_mode: PartitionMode::Partitioned, + expecting_swap: true, + }); + cases.push(TestCase { + case: "Bounded - Unbounded".to_string(), + initial_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + initial_join_type: join_type, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + expected_join_type: join_type, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }); + cases.push(TestCase { + case: "Bounded - Bounded".to_string(), + initial_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), + initial_join_type: join_type, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), + expected_join_type: join_type, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }); + cases.push(TestCase { + case: "Unbounded - Unbounded".to_string(), + initial_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), + initial_join_type: join_type, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), + expected_join_type: join_type, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }); + } + + for case in cases.into_iter() { + test_join_with_maybe_swap_unbounded_case(case).await? + } + Ok(()) +} + +#[tokio::test] +async fn test_not_supporting_swaps_possible_collect_left() -> Result<()> { + let mut cases = vec![]; + let the_ones_not_support_collect_left = + vec![JoinType::Right, JoinType::RightAnti, JoinType::RightSemi]; + for join_type in the_ones_not_support_collect_left { + // We expect that (SourceType::Unbounded, SourceType::Bounded) will change, regardless of the + // statistics. + cases.push(TestCase { + case: "Unbounded - Bounded / CollectLeft".to_string(), + initial_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), + initial_join_type: join_type, + initial_mode: PartitionMode::CollectLeft, + expected_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), + expected_join_type: join_type, + expected_mode: PartitionMode::CollectLeft, + expecting_swap: false, + }); + // We expect that (SourceType::Bounded, SourceType::Unbounded) will stay same, regardless of the + // statistics. + cases.push(TestCase { + case: "Bounded - Unbounded / CollectLeft".to_string(), + initial_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + initial_join_type: join_type, + initial_mode: PartitionMode::CollectLeft, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + expected_join_type: join_type, + expected_mode: PartitionMode::CollectLeft, + expecting_swap: false, + }); + cases.push(TestCase { + case: "Unbounded - Unbounded / CollectLeft".to_string(), + initial_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), + initial_join_type: join_type, + initial_mode: PartitionMode::CollectLeft, + expected_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), + expected_join_type: join_type, + expected_mode: PartitionMode::CollectLeft, + expecting_swap: false, + }); + // + cases.push(TestCase { + case: "Bounded - Bounded / CollectLeft".to_string(), + initial_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), + initial_join_type: join_type, + initial_mode: PartitionMode::CollectLeft, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), + expected_join_type: join_type, + expected_mode: PartitionMode::CollectLeft, + expecting_swap: false, + }); + // If cases are partitioned, only unbounded & bounded check will affect the order. + cases.push(TestCase { + case: "Unbounded - Bounded / Partitioned".to_string(), + initial_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), + initial_join_type: join_type, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), + expected_join_type: join_type, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }); + cases.push(TestCase { + case: "Bounded - Unbounded / Partitioned".to_string(), + initial_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + initial_join_type: join_type, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), + expected_join_type: join_type, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }); + cases.push(TestCase { + case: "Bounded - Bounded / Partitioned".to_string(), + initial_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), + initial_join_type: join_type, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), + expected_join_type: join_type, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }); + cases.push(TestCase { + case: "Unbounded - Unbounded / Partitioned".to_string(), + initial_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), + initial_join_type: join_type, + initial_mode: PartitionMode::Partitioned, + expected_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), + expected_join_type: join_type, + expected_mode: PartitionMode::Partitioned, + expecting_swap: false, + }); + } + + for case in cases.into_iter() { + test_join_with_maybe_swap_unbounded_case(case).await? + } + Ok(()) +} + +async fn test_join_with_maybe_swap_unbounded_case(t: TestCase) -> Result<()> { + let left_unbounded = t.initial_sources_unbounded.0 == SourceType::Unbounded; + let right_unbounded = t.initial_sources_unbounded.1 == SourceType::Unbounded; + let left_exec = Arc::new(UnboundedExec::new( + (!left_unbounded).then_some(1), + RecordBatch::new_empty(Arc::new(Schema::new(vec![Field::new( + "a", + DataType::Int32, + false, + )]))), + 2, + )) as _; + let right_exec = Arc::new(UnboundedExec::new( + (!right_unbounded).then_some(1), + RecordBatch::new_empty(Arc::new(Schema::new(vec![Field::new( + "b", + DataType::Int32, + false, + )]))), + 2, + )) as _; + + let join = Arc::new(HashJoinExec::try_new( + Arc::clone(&left_exec), + Arc::clone(&right_exec), + vec![( + col("a", &left_exec.schema())?, + col("b", &right_exec.schema())?, + )], + None, + &t.initial_join_type, + None, + t.initial_mode, + false, + )?) as _; + + let optimized_join_plan = hash_join_swap_subrule(join, &ConfigOptions::new())?; + + // If swap did happen + let projection_added = optimized_join_plan.as_any().is::(); + let plan = if projection_added { + let proj = optimized_join_plan + .as_any() + .downcast_ref::() + .expect("A proj is required to swap columns back to their original order"); + Arc::::clone(proj.input()) + } else { + optimized_join_plan + }; + + if let Some(HashJoinExec { + left, + right, + join_type, + mode, + .. + }) = plan.as_any().downcast_ref::() + { + let left_changed = Arc::ptr_eq(left, &right_exec); + let right_changed = Arc::ptr_eq(right, &left_exec); + // If this is not equal, we have a bigger problem. + assert_eq!(left_changed, right_changed); + assert_eq!( + ( + t.case.as_str(), + if left.boundedness().is_unbounded() { + SourceType::Unbounded + } else { + SourceType::Bounded + }, + if right.boundedness().is_unbounded() { + SourceType::Unbounded + } else { + SourceType::Bounded + }, + join_type, + mode, + left_changed && right_changed + ), + ( + t.case.as_str(), + t.expected_sources_unbounded.0, + t.expected_sources_unbounded.1, + &t.expected_join_type, + &t.expected_mode, + t.expecting_swap + ) + ); + }; + Ok(()) +} diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs new file mode 100644 index 000000000000..49490b2a3d48 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -0,0 +1,490 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use arrow::compute::SortOptions; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion_common::config::ConfigOptions; +use datafusion_common::error::Result; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_expr::Operator; +use datafusion_physical_expr::expressions::BinaryExpr; +use datafusion_physical_expr::expressions::{col, lit}; +use datafusion_physical_expr::{Partitioning, PhysicalSortExpr}; +use datafusion_physical_optimizer::limit_pushdown::LimitPushdown; +use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::empty::EmptyExec; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; +use datafusion_physical_plan::{get_plan_string, ExecutionPlan, ExecutionPlanProperties}; + +fn create_schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("c1", DataType::Int32, true), + Field::new("c2", DataType::Int32, true), + Field::new("c3", DataType::Int32, true), + ])) +} + +fn streaming_table_exec(schema: SchemaRef) -> Result> { + Ok(Arc::new(StreamingTableExec::try_new( + Arc::clone(&schema), + vec![Arc::new(DummyStreamPartition { schema }) as _], + None, + None, + true, + None, + )?)) +} + +fn global_limit_exec( + input: Arc, + skip: usize, + fetch: Option, +) -> Arc { + Arc::new(GlobalLimitExec::new(input, skip, fetch)) +} + +fn local_limit_exec( + input: Arc, + fetch: usize, +) -> Arc { + Arc::new(LocalLimitExec::new(input, fetch)) +} + +fn sort_exec( + sort_exprs: impl IntoIterator, + input: Arc, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + Arc::new(SortExec::new(sort_exprs, input)) +} + +fn sort_preserving_merge_exec( + sort_exprs: impl IntoIterator, + input: Arc, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + Arc::new(SortPreservingMergeExec::new(sort_exprs, input)) +} + +fn projection_exec( + schema: SchemaRef, + input: Arc, +) -> Result> { + Ok(Arc::new(ProjectionExec::try_new( + vec![ + (col("c1", schema.as_ref()).unwrap(), "c1".to_string()), + (col("c2", schema.as_ref()).unwrap(), "c2".to_string()), + (col("c3", schema.as_ref()).unwrap(), "c3".to_string()), + ], + input, + )?)) +} + +fn filter_exec( + schema: SchemaRef, + input: Arc, +) -> Result> { + Ok(Arc::new(FilterExec::try_new( + Arc::new(BinaryExpr::new( + col("c3", schema.as_ref()).unwrap(), + Operator::Gt, + lit(0), + )), + input, + )?)) +} + +fn coalesce_batches_exec(input: Arc) -> Arc { + Arc::new(CoalesceBatchesExec::new(input, 8192)) +} + +fn coalesce_partitions_exec( + local_limit: Arc, +) -> Arc { + Arc::new(CoalescePartitionsExec::new(local_limit)) +} + +fn repartition_exec( + streaming_table: Arc, +) -> Result> { + Ok(Arc::new(RepartitionExec::try_new( + streaming_table, + Partitioning::RoundRobinBatch(8), + )?)) +} + +fn empty_exec(schema: SchemaRef) -> Arc { + Arc::new(EmptyExec::new(schema)) +} + +#[derive(Debug)] +struct DummyStreamPartition { + schema: SchemaRef, +} +impl PartitionStream for DummyStreamPartition { + fn schema(&self) -> &SchemaRef { + &self.schema + } + fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { + unreachable!() + } +} + +#[test] +fn transforms_streaming_table_exec_into_fetching_version_when_skip_is_zero() -> Result<()> +{ + let schema = create_schema(); + let streaming_table = streaming_table_exec(schema)?; + let global_limit = global_limit_exec(streaming_table, 0, Some(5)); + + let initial = get_plan_string(&global_limit); + let expected_initial = [ + "GlobalLimitExec: skip=0, fetch=5", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; + + let expected = [ + "StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true, fetch=5" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn transforms_streaming_table_exec_into_fetching_version_and_keeps_the_global_limit_when_skip_is_nonzero( +) -> Result<()> { + let schema = create_schema(); + let streaming_table = streaming_table_exec(schema)?; + let global_limit = global_limit_exec(streaming_table, 2, Some(5)); + + let initial = get_plan_string(&global_limit); + let expected_initial = [ + "GlobalLimitExec: skip=2, fetch=5", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; + + let expected = [ + "GlobalLimitExec: skip=2, fetch=5", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true, fetch=7" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn transforms_coalesce_batches_exec_into_fetching_version_and_removes_local_limit( +) -> Result<()> { + let schema = create_schema(); + let streaming_table = streaming_table_exec(Arc::clone(&schema))?; + let repartition = repartition_exec(streaming_table)?; + let filter = filter_exec(schema, repartition)?; + let coalesce_batches = coalesce_batches_exec(filter); + let local_limit = local_limit_exec(coalesce_batches, 5); + let coalesce_partitions = coalesce_partitions_exec(local_limit); + let global_limit = global_limit_exec(coalesce_partitions, 0, Some(5)); + + let initial = get_plan_string(&global_limit); + let expected_initial = [ + "GlobalLimitExec: skip=0, fetch=5", + " CoalescePartitionsExec", + " LocalLimitExec: fetch=5", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c3@2 > 0", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; + + let expected = [ + "GlobalLimitExec: skip=0, fetch=5", + " CoalescePartitionsExec", + " CoalesceBatchesExec: target_batch_size=8192, fetch=5", + " FilterExec: c3@2 > 0", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn pushes_global_limit_exec_through_projection_exec() -> Result<()> { + let schema = create_schema(); + let streaming_table = streaming_table_exec(Arc::clone(&schema))?; + let filter = filter_exec(Arc::clone(&schema), streaming_table)?; + let projection = projection_exec(schema, filter)?; + let global_limit = global_limit_exec(projection, 0, Some(5)); + + let initial = get_plan_string(&global_limit); + let expected_initial = [ + "GlobalLimitExec: skip=0, fetch=5", + " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", + " FilterExec: c3@2 > 0", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; + + let expected = [ + "ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", + " GlobalLimitExec: skip=0, fetch=5", + " FilterExec: c3@2 > 0", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn pushes_global_limit_exec_through_projection_exec_and_transforms_coalesce_batches_exec_into_fetching_version( +) -> Result<()> { + let schema = create_schema(); + let streaming_table = streaming_table_exec(Arc::clone(&schema)).unwrap(); + let coalesce_batches = coalesce_batches_exec(streaming_table); + let projection = projection_exec(schema, coalesce_batches)?; + let global_limit = global_limit_exec(projection, 0, Some(5)); + + let initial = get_plan_string(&global_limit); + let expected_initial = [ + "GlobalLimitExec: skip=0, fetch=5", + " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", + " CoalesceBatchesExec: target_batch_size=8192", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + ]; + + assert_eq!(initial, expected_initial); + + let after_optimize = + LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; + + let expected = [ + "ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", + " CoalesceBatchesExec: target_batch_size=8192, fetch=5", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn pushes_global_limit_into_multiple_fetch_plans() -> Result<()> { + let schema = create_schema(); + let streaming_table = streaming_table_exec(Arc::clone(&schema)).unwrap(); + let coalesce_batches = coalesce_batches_exec(streaming_table); + let projection = projection_exec(Arc::clone(&schema), coalesce_batches)?; + let repartition = repartition_exec(projection)?; + let sort = sort_exec( + vec![PhysicalSortExpr { + expr: col("c1", &schema)?, + options: SortOptions::default(), + }], + repartition, + ); + let spm = sort_preserving_merge_exec(sort.output_ordering().unwrap().to_vec(), sort); + let global_limit = global_limit_exec(spm, 0, Some(5)); + + let initial = get_plan_string(&global_limit); + let expected_initial = [ + "GlobalLimitExec: skip=0, fetch=5", + " SortPreservingMergeExec: [c1@0 ASC]", + " SortExec: expr=[c1@0 ASC], preserve_partitioning=[false]", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", + " CoalesceBatchesExec: target_batch_size=8192", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + ]; + + assert_eq!(initial, expected_initial); + + let after_optimize = + LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; + + let expected = [ + "SortPreservingMergeExec: [c1@0 ASC], fetch=5", + " SortExec: TopK(fetch=5), expr=[c1@0 ASC], preserve_partitioning=[false]", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", + " CoalesceBatchesExec: target_batch_size=8192", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn keeps_pushed_local_limit_exec_when_there_are_multiple_input_partitions() -> Result<()> +{ + let schema = create_schema(); + let streaming_table = streaming_table_exec(Arc::clone(&schema))?; + let repartition = repartition_exec(streaming_table)?; + let filter = filter_exec(schema, repartition)?; + let coalesce_partitions = coalesce_partitions_exec(filter); + let global_limit = global_limit_exec(coalesce_partitions, 0, Some(5)); + + let initial = get_plan_string(&global_limit); + let expected_initial = [ + "GlobalLimitExec: skip=0, fetch=5", + " CoalescePartitionsExec", + " FilterExec: c3@2 > 0", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; + + let expected = [ + "GlobalLimitExec: skip=0, fetch=5", + " CoalescePartitionsExec", + " FilterExec: c3@2 > 0", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn merges_local_limit_with_local_limit() -> Result<()> { + let schema = create_schema(); + let empty_exec = empty_exec(schema); + let child_local_limit = local_limit_exec(empty_exec, 10); + let parent_local_limit = local_limit_exec(child_local_limit, 20); + + let initial = get_plan_string(&parent_local_limit); + let expected_initial = [ + "LocalLimitExec: fetch=20", + " LocalLimitExec: fetch=10", + " EmptyExec", + ]; + + assert_eq!(initial, expected_initial); + + let after_optimize = + LimitPushdown::new().optimize(parent_local_limit, &ConfigOptions::new())?; + + let expected = ["GlobalLimitExec: skip=0, fetch=10", " EmptyExec"]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn merges_global_limit_with_global_limit() -> Result<()> { + let schema = create_schema(); + let empty_exec = empty_exec(schema); + let child_global_limit = global_limit_exec(empty_exec, 10, Some(30)); + let parent_global_limit = global_limit_exec(child_global_limit, 10, Some(20)); + + let initial = get_plan_string(&parent_global_limit); + let expected_initial = [ + "GlobalLimitExec: skip=10, fetch=20", + " GlobalLimitExec: skip=10, fetch=30", + " EmptyExec", + ]; + + assert_eq!(initial, expected_initial); + + let after_optimize = + LimitPushdown::new().optimize(parent_global_limit, &ConfigOptions::new())?; + + let expected = ["GlobalLimitExec: skip=20, fetch=20", " EmptyExec"]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn merges_global_limit_with_local_limit() -> Result<()> { + let schema = create_schema(); + let empty_exec = empty_exec(schema); + let local_limit = local_limit_exec(empty_exec, 40); + let global_limit = global_limit_exec(local_limit, 20, Some(30)); + + let initial = get_plan_string(&global_limit); + let expected_initial = [ + "GlobalLimitExec: skip=20, fetch=30", + " LocalLimitExec: fetch=40", + " EmptyExec", + ]; + + assert_eq!(initial, expected_initial); + + let after_optimize = + LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; + + let expected = ["GlobalLimitExec: skip=20, fetch=20", " EmptyExec"]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn merges_local_limit_with_global_limit() -> Result<()> { + let schema = create_schema(); + let empty_exec = empty_exec(schema); + let global_limit = global_limit_exec(empty_exec, 20, Some(30)); + let local_limit = local_limit_exec(global_limit, 20); + + let initial = get_plan_string(&local_limit); + let expected_initial = [ + "LocalLimitExec: fetch=20", + " GlobalLimitExec: skip=20, fetch=30", + " EmptyExec", + ]; + + assert_eq!(initial, expected_initial); + + let after_optimize = + LimitPushdown::new().optimize(local_limit, &ConfigOptions::new())?; + + let expected = ["GlobalLimitExec: skip=20, fetch=20", " EmptyExec"]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index 7c04d1239bc8..565cee47e3b9 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -19,7 +19,10 @@ use std::sync::Arc; -use crate::physical_optimizer::parquet_exec_with_sort; +use crate::physical_optimizer::test_utils::{ + assert_plan_matches_expected, build_group_by, mock_data, parquet_exec_with_sort, + schema, TestAggregate, +}; use arrow::{compute::SortOptions, util::pretty::pretty_format_batches}; use arrow_schema::DataType; @@ -30,9 +33,6 @@ use datafusion_expr::Operator; use datafusion_physical_expr::expressions::cast; use datafusion_physical_expr::{expressions, expressions::col, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_optimizer::test_utils::{ - assert_plan_matches_expected, build_group_by, mock_data, schema, TestAggregate, -}; use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode}, collect, diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 19da0ef7855e..7d5d07715eeb 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -21,36 +21,10 @@ mod aggregate_statistics; mod combine_partial_final_agg; mod enforce_distribution; mod enforce_sorting; +mod join_selection; +mod limit_pushdown; mod limited_distinct_aggregation; +mod projection_pushdown; mod replace_with_order_preserving_variants; mod sanity_checker; - -use std::sync::Arc; - -use arrow_schema::SchemaRef; -use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; -use datafusion_execution::object_store::ObjectStoreUrl; -use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_optimizer::test_utils::schema; - -/// Create a non sorted parquet exec -pub fn parquet_exec(schema: &SchemaRef) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) - .with_file(PartitionedFile::new("x".to_string(), 100)), - ) - .build_arc() -} - -/// Create a single parquet file that is sorted -pub(crate) fn parquet_exec_with_sort( - output_ordering: Vec, -) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering), - ) - .build_arc() -} +mod test_utils; diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs new file mode 100644 index 000000000000..fc576e929591 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -0,0 +1,1403 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::any::Any; +use std::sync::Arc; + +use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; +use datafusion::datasource::file_format::file_compression_type::FileCompressionType; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig}; +use datafusion_common::config::ConfigOptions; +use datafusion_common::Result; +use datafusion_common::{JoinSide, JoinType, ScalarValue}; +use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_expr::{ + ColumnarValue, Operator, ScalarUDF, ScalarUDFImpl, Signature, Volatility, +}; +use datafusion_physical_expr::expressions::{ + binary, col, BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, +}; +use datafusion_physical_expr::ScalarFunctionExpr; +use datafusion_physical_expr::{ + Distribution, Partitioning, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, +}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; +use datafusion_physical_optimizer::projection_pushdown::ProjectionPushdown; +use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::joins::utils::{ColumnIndex, JoinFilter}; +use datafusion_physical_plan::joins::{ + HashJoinExec, NestedLoopJoinExec, PartitionMode, StreamJoinPartitionMode, + SymmetricHashJoinExec, +}; +use datafusion_physical_plan::memory::MemoryExec; +use datafusion_physical_plan::projection::{update_expr, ProjectionExec}; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::streaming::PartitionStream; +use datafusion_physical_plan::streaming::StreamingTableExec; +use datafusion_physical_plan::union::UnionExec; +use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; + +use itertools::Itertools; + +/// Mocked UDF +#[derive(Debug)] +struct DummyUDF { + signature: Signature, +} + +impl DummyUDF { + fn new() -> Self { + Self { + signature: Signature::variadic_any(Volatility::Immutable), + } + } +} + +impl ScalarUDFImpl for DummyUDF { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "dummy_udf" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Int32) + } + + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { + unimplemented!("DummyUDF::invoke") + } +} + +#[test] +fn test_update_matching_exprs() -> Result<()> { + let exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Divide, + Arc::new(Column::new("e", 5)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 3)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(ScalarUDF::new_from_impl(DummyUDF::new())), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Divide, + Arc::new(Column::new("c", 0)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Divide, + Arc::new(Column::new("b", 1)), + )), + ], + DataType::Int32, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d", 2))), + vec![ + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 2)), + Operator::Plus, + Arc::new(Column::new("e", 5)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 5)), + Operator::Plus, + Arc::new(Column::new("d", 2)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Modulo, + Arc::new(Column::new("e", 5)), + ))), + )?), + ]; + let child: Vec<(Arc, String)> = vec![ + (Arc::new(Column::new("c", 2)), "c".to_owned()), + (Arc::new(Column::new("b", 1)), "b".to_owned()), + (Arc::new(Column::new("d", 3)), "d".to_owned()), + (Arc::new(Column::new("a", 0)), "a".to_owned()), + (Arc::new(Column::new("f", 5)), "f".to_owned()), + (Arc::new(Column::new("e", 4)), "e".to_owned()), + ]; + + let expected_exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Divide, + Arc::new(Column::new("e", 4)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 0)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 5)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(ScalarUDF::new_from_impl(DummyUDF::new())), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Divide, + Arc::new(Column::new("c", 2)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Divide, + Arc::new(Column::new("b", 1)), + )), + ], + DataType::Int32, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d", 3))), + vec![ + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 3)), + Operator::Plus, + Arc::new(Column::new("e", 4)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 4)), + Operator::Plus, + Arc::new(Column::new("d", 3)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Modulo, + Arc::new(Column::new("e", 4)), + ))), + )?), + ]; + + for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { + assert!(update_expr(&expr, &child, true)? + .unwrap() + .eq(&expected_expr)); + } + + Ok(()) +} + +#[test] +fn test_update_projected_exprs() -> Result<()> { + let exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Divide, + Arc::new(Column::new("e", 5)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 3)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(ScalarUDF::new_from_impl(DummyUDF::new())), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Divide, + Arc::new(Column::new("c", 0)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Divide, + Arc::new(Column::new("b", 1)), + )), + ], + DataType::Int32, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d", 2))), + vec![ + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 2)), + Operator::Plus, + Arc::new(Column::new("e", 5)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 5)), + Operator::Plus, + Arc::new(Column::new("d", 2)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Modulo, + Arc::new(Column::new("e", 5)), + ))), + )?), + ]; + let projected_exprs: Vec<(Arc, String)> = vec![ + (Arc::new(Column::new("a", 3)), "a".to_owned()), + (Arc::new(Column::new("b", 1)), "b_new".to_owned()), + (Arc::new(Column::new("c", 0)), "c".to_owned()), + (Arc::new(Column::new("d", 2)), "d_new".to_owned()), + (Arc::new(Column::new("e", 5)), "e".to_owned()), + (Arc::new(Column::new("f", 4)), "f_new".to_owned()), + ]; + + let expected_exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Divide, + Arc::new(Column::new("e", 4)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 0)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f_new", 5)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(ScalarUDF::new_from_impl(DummyUDF::new())), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b_new", 1)), + Operator::Divide, + Arc::new(Column::new("c", 2)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Divide, + Arc::new(Column::new("b_new", 1)), + )), + ], + DataType::Int32, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d_new", 3))), + vec![ + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d_new", 3)), + Operator::Plus, + Arc::new(Column::new("e", 4)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 4)), + Operator::Plus, + Arc::new(Column::new("d_new", 3)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Modulo, + Arc::new(Column::new("e", 4)), + ))), + )?), + ]; + + for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { + assert!(update_expr(&expr, &projected_exprs, false)? + .unwrap() + .eq(&expected_expr)); + } + + Ok(()) +} + +fn create_simple_csv_exec() -> Arc { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])); + Arc::new( + CsvExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_projection(Some(vec![0, 1, 2, 3, 4])), + ) + .with_has_header(false) + .with_delimeter(0) + .with_quote(0) + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), + ) +} + +fn create_projecting_csv_exec() -> Arc { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + ])); + Arc::new( + CsvExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_projection(Some(vec![3, 2, 1])), + ) + .with_has_header(false) + .with_delimeter(0) + .with_quote(0) + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), + ) +} + +fn create_projecting_memory_exec() -> Arc { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])); + + Arc::new(MemoryExec::try_new(&[], schema, Some(vec![2, 0, 3, 4])).unwrap()) +} + +#[test] +fn test_csv_after_projection() -> Result<()> { + let csv = create_projecting_csv_exec(); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("b", 2)), "b".to_string()), + (Arc::new(Column::new("d", 0)), "d".to_string()), + ], + csv.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[b@2 as b, d@0 as d]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[d, c, b], has_header=false", + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = + ["CsvExec: file_groups={1 group: [[x]]}, projection=[b, d], has_header=false"]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn test_memory_after_projection() -> Result<()> { + let memory = create_projecting_memory_exec(); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("d", 2)), "d".to_string()), + (Arc::new(Column::new("e", 3)), "e".to_string()), + (Arc::new(Column::new("a", 1)), "a".to_string()), + ], + memory.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[d@2 as d, e@3 as e, a@1 as a]", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = ["MemoryExec: partitions=0, partition_sizes=[]"]; + assert_eq!(get_plan_string(&after_optimize), expected); + assert_eq!( + after_optimize + .clone() + .as_any() + .downcast_ref::() + .unwrap() + .projection() + .clone() + .unwrap(), + vec![3, 4, 0] + ); + + Ok(()) +} + +#[test] +fn test_streaming_table_after_projection() -> Result<()> { + #[derive(Debug)] + struct DummyStreamPartition { + schema: SchemaRef, + } + impl PartitionStream for DummyStreamPartition { + fn schema(&self) -> &SchemaRef { + &self.schema + } + fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { + unreachable!() + } + } + + let streaming_table = StreamingTableExec::try_new( + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])), + vec![Arc::new(DummyStreamPartition { + schema: Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])), + }) as _], + Some(&vec![0_usize, 2, 4, 3]), + vec![ + LexOrdering::new(vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("e", 2)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: SortOptions::default(), + }, + ]), + LexOrdering::new(vec![PhysicalSortExpr { + expr: Arc::new(Column::new("d", 3)), + options: SortOptions::default(), + }]), + ] + .into_iter(), + true, + None, + )?; + let projection = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("d", 3)), "d".to_string()), + (Arc::new(Column::new("e", 2)), "e".to_string()), + (Arc::new(Column::new("a", 0)), "a".to_string()), + ], + Arc::new(streaming_table) as _, + )?) as _; + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let result = after_optimize + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!( + result.partition_schema(), + &Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])) + ); + assert_eq!( + result.projection().clone().unwrap().to_vec(), + vec![3_usize, 4, 0] + ); + assert_eq!( + result.projected_schema(), + &Schema::new(vec![ + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + Field::new("a", DataType::Int32, true), + ]) + ); + assert_eq!( + result.projected_output_ordering().into_iter().collect_vec(), + vec![ + LexOrdering::new(vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("e", 1)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 2)), + options: SortOptions::default(), + }, + ]), + LexOrdering::new(vec![PhysicalSortExpr { + expr: Arc::new(Column::new("d", 0)), + options: SortOptions::default(), + }]), + ] + ); + assert!(result.is_infinite()); + + Ok(()) +} + +#[test] +fn test_projection_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let child_projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("e", 4)), "new_e".to_string()), + (Arc::new(Column::new("a", 0)), "a".to_string()), + (Arc::new(Column::new("b", 1)), "new_b".to_string()), + ], + csv.clone(), + )?); + let top_projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("new_b", 3)), "new_b".to_string()), + ( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Plus, + Arc::new(Column::new("new_e", 1)), + )), + "binary".to_string(), + ), + (Arc::new(Column::new("new_b", 3)), "newest_b".to_string()), + ], + child_projection.clone(), + )?); + + let initial = get_plan_string(&top_projection); + let expected_initial = [ + "ProjectionExec: expr=[new_b@3 as new_b, c@0 + new_e@1 as binary, new_b@3 as newest_b]", + " ProjectionExec: expr=[c@2 as c, e@4 as new_e, a@0 as a, b@1 as new_b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(top_projection, &ConfigOptions::new())?; + + let expected = [ + "ProjectionExec: expr=[b@1 as new_b, c@2 + e@4 as binary, b@1 as newest_b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn test_output_req_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let sort_req: Arc = Arc::new(OutputRequirementExec::new( + csv.clone(), + Some(LexRequirement::new(vec![ + PhysicalSortRequirement { + expr: Arc::new(Column::new("b", 1)), + options: Some(SortOptions::default()), + }, + PhysicalSortRequirement { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )), + options: Some(SortOptions::default()), + }, + ])), + Distribution::HashPartitioned(vec![ + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + ]), + )); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + sort_req.clone(), + )?); + + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " OutputRequirementExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected: [&str; 3] = [ + "OutputRequirementExec", + " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + + assert_eq!(get_plan_string(&after_optimize), expected); + let expected_reqs = LexRequirement::new(vec![ + PhysicalSortRequirement { + expr: Arc::new(Column::new("b", 2)), + options: Some(SortOptions::default()), + }, + PhysicalSortRequirement { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Plus, + Arc::new(Column::new("new_a", 1)), + )), + options: Some(SortOptions::default()), + }, + ]); + assert_eq!( + after_optimize + .as_any() + .downcast_ref::() + .unwrap() + .required_input_ordering()[0] + .clone() + .unwrap(), + expected_reqs + ); + let expected_distribution: Vec> = vec![ + Arc::new(Column::new("new_a", 1)), + Arc::new(Column::new("b", 2)), + ]; + if let Distribution::HashPartitioned(vec) = after_optimize + .as_any() + .downcast_ref::() + .unwrap() + .required_input_distribution()[0] + .clone() + { + assert!(vec + .iter() + .zip(expected_distribution) + .all(|(actual, expected)| actual.eq(&expected))); + } else { + panic!("Expected HashPartitioned distribution!"); + }; + + Ok(()) +} + +#[test] +fn test_coalesce_partitions_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let coalesce_partitions: Arc = + Arc::new(CoalescePartitionsExec::new(csv)); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("b", 1)), "b".to_string()), + (Arc::new(Column::new("a", 0)), "a_new".to_string()), + (Arc::new(Column::new("d", 3)), "d".to_string()), + ], + coalesce_partitions, + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", + " CoalescePartitionsExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "CoalescePartitionsExec", + " ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn test_filter_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let predicate = Arc::new(BinaryExpr::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Minus, + Arc::new(Column::new("a", 0)), + )), + Operator::Gt, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 3)), + Operator::Minus, + Arc::new(Column::new("a", 0)), + )), + )); + let filter: Arc = Arc::new(FilterExec::try_new(predicate, csv)?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("a", 0)), "a_new".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + (Arc::new(Column::new("d", 3)), "d".to_string()), + ], + filter.clone(), + )?); + + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", + " FilterExec: b@1 - a@0 > d@3 - a@0", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "FilterExec: b@1 - a_new@0 > d@2 - a_new@0", + " ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn test_join_after_projection() -> Result<()> { + let left_csv = create_simple_csv_exec(); + let right_csv = create_simple_csv_exec(); + + let join: Arc = Arc::new(SymmetricHashJoinExec::try_new( + left_csv, + right_csv, + vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], + // b_left-(1+a_right)<=a_right+c_left + Some(JoinFilter::new( + Arc::new(BinaryExpr::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b_left_inter", 0)), + Operator::Minus, + Arc::new(BinaryExpr::new( + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), + Operator::Plus, + Arc::new(Column::new("a_right_inter", 1)), + )), + )), + Operator::LtEq, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a_right_inter", 1)), + Operator::Plus, + Arc::new(Column::new("c_left_inter", 2)), + )), + )), + vec![ + ColumnIndex { + index: 1, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ColumnIndex { + index: 2, + side: JoinSide::Left, + }, + ], + Arc::new(Schema::new(vec![ + Field::new("b_left_inter", DataType::Int32, true), + Field::new("a_right_inter", DataType::Int32, true), + Field::new("c_left_inter", DataType::Int32, true), + ])), + )), + &JoinType::Inner, + true, + None, + None, + StreamJoinPartitionMode::SinglePartition, + )?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c_from_left".to_string()), + (Arc::new(Column::new("b", 1)), "b_from_left".to_string()), + (Arc::new(Column::new("a", 0)), "a_from_left".to_string()), + (Arc::new(Column::new("a", 5)), "a_from_right".to_string()), + (Arc::new(Column::new("c", 7)), "c_from_right".to_string()), + ], + join, + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, a@5 as a_from_right, c@7 as c_from_right]", + " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b_from_left@1, c_from_right@1)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", + " ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " ProjectionExec: expr=[a@0 as a_from_right, c@2 as c_from_right]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + let expected_filter_col_ind = vec![ + ColumnIndex { + index: 1, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ColumnIndex { + index: 0, + side: JoinSide::Left, + }, + ]; + + assert_eq!( + expected_filter_col_ind, + after_optimize + .as_any() + .downcast_ref::() + .unwrap() + .filter() + .unwrap() + .column_indices() + ); + + Ok(()) +} + +#[test] +fn test_join_after_required_projection() -> Result<()> { + let left_csv = create_simple_csv_exec(); + let right_csv = create_simple_csv_exec(); + + let join: Arc = Arc::new(SymmetricHashJoinExec::try_new( + left_csv, + right_csv, + vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], + // b_left-(1+a_right)<=a_right+c_left + Some(JoinFilter::new( + Arc::new(BinaryExpr::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b_left_inter", 0)), + Operator::Minus, + Arc::new(BinaryExpr::new( + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), + Operator::Plus, + Arc::new(Column::new("a_right_inter", 1)), + )), + )), + Operator::LtEq, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a_right_inter", 1)), + Operator::Plus, + Arc::new(Column::new("c_left_inter", 2)), + )), + )), + vec![ + ColumnIndex { + index: 1, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ColumnIndex { + index: 2, + side: JoinSide::Left, + }, + ], + Arc::new(Schema::new(vec![ + Field::new("b_left_inter", DataType::Int32, true), + Field::new("a_right_inter", DataType::Int32, true), + Field::new("c_left_inter", DataType::Int32, true), + ])), + )), + &JoinType::Inner, + true, + None, + None, + StreamJoinPartitionMode::SinglePartition, + )?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("a", 5)), "a".to_string()), + (Arc::new(Column::new("b", 6)), "b".to_string()), + (Arc::new(Column::new("c", 7)), "c".to_string()), + (Arc::new(Column::new("d", 8)), "d".to_string()), + (Arc::new(Column::new("e", 9)), "e".to_string()), + (Arc::new(Column::new("a", 0)), "a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("d", 3)), "d".to_string()), + (Arc::new(Column::new("e", 4)), "e".to_string()), + ], + join, + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[a@5 as a, b@6 as b, c@7 as c, d@8 as d, e@9 as e, a@0 as a, b@1 as b, c@2 as c, d@3 as d, e@4 as e]", + " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "ProjectionExec: expr=[a@5 as a, b@6 as b, c@7 as c, d@8 as d, e@9 as e, a@0 as a, b@1 as b, c@2 as c, d@3 as d, e@4 as e]", + " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) +} + +#[test] +fn test_nested_loop_join_after_projection() -> Result<()> { + let left_csv = create_simple_csv_exec(); + let right_csv = create_simple_csv_exec(); + + let col_left_a = col("a", &left_csv.schema())?; + let col_right_b = col("b", &right_csv.schema())?; + let col_left_c = col("c", &left_csv.schema())?; + // left_a < right_b + let filter_expr = binary(col_left_a, Operator::Lt, col_right_b, &Schema::empty())?; + let filter_column_indices = vec![ + ColumnIndex { + index: 0, + side: JoinSide::Left, + }, + ColumnIndex { + index: 1, + side: JoinSide::Right, + }, + ColumnIndex { + index: 2, + side: JoinSide::Right, + }, + ]; + let filter_schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + + let join: Arc = Arc::new(NestedLoopJoinExec::try_new( + left_csv, + right_csv, + Some(JoinFilter::new( + filter_expr, + filter_column_indices, + Arc::new(filter_schema), + )), + &JoinType::Inner, + None, + )?); + + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![(col_left_c, "c".to_string())], + Arc::clone(&join), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c]", + " NestedLoopJoinExec: join_type=Inner, filter=a@0 < b@1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + let expected = [ + "NestedLoopJoinExec: join_type=Inner, filter=a@0 < b@1, projection=[c@2]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) +} + +#[test] +fn test_hash_join_after_projection() -> Result<()> { + // sql like + // SELECT t1.c as c_from_left, t1.b as b_from_left, t1.a as a_from_left, t2.c as c_from_right FROM t1 JOIN t2 ON t1.b = t2.c WHERE t1.b - (1 + t2.a) <= t2.a + t1.c + let left_csv = create_simple_csv_exec(); + let right_csv = create_simple_csv_exec(); + + let join: Arc = Arc::new(HashJoinExec::try_new( + left_csv, + right_csv, + vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], + // b_left-(1+a_right)<=a_right+c_left + Some(JoinFilter::new( + Arc::new(BinaryExpr::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b_left_inter", 0)), + Operator::Minus, + Arc::new(BinaryExpr::new( + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), + Operator::Plus, + Arc::new(Column::new("a_right_inter", 1)), + )), + )), + Operator::LtEq, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a_right_inter", 1)), + Operator::Plus, + Arc::new(Column::new("c_left_inter", 2)), + )), + )), + vec![ + ColumnIndex { + index: 1, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ColumnIndex { + index: 2, + side: JoinSide::Left, + }, + ], + Arc::new(Schema::new(vec![ + Field::new("b_left_inter", DataType::Int32, true), + Field::new("a_right_inter", DataType::Int32, true), + Field::new("c_left_inter", DataType::Int32, true), + ])), + )), + &JoinType::Inner, + None, + PartitionMode::Auto, + true, + )?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c_from_left".to_string()), + (Arc::new(Column::new("b", 1)), "b_from_left".to_string()), + (Arc::new(Column::new("a", 0)), "a_from_left".to_string()), + (Arc::new(Column::new("c", 7)), "c_from_right".to_string()), + ], + join.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@7 as c_from_right]", " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + // HashJoinExec only returns result after projection. Because there are some alias columns in the projection, the ProjectionExec is not removed. + let expected = ["ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, c@3 as c_from_right]", " HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[a@0, b@1, c@2, c@7]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false"]; + assert_eq!(get_plan_string(&after_optimize), expected); + + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("a", 0)), "a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("c", 7)), "c".to_string()), + ], + join.clone(), + )?); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + // Comparing to the previous result, this projection don't have alias columns either change the order of output fields. So the ProjectionExec is removed. + let expected = ["HashJoinExec: mode=Auto, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2, projection=[a@0, b@1, c@2, c@7]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false"]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn test_repartition_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let repartition: Arc = Arc::new(RepartitionExec::try_new( + csv, + Partitioning::Hash( + vec![ + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("d", 3)), + ], + 6, + ), + )?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("b", 1)), "b_new".to_string()), + (Arc::new(Column::new("a", 0)), "a".to_string()), + (Arc::new(Column::new("d", 3)), "d_new".to_string()), + ], + repartition, + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", + " RepartitionExec: partitioning=Hash([a@0, b@1, d@3], 6), input_partitions=1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "RepartitionExec: partitioning=Hash([a@1, b_new@0, d_new@2], 6), input_partitions=1", + " ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + assert_eq!( + after_optimize + .as_any() + .downcast_ref::() + .unwrap() + .partitioning() + .clone(), + Partitioning::Hash( + vec![ + Arc::new(Column::new("a", 1)), + Arc::new(Column::new("b_new", 0)), + Arc::new(Column::new("d_new", 2)), + ], + 6, + ), + ); + + Ok(()) +} + +#[test] +fn test_sort_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let sort_req: Arc = Arc::new(SortExec::new( + LexOrdering::new(vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )), + options: SortOptions::default(), + }, + ]), + csv.clone(), + )); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + sort_req.clone(), + )?); + + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " SortExec: expr=[b@1 ASC, c@2 + a@0 ASC], preserve_partitioning=[false]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "SortExec: expr=[b@2 ASC, c@0 + new_a@1 ASC], preserve_partitioning=[false]", + " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn test_sort_preserving_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let sort_req: Arc = Arc::new(SortPreservingMergeExec::new( + LexOrdering::new(vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )), + options: SortOptions::default(), + }, + ]), + csv.clone(), + )); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + sort_req.clone(), + )?); + + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " SortPreservingMergeExec: [b@1 ASC, c@2 + a@0 ASC]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "SortPreservingMergeExec: [b@2 ASC, c@0 + new_a@1 ASC]", + " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} + +#[test] +fn test_union_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let union: Arc = + Arc::new(UnionExec::new(vec![csv.clone(), csv.clone(), csv])); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + union.clone(), + )?); + + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " UnionExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + + let after_optimize = + ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "UnionExec", + " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + + Ok(()) +} diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index 912683083738..52cd5e5754fa 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -17,6 +17,10 @@ use std::sync::Arc; +use crate::physical_optimizer::test_utils::{ + check_integrity, stream_exec_ordered_with_projection, +}; + use datafusion::prelude::SessionContext; use arrow::array::{ArrayRef, Int32Array}; use arrow::compute::SortOptions; @@ -40,7 +44,6 @@ use datafusion_common::Result; use datafusion_expr::{JoinType, Operator}; use datafusion_physical_expr::expressions::{self, col, Column}; use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_optimizer::test_utils::{check_integrity, stream_exec_ordered_with_projection}; use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; use datafusion_common::config::ConfigOptions; diff --git a/datafusion/core/tests/physical_optimizer/sanity_checker.rs b/datafusion/core/tests/physical_optimizer/sanity_checker.rs index 7f723ae67e8e..3057ca819e82 100644 --- a/datafusion/core/tests/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/tests/physical_optimizer/sanity_checker.rs @@ -15,25 +15,27 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use crate::physical_optimizer::test_utils::{ + bounded_window_exec, global_limit_exec, local_limit_exec, memory_exec, + repartition_exec, sort_exec, sort_expr_options, sort_merge_join_exec, +}; + use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; use datafusion::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; use datafusion::prelude::{CsvReadOptions, SessionContext}; -use datafusion_common::{JoinType, Result}; -use std::sync::Arc; - -use async_trait::async_trait; use datafusion_common::config::ConfigOptions; +use datafusion_common::{JoinType, Result}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::Partitioning; use datafusion_physical_optimizer::sanity_checker::SanityCheckPlan; -use datafusion_physical_optimizer::test_utils::{ - bounded_window_exec, global_limit_exec, local_limit_exec, memory_exec, - repartition_exec, sort_exec, sort_expr_options, sort_merge_join_exec, -}; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::{displayable, ExecutionPlan}; +use async_trait::async_trait; + async fn register_current_csv( ctx: &SessionContext, table_name: &str, diff --git a/datafusion/physical-optimizer/src/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs similarity index 94% rename from datafusion/physical-optimizer/src/test_utils.rs rename to datafusion/core/tests/physical_optimizer/test_utils.rs index 56830d3b3f89..6f0a4e46cd9e 100644 --- a/datafusion/physical-optimizer/src/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -17,15 +17,20 @@ //! Test utilities for physical optimizer tests -use crate::limited_distinct_aggregation::LimitedDistinctAggregation; -use crate::PhysicalOptimizerRule; +use std::any::Any; +use std::fmt::Formatter; +use std::sync::Arc; + use arrow::array::Int32Array; use arrow::record_batch::RecordBatch; use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; use datafusion_common::{JoinType, Result}; +use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; use datafusion_functions_aggregate::count::count_udaf; @@ -34,6 +39,8 @@ use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{expressions, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_optimizer::limited_distinct_aggregation::LimitedDistinctAggregation; +use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; @@ -56,9 +63,27 @@ use datafusion_physical_plan::{ displayable, DisplayAs, DisplayFormatType, PlanProperties, }; use datafusion_physical_plan::{InputOrderMode, Partitioning}; -use std::any::Any; -use std::fmt::Formatter; -use std::sync::Arc; + +/// Create a non sorted parquet exec +pub fn parquet_exec(schema: &SchemaRef) -> Arc { + ParquetExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) + .with_file(PartitionedFile::new("x".to_string(), 100)), + ) + .build_arc() +} + +/// Create a single parquet file that is sorted +pub(crate) fn parquet_exec_with_sort( + output_ordering: Vec, +) -> Arc { + ParquetExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(output_ordering), + ) + .build_arc() +} pub fn schema() -> SchemaRef { Arc::new(Schema::new(vec![ @@ -383,7 +408,7 @@ pub fn trim_plan_display(plan: &str) -> Vec<&str> { // construct a stream partition for test purposes #[derive(Debug)] -pub(crate) struct TestStreamPartition { +pub struct TestStreamPartition { pub schema: SchemaRef, } diff --git a/datafusion/functions-aggregate/Cargo.toml b/datafusion/functions-aggregate/Cargo.toml index 81e5233a1516..333f0d9cdd79 100644 --- a/datafusion/functions-aggregate/Cargo.toml +++ b/datafusion/functions-aggregate/Cargo.toml @@ -66,3 +66,7 @@ harness = false [[bench]] name = "sum" harness = false + +[[bench]] +name = "array_agg" +harness = false diff --git a/datafusion/functions-aggregate/benches/array_agg.rs b/datafusion/functions-aggregate/benches/array_agg.rs new file mode 100644 index 000000000000..c4599cdfc9b3 --- /dev/null +++ b/datafusion/functions-aggregate/benches/array_agg.rs @@ -0,0 +1,186 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use arrow::array::{Array, ArrayRef, ArrowPrimitiveType, AsArray, ListArray}; +use arrow::datatypes::Int64Type; +use arrow::util::bench_util::create_primitive_array; +use arrow_schema::Field; +use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use datafusion_expr::Accumulator; +use datafusion_functions_aggregate::array_agg::ArrayAggAccumulator; + +use arrow::util::test_util::seedable_rng; +use arrow_buffer::{NullBufferBuilder, OffsetBuffer}; +use rand::distributions::{Distribution, Standard}; +use rand::Rng; + +fn merge_batch_bench(c: &mut Criterion, name: &str, values: ArrayRef) { + let list_item_data_type = values.as_list::().values().data_type().clone(); + c.bench_function(name, |b| { + b.iter(|| { + #[allow(clippy::unit_arg)] + black_box( + ArrayAggAccumulator::try_new(&list_item_data_type) + .unwrap() + .merge_batch(&[values.clone()]) + .unwrap(), + ) + }) + }); +} + +/// Create List array with the given item data type, null density, null locations and zero length lists density +/// Creates an random (but fixed-seeded) array of a given size and null density +pub fn create_list_array( + size: usize, + null_density: f32, + zero_length_lists_probability: f32, +) -> ListArray +where + T: ArrowPrimitiveType, + Standard: Distribution, +{ + let mut nulls_builder = NullBufferBuilder::new(size); + let mut rng = seedable_rng(); + + let offsets = OffsetBuffer::from_lengths((0..size).map(|_| { + let is_null = rng.gen::() < null_density; + + let mut length = rng.gen_range(1..10); + + if is_null { + nulls_builder.append_null(); + + if rng.gen::() <= zero_length_lists_probability { + length = 0; + } + } else { + nulls_builder.append_non_null(); + } + + length + })); + + let length = *offsets.last().unwrap() as usize; + + let values = create_primitive_array::(length, 0.0); + + let field = Field::new_list_field(T::DATA_TYPE, true); + + ListArray::new( + Arc::new(field), + offsets, + Arc::new(values), + nulls_builder.finish(), + ) +} + +fn array_agg_benchmark(c: &mut Criterion) { + let values = Arc::new(create_list_array::(8192, 0.0, 1.0)) as ArrayRef; + merge_batch_bench(c, "array_agg i64 merge_batch no nulls", values); + + let values = Arc::new(create_list_array::(8192, 1.0, 1.0)) as ArrayRef; + merge_batch_bench( + c, + "array_agg i64 merge_batch all nulls, 100% of nulls point to a zero length array", + values, + ); + + let values = Arc::new(create_list_array::(8192, 1.0, 0.9)) as ArrayRef; + merge_batch_bench( + c, + "array_agg i64 merge_batch all nulls, 90% of nulls point to a zero length array", + values, + ); + + // All nulls point to a 0 length array + + let values = Arc::new(create_list_array::(8192, 0.3, 1.0)) as ArrayRef; + merge_batch_bench( + c, + "array_agg i64 merge_batch 30% nulls, 100% of nulls point to a zero length array", + values, + ); + + let values = Arc::new(create_list_array::(8192, 0.7, 1.0)) as ArrayRef; + merge_batch_bench( + c, + "array_agg i64 merge_batch 70% nulls, 100% of nulls point to a zero length array", + values, + ); + + let values = Arc::new(create_list_array::(8192, 0.3, 0.99)) as ArrayRef; + merge_batch_bench( + c, + "array_agg i64 merge_batch 30% nulls, 99% of nulls point to a zero length array", + values, + ); + + let values = Arc::new(create_list_array::(8192, 0.7, 0.99)) as ArrayRef; + merge_batch_bench( + c, + "array_agg i64 merge_batch 70% nulls, 99% of nulls point to a zero length array", + values, + ); + + let values = Arc::new(create_list_array::(8192, 0.3, 0.9)) as ArrayRef; + merge_batch_bench( + c, + "array_agg i64 merge_batch 30% nulls, 90% of nulls point to a zero length array", + values, + ); + + let values = Arc::new(create_list_array::(8192, 0.7, 0.9)) as ArrayRef; + merge_batch_bench( + c, + "array_agg i64 merge_batch 70% nulls, 90% of nulls point to a zero length array", + values, + ); + + let values = Arc::new(create_list_array::(8192, 0.3, 0.50)) as ArrayRef; + merge_batch_bench( + c, + "array_agg i64 merge_batch 30% nulls, 50% of nulls point to a zero length array", + values, + ); + + let values = Arc::new(create_list_array::(8192, 0.7, 0.50)) as ArrayRef; + merge_batch_bench( + c, + "array_agg i64 merge_batch 70% nulls, 50% of nulls point to a zero length array", + values, + ); + + let values = Arc::new(create_list_array::(8192, 0.3, 0.0)) as ArrayRef; + merge_batch_bench( + c, + "array_agg i64 merge_batch 30% nulls, 0% of nulls point to a zero length array", + values, + ); + + let values = Arc::new(create_list_array::(8192, 0.7, 0.0)) as ArrayRef; + merge_batch_bench( + c, + "array_agg i64 merge_batch 70% nulls, 0% of nulls point to a zero length array", + values, + ); +} + +criterion_group!(benches, array_agg_benchmark); +criterion_main!(benches); diff --git a/datafusion/optimizer/src/lib.rs b/datafusion/optimizer/src/lib.rs index 263770b81fcd..614284e1b477 100644 --- a/datafusion/optimizer/src/lib.rs +++ b/datafusion/optimizer/src/lib.rs @@ -62,7 +62,9 @@ pub mod utils; pub mod test; pub use analyzer::{Analyzer, AnalyzerRule}; -pub use optimizer::{Optimizer, OptimizerConfig, OptimizerContext, OptimizerRule}; +pub use optimizer::{ + ApplyOrder, Optimizer, OptimizerConfig, OptimizerContext, OptimizerRule, +}; #[allow(deprecated)] pub use utils::optimize_children; diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr/src/expressions/cast.rs index 7eda5fb4beaa..8a093e0ae92e 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr/src/expressions/cast.rs @@ -242,6 +242,7 @@ mod tests { }, datatypes::*, }; + use datafusion_common::assert_contains; // runs an end-to-end test of physical type cast // 1. construct a record batch with a column "a" of type A @@ -399,6 +400,45 @@ mod tests { Ok(()) } + #[test] + fn test_cast_decimal_to_decimal_overflow() -> Result<()> { + let array = vec![Some(123456789)]; + + let decimal_array = array + .clone() + .into_iter() + .collect::() + .with_precision_and_scale(10, 3)?; + + let schema = Schema::new(vec![Field::new("a", Decimal128(10, 3), false)]); + let batch = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![Arc::new(decimal_array)], + )?; + let expression = + cast_with_options(col("a", &schema)?, &schema, Decimal128(6, 2), None)?; + let e = expression.evaluate(&batch).unwrap_err(); // panics on OK + assert_contains!( + e.to_string(), + "Arrow error: Invalid argument error: 12345679 is too large to store in a Decimal128 of precision 6. Max is 999999" + ); + + let expression_safe = cast_with_options( + col("a", &schema)?, + &schema, + Decimal128(6, 2), + Some(DEFAULT_SAFE_CAST_OPTIONS), + )?; + let result_safe = expression_safe + .evaluate(&batch)? + .into_array(batch.num_rows()) + .expect("failed to convert to array"); + + assert!(result_safe.is_null(0)); + + Ok(()) + } + #[test] fn test_cast_decimal_to_numeric() -> Result<()> { let array = vec![Some(1), Some(2), Some(3), Some(4), Some(5), None]; diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index a40827bda209..4dc9ac22f173 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -41,7 +41,6 @@ datafusion-common = { workspace = true, default-features = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-expr-common = { workspace = true, default-features = true } -datafusion-functions-aggregate = { workspace = true } datafusion-physical-expr = { workspace = true } datafusion-physical-expr-common = { workspace = true } datafusion-physical-plan = { workspace = true } diff --git a/datafusion/physical-optimizer/src/join_selection.rs b/datafusion/physical-optimizer/src/join_selection.rs index d5f70938a7d4..03bfb6978890 100644 --- a/datafusion/physical-optimizer/src/join_selection.rs +++ b/datafusion/physical-optimizer/src/join_selection.rs @@ -526,7 +526,7 @@ fn hash_join_convert_symmetric_subrule( /// +--------------+ +--------------+ /// /// ``` -fn hash_join_swap_subrule( +pub fn hash_join_swap_subrule( mut input: Arc, _config_options: &ConfigOptions, ) -> Result> { @@ -589,1550 +589,4 @@ fn apply_subrules( Ok(Transformed::yes(input)) } -#[cfg(test)] -mod tests_statistical { - use super::*; - use util_tests::StatisticsExec; - - use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::{ - stats::Precision, ColumnStatistics, JoinType, ScalarValue, Statistics, - }; - use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr::expressions::BinaryExpr; - use datafusion_physical_expr::{PhysicalExpr, PhysicalExprRef}; - use datafusion_physical_plan::displayable; - use datafusion_physical_plan::projection::ProjectionExec; - use rstest::rstest; - - /// Return statistics for empty table - fn empty_statistics() -> Statistics { - Statistics { - num_rows: Precision::Absent, - total_byte_size: Precision::Absent, - column_statistics: vec![ColumnStatistics::new_unknown()], - } - } - - /// Get table thresholds: (num_rows, byte_size) - fn get_thresholds() -> (usize, usize) { - let optimizer_options = ConfigOptions::new().optimizer; - ( - optimizer_options.hash_join_single_partition_threshold_rows, - optimizer_options.hash_join_single_partition_threshold, - ) - } - - /// Return statistics for small table - fn small_statistics() -> Statistics { - let (threshold_num_rows, threshold_byte_size) = get_thresholds(); - Statistics { - num_rows: Precision::Inexact(threshold_num_rows / 128), - total_byte_size: Precision::Inexact(threshold_byte_size / 128), - column_statistics: vec![ColumnStatistics::new_unknown()], - } - } - - /// Return statistics for big table - fn big_statistics() -> Statistics { - let (threshold_num_rows, threshold_byte_size) = get_thresholds(); - Statistics { - num_rows: Precision::Inexact(threshold_num_rows * 2), - total_byte_size: Precision::Inexact(threshold_byte_size * 2), - column_statistics: vec![ColumnStatistics::new_unknown()], - } - } - - /// Return statistics for big table - fn bigger_statistics() -> Statistics { - let (threshold_num_rows, threshold_byte_size) = get_thresholds(); - Statistics { - num_rows: Precision::Inexact(threshold_num_rows * 4), - total_byte_size: Precision::Inexact(threshold_byte_size * 4), - column_statistics: vec![ColumnStatistics::new_unknown()], - } - } - - fn create_big_and_small() -> (Arc, Arc) { - let big = Arc::new(StatisticsExec::new( - big_statistics(), - Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), - )); - - let small = Arc::new(StatisticsExec::new( - small_statistics(), - Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), - )); - (big, small) - } - - /// Create a column statistics vector for a single column - /// that has the given min/max/distinct_count properties. - /// - /// Given min/max will be mapped to a [`ScalarValue`] if - /// they are not `None`. - fn create_column_stats( - min: Option, - max: Option, - distinct_count: Option, - ) -> Vec { - vec![ColumnStatistics { - distinct_count: distinct_count - .map(Precision::Inexact) - .unwrap_or(Precision::Absent), - min_value: min - .map(|size| Precision::Inexact(ScalarValue::UInt64(Some(size)))) - .unwrap_or(Precision::Absent), - max_value: max - .map(|size| Precision::Inexact(ScalarValue::UInt64(Some(size)))) - .unwrap_or(Precision::Absent), - ..Default::default() - }] - } - - /// Create join filter for NLJoinExec with expression `big_col > small_col` - /// where both columns are 0-indexed and come from left and right inputs respectively - fn nl_join_filter() -> Option { - let column_indices = vec![ - ColumnIndex { - index: 0, - side: JoinSide::Left, - }, - ColumnIndex { - index: 0, - side: JoinSide::Right, - }, - ]; - let intermediate_schema = Schema::new(vec![ - Field::new("big_col", DataType::Int32, false), - Field::new("small_col", DataType::Int32, false), - ]); - let expression = Arc::new(BinaryExpr::new( - Arc::new(Column::new_with_schema("big_col", &intermediate_schema).unwrap()), - Operator::Gt, - Arc::new(Column::new_with_schema("small_col", &intermediate_schema).unwrap()), - )) as _; - Some(JoinFilter::new( - expression, - column_indices, - Arc::new(intermediate_schema), - )) - } - - /// Returns three plans with statistics of (min, max, distinct_count) - /// * big 100K rows @ (0, 50k, 50k) - /// * medium 10K rows @ (1k, 5k, 1k) - /// * small 1K rows @ (0, 100k, 1k) - fn create_nested_with_min_max() -> ( - Arc, - Arc, - Arc, - ) { - let big = Arc::new(StatisticsExec::new( - Statistics { - num_rows: Precision::Inexact(100_000), - column_statistics: create_column_stats( - Some(0), - Some(50_000), - Some(50_000), - ), - total_byte_size: Precision::Absent, - }, - Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), - )); - - let medium = Arc::new(StatisticsExec::new( - Statistics { - num_rows: Precision::Inexact(10_000), - column_statistics: create_column_stats( - Some(1000), - Some(5000), - Some(1000), - ), - total_byte_size: Precision::Absent, - }, - Schema::new(vec![Field::new("medium_col", DataType::Int32, false)]), - )); - - let small = Arc::new(StatisticsExec::new( - Statistics { - num_rows: Precision::Inexact(1000), - column_statistics: create_column_stats( - Some(0), - Some(100_000), - Some(1000), - ), - total_byte_size: Precision::Absent, - }, - Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), - )); - - (big, medium, small) - } - - #[tokio::test] - async fn test_join_with_swap() { - let (big, small) = create_big_and_small(); - - let join = Arc::new( - HashJoinExec::try_new( - Arc::clone(&big), - Arc::clone(&small), - vec![( - Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()), - Arc::new( - Column::new_with_schema("small_col", &small.schema()).unwrap(), - ), - )], - None, - &JoinType::Left, - None, - PartitionMode::CollectLeft, - false, - ) - .unwrap(), - ); - - let optimized_join = JoinSelection::new() - .optimize(join, &ConfigOptions::new()) - .unwrap(); - - let swapping_projection = optimized_join - .as_any() - .downcast_ref::() - .expect("A proj is required to swap columns back to their original order"); - - assert_eq!(swapping_projection.expr().len(), 2); - let (col, name) = &swapping_projection.expr()[0]; - assert_eq!(name, "big_col"); - assert_col_expr(col, "big_col", 1); - let (col, name) = &swapping_projection.expr()[1]; - assert_eq!(name, "small_col"); - assert_col_expr(col, "small_col", 0); - - let swapped_join = swapping_projection - .input() - .as_any() - .downcast_ref::() - .expect("The type of the plan should not be changed"); - - assert_eq!( - swapped_join.left().statistics().unwrap().total_byte_size, - Precision::Inexact(8192) - ); - assert_eq!( - swapped_join.right().statistics().unwrap().total_byte_size, - Precision::Inexact(2097152) - ); - } - - #[tokio::test] - async fn test_left_join_no_swap() { - let (big, small) = create_big_and_small(); - - let join = Arc::new( - HashJoinExec::try_new( - Arc::clone(&small), - Arc::clone(&big), - vec![( - Arc::new( - Column::new_with_schema("small_col", &small.schema()).unwrap(), - ), - Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()), - )], - None, - &JoinType::Left, - None, - PartitionMode::CollectLeft, - false, - ) - .unwrap(), - ); - - let optimized_join = JoinSelection::new() - .optimize(join, &ConfigOptions::new()) - .unwrap(); - - let swapped_join = optimized_join - .as_any() - .downcast_ref::() - .expect("The type of the plan should not be changed"); - - assert_eq!( - swapped_join.left().statistics().unwrap().total_byte_size, - Precision::Inexact(8192) - ); - assert_eq!( - swapped_join.right().statistics().unwrap().total_byte_size, - Precision::Inexact(2097152) - ); - } - - #[tokio::test] - async fn test_join_with_swap_semi() { - let join_types = [JoinType::LeftSemi, JoinType::LeftAnti]; - for join_type in join_types { - let (big, small) = create_big_and_small(); - - let join = HashJoinExec::try_new( - Arc::clone(&big), - Arc::clone(&small), - vec![( - Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()), - Arc::new( - Column::new_with_schema("small_col", &small.schema()).unwrap(), - ), - )], - None, - &join_type, - None, - PartitionMode::Partitioned, - false, - ) - .unwrap(); - - let original_schema = join.schema(); - - let optimized_join = JoinSelection::new() - .optimize(Arc::new(join), &ConfigOptions::new()) - .unwrap(); - - let swapped_join = optimized_join - .as_any() - .downcast_ref::() - .expect( - "A proj is not required to swap columns back to their original order", - ); - - assert_eq!(swapped_join.schema().fields().len(), 1); - assert_eq!( - swapped_join.left().statistics().unwrap().total_byte_size, - Precision::Inexact(8192) - ); - assert_eq!( - swapped_join.right().statistics().unwrap().total_byte_size, - Precision::Inexact(2097152) - ); - assert_eq!(original_schema, swapped_join.schema()); - } - } - - /// Compare the input plan with the plan after running the probe order optimizer. - macro_rules! assert_optimized { - ($EXPECTED_LINES: expr, $PLAN: expr) => { - let expected_lines = - $EXPECTED_LINES.iter().map(|s| *s).collect::>(); - - let plan = Arc::new($PLAN); - let optimized = JoinSelection::new() - .optimize(plan.clone(), &ConfigOptions::new()) - .unwrap(); - - let plan_string = displayable(optimized.as_ref()).indent(true).to_string(); - let actual_lines = plan_string.split("\n").collect::>(); - - assert_eq!( - &expected_lines, &actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines - ); - }; - } - - #[tokio::test] - async fn test_nested_join_swap() { - let (big, medium, small) = create_nested_with_min_max(); - - // Form the inner join: big JOIN small - let child_join = HashJoinExec::try_new( - Arc::clone(&big), - Arc::clone(&small), - vec![( - col("big_col", &big.schema()).unwrap(), - col("small_col", &small.schema()).unwrap(), - )], - None, - &JoinType::Inner, - None, - PartitionMode::CollectLeft, - false, - ) - .unwrap(); - let child_schema = child_join.schema(); - - // Form join tree `medium LEFT JOIN (big JOIN small)` - let join = HashJoinExec::try_new( - Arc::clone(&medium), - Arc::new(child_join), - vec![( - col("medium_col", &medium.schema()).unwrap(), - col("small_col", &child_schema).unwrap(), - )], - None, - &JoinType::Left, - None, - PartitionMode::CollectLeft, - false, - ) - .unwrap(); - - // Hash join uses the left side to build the hash table, and right side to probe it. We want - // to keep left as small as possible, so if we can estimate (with a reasonable margin of error) - // that the left side is smaller than the right side, we should swap the sides. - // - // The first hash join's left is 'small' table (with 1000 rows), and the second hash join's - // left is the F(small IJ big) which has an estimated cardinality of 2000 rows (vs medium which - // has an exact cardinality of 10_000 rows). - let expected = [ - "ProjectionExec: expr=[medium_col@2 as medium_col, big_col@0 as big_col, small_col@1 as small_col]", - " HashJoinExec: mode=CollectLeft, join_type=Right, on=[(small_col@1, medium_col@0)]", - " ProjectionExec: expr=[big_col@1 as big_col, small_col@0 as small_col]", - " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(small_col@0, big_col@0)]", - " StatisticsExec: col_count=1, row_count=Inexact(1000)", - " StatisticsExec: col_count=1, row_count=Inexact(100000)", - " StatisticsExec: col_count=1, row_count=Inexact(10000)", - "", - ]; - assert_optimized!(expected, join); - } - - #[tokio::test] - async fn test_join_no_swap() { - let (big, small) = create_big_and_small(); - let join = Arc::new( - HashJoinExec::try_new( - Arc::clone(&small), - Arc::clone(&big), - vec![( - Arc::new( - Column::new_with_schema("small_col", &small.schema()).unwrap(), - ), - Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()), - )], - None, - &JoinType::Inner, - None, - PartitionMode::CollectLeft, - false, - ) - .unwrap(), - ); - - let optimized_join = JoinSelection::new() - .optimize(join, &ConfigOptions::new()) - .unwrap(); - - let swapped_join = optimized_join - .as_any() - .downcast_ref::() - .expect("The type of the plan should not be changed"); - - assert_eq!( - swapped_join.left().statistics().unwrap().total_byte_size, - Precision::Inexact(8192) - ); - assert_eq!( - swapped_join.right().statistics().unwrap().total_byte_size, - Precision::Inexact(2097152) - ); - } - - #[rstest( - join_type, - case::inner(JoinType::Inner), - case::left(JoinType::Left), - case::right(JoinType::Right), - case::full(JoinType::Full) - )] - #[tokio::test] - async fn test_nl_join_with_swap(join_type: JoinType) { - let (big, small) = create_big_and_small(); - - let join = Arc::new( - NestedLoopJoinExec::try_new( - Arc::clone(&big), - Arc::clone(&small), - nl_join_filter(), - &join_type, - None, - ) - .unwrap(), - ); - - let optimized_join = JoinSelection::new() - .optimize(join, &ConfigOptions::new()) - .unwrap(); - - let swapping_projection = optimized_join - .as_any() - .downcast_ref::() - .expect("A proj is required to swap columns back to their original order"); - - assert_eq!(swapping_projection.expr().len(), 2); - let (col, name) = &swapping_projection.expr()[0]; - assert_eq!(name, "big_col"); - assert_col_expr(col, "big_col", 1); - let (col, name) = &swapping_projection.expr()[1]; - assert_eq!(name, "small_col"); - assert_col_expr(col, "small_col", 0); - - let swapped_join = swapping_projection - .input() - .as_any() - .downcast_ref::() - .expect("The type of the plan should not be changed"); - - // Assert join side of big_col swapped in filter expression - let swapped_filter = swapped_join.filter().unwrap(); - let swapped_big_col_idx = swapped_filter.schema().index_of("big_col").unwrap(); - let swapped_big_col_side = swapped_filter - .column_indices() - .get(swapped_big_col_idx) - .unwrap() - .side; - assert_eq!( - swapped_big_col_side, - JoinSide::Right, - "Filter column side should be swapped" - ); - - assert_eq!( - swapped_join.left().statistics().unwrap().total_byte_size, - Precision::Inexact(8192) - ); - assert_eq!( - swapped_join.right().statistics().unwrap().total_byte_size, - Precision::Inexact(2097152) - ); - } - - #[rstest( - join_type, - case::left_semi(JoinType::LeftSemi), - case::left_anti(JoinType::LeftAnti), - case::right_semi(JoinType::RightSemi), - case::right_anti(JoinType::RightAnti) - )] - #[tokio::test] - async fn test_nl_join_with_swap_no_proj(join_type: JoinType) { - let (big, small) = create_big_and_small(); - - let join = Arc::new( - NestedLoopJoinExec::try_new( - Arc::clone(&big), - Arc::clone(&small), - nl_join_filter(), - &join_type, - None, - ) - .unwrap(), - ); - - let optimized_join = JoinSelection::new() - .optimize( - Arc::::clone(&join), - &ConfigOptions::new(), - ) - .unwrap(); - - let swapped_join = optimized_join - .as_any() - .downcast_ref::() - .expect("The type of the plan should not be changed"); - - // Assert before/after schemas are equal - assert_eq!( - join.schema(), - swapped_join.schema(), - "Join schema should not be modified while optimization" - ); - - // Assert join side of big_col swapped in filter expression - let swapped_filter = swapped_join.filter().unwrap(); - let swapped_big_col_idx = swapped_filter.schema().index_of("big_col").unwrap(); - let swapped_big_col_side = swapped_filter - .column_indices() - .get(swapped_big_col_idx) - .unwrap() - .side; - assert_eq!( - swapped_big_col_side, - JoinSide::Right, - "Filter column side should be swapped" - ); - - assert_eq!( - swapped_join.left().statistics().unwrap().total_byte_size, - Precision::Inexact(8192) - ); - assert_eq!( - swapped_join.right().statistics().unwrap().total_byte_size, - Precision::Inexact(2097152) - ); - } - - #[rstest( - join_type, projection, small_on_right, - case::inner(JoinType::Inner, vec![1], true), - case::left(JoinType::Left, vec![1], true), - case::right(JoinType::Right, vec![1], true), - case::full(JoinType::Full, vec![1], true), - case::left_anti(JoinType::LeftAnti, vec![0], false), - case::left_semi(JoinType::LeftSemi, vec![0], false), - case::right_anti(JoinType::RightAnti, vec![0], true), - case::right_semi(JoinType::RightSemi, vec![0], true), - )] - #[tokio::test] - async fn test_hash_join_swap_on_joins_with_projections( - join_type: JoinType, - projection: Vec, - small_on_right: bool, - ) -> Result<()> { - let (big, small) = create_big_and_small(); - - let left = if small_on_right { &big } else { &small }; - let right = if small_on_right { &small } else { &big }; - - let left_on = if small_on_right { - "big_col" - } else { - "small_col" - }; - let right_on = if small_on_right { - "small_col" - } else { - "big_col" - }; - - let join = Arc::new(HashJoinExec::try_new( - Arc::clone(left), - Arc::clone(right), - vec![( - Arc::new(Column::new_with_schema(left_on, &left.schema())?), - Arc::new(Column::new_with_schema(right_on, &right.schema())?), - )], - None, - &join_type, - Some(projection), - PartitionMode::Partitioned, - false, - )?); - - let swapped = join - .swap_inputs(PartitionMode::Partitioned) - .expect("swap_hash_join must support joins with projections"); - let swapped_join = swapped.as_any().downcast_ref::().expect( - "ProjectionExec won't be added above if HashJoinExec contains embedded projection", - ); - - assert_eq!(swapped_join.projection, Some(vec![0_usize])); - assert_eq!(swapped.schema().fields.len(), 1); - assert_eq!(swapped.schema().fields[0].name(), "small_col"); - Ok(()) - } - - fn assert_col_expr(expr: &Arc, name: &str, index: usize) { - let col = expr - .as_any() - .downcast_ref::() - .expect("Projection items should be Column expression"); - assert_eq!(col.name(), name); - assert_eq!(col.index(), index); - } - - #[tokio::test] - async fn test_join_selection_collect_left() { - let big = Arc::new(StatisticsExec::new( - big_statistics(), - Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), - )); - - let small = Arc::new(StatisticsExec::new( - small_statistics(), - Schema::new(vec![Field::new("small_col", DataType::Int32, false)]), - )); - - let empty = Arc::new(StatisticsExec::new( - empty_statistics(), - Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), - )); - - let join_on = vec![( - col("small_col", &small.schema()).unwrap(), - col("big_col", &big.schema()).unwrap(), - )]; - check_join_partition_mode( - Arc::::clone(&small), - Arc::::clone(&big), - join_on, - false, - PartitionMode::CollectLeft, - ); - - let join_on = vec![( - col("big_col", &big.schema()).unwrap(), - col("small_col", &small.schema()).unwrap(), - )]; - check_join_partition_mode( - big, - Arc::::clone(&small), - join_on, - true, - PartitionMode::CollectLeft, - ); - - let join_on = vec![( - col("small_col", &small.schema()).unwrap(), - col("empty_col", &empty.schema()).unwrap(), - )]; - check_join_partition_mode( - Arc::::clone(&small), - Arc::::clone(&empty), - join_on, - false, - PartitionMode::CollectLeft, - ); - - let join_on = vec![( - col("empty_col", &empty.schema()).unwrap(), - col("small_col", &small.schema()).unwrap(), - )]; - check_join_partition_mode( - empty, - small, - join_on, - true, - PartitionMode::CollectLeft, - ); - } - - #[tokio::test] - async fn test_join_selection_partitioned() { - let bigger = Arc::new(StatisticsExec::new( - bigger_statistics(), - Schema::new(vec![Field::new("bigger_col", DataType::Int32, false)]), - )); - - let big = Arc::new(StatisticsExec::new( - big_statistics(), - Schema::new(vec![Field::new("big_col", DataType::Int32, false)]), - )); - - let empty = Arc::new(StatisticsExec::new( - empty_statistics(), - Schema::new(vec![Field::new("empty_col", DataType::Int32, false)]), - )); - - let join_on = vec![( - Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("bigger_col", &bigger.schema()).unwrap()) - as _, - )]; - check_join_partition_mode( - Arc::::clone(&big), - Arc::::clone(&bigger), - join_on, - false, - PartitionMode::Partitioned, - ); - - let join_on = vec![( - Arc::new(Column::new_with_schema("bigger_col", &bigger.schema()).unwrap()) - as _, - Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()) as _, - )]; - check_join_partition_mode( - bigger, - Arc::::clone(&big), - join_on, - true, - PartitionMode::Partitioned, - ); - - let join_on = vec![( - Arc::new(Column::new_with_schema("empty_col", &empty.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()) as _, - )]; - check_join_partition_mode( - Arc::::clone(&empty), - Arc::::clone(&big), - join_on, - false, - PartitionMode::Partitioned, - ); - - let join_on = vec![( - Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("empty_col", &empty.schema()).unwrap()) as _, - )]; - check_join_partition_mode(big, empty, join_on, false, PartitionMode::Partitioned); - } - - fn check_join_partition_mode( - left: Arc, - right: Arc, - on: Vec<(PhysicalExprRef, PhysicalExprRef)>, - is_swapped: bool, - expected_mode: PartitionMode, - ) { - let join = Arc::new( - HashJoinExec::try_new( - left, - right, - on, - None, - &JoinType::Inner, - None, - PartitionMode::Auto, - false, - ) - .unwrap(), - ); - - let optimized_join = JoinSelection::new() - .optimize(join, &ConfigOptions::new()) - .unwrap(); - - if !is_swapped { - let swapped_join = optimized_join - .as_any() - .downcast_ref::() - .expect("The type of the plan should not be changed"); - assert_eq!(*swapped_join.partition_mode(), expected_mode); - } else { - let swapping_projection = optimized_join - .as_any() - .downcast_ref::() - .expect( - "A proj is required to swap columns back to their original order", - ); - let swapped_join = swapping_projection - .input() - .as_any() - .downcast_ref::() - .expect("The type of the plan should not be changed"); - - assert_eq!(*swapped_join.partition_mode(), expected_mode); - } - } -} - -#[cfg(test)] -mod util_tests { - use std::{ - any::Any, - pin::Pin, - sync::Arc, - task::{Context, Poll}, - }; - - use arrow::{ - array::RecordBatch, - datatypes::{DataType, Field, Schema, SchemaRef}, - }; - use datafusion_common::{Result, Statistics}; - use datafusion_execution::{ - RecordBatchStream, SendableRecordBatchStream, TaskContext, - }; - use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::{BinaryExpr, Column, NegativeExpr}; - use datafusion_physical_expr::intervals::utils::check_support; - use datafusion_physical_expr::{EquivalenceProperties, Partitioning, PhysicalExpr}; - use datafusion_physical_plan::{ - execution_plan::{Boundedness, EmissionType}, - DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, - }; - use futures::Stream; - - #[derive(Debug)] - struct UnboundedStream { - batch_produce: Option, - count: usize, - batch: RecordBatch, - } - - impl Stream for UnboundedStream { - type Item = Result; - - fn poll_next( - mut self: Pin<&mut Self>, - _cx: &mut Context<'_>, - ) -> Poll> { - if let Some(val) = self.batch_produce { - if val <= self.count { - return Poll::Ready(None); - } - } - self.count += 1; - Poll::Ready(Some(Ok(self.batch.clone()))) - } - } - - impl RecordBatchStream for UnboundedStream { - fn schema(&self) -> SchemaRef { - self.batch.schema() - } - } - - /// A mock execution plan that simply returns the provided data source characteristic - #[derive(Debug, Clone)] - pub struct UnboundedExec { - batch_produce: Option, - batch: RecordBatch, - cache: PlanProperties, - } - - impl UnboundedExec { - /// Create new exec that clones the given record batch to its output. - /// - /// Set `batch_produce` to `Some(n)` to emit exactly `n` batches per partition. - pub fn new( - batch_produce: Option, - batch: RecordBatch, - partitions: usize, - ) -> Self { - let cache = - Self::compute_properties(batch.schema(), batch_produce, partitions); - Self { - batch_produce, - batch, - cache, - } - } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties( - schema: SchemaRef, - batch_produce: Option, - n_partitions: usize, - ) -> PlanProperties { - let boundedness = if batch_produce.is_none() { - Boundedness::Unbounded { - requires_infinite_memory: false, - } - } else { - Boundedness::Bounded - }; - PlanProperties::new( - EquivalenceProperties::new(schema), - Partitioning::UnknownPartitioning(n_partitions), - EmissionType::Incremental, - boundedness, - ) - } - } - - impl DisplayAs for UnboundedExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!( - f, - "UnboundedExec: unbounded={}", - self.batch_produce.is_none(), - ) - } - } - } - } - - impl ExecutionPlan for UnboundedExec { - fn name(&self) -> &'static str { - Self::static_name() - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn properties(&self) -> &PlanProperties { - &self.cache - } - - fn children(&self) -> Vec<&Arc> { - vec![] - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - Ok(self) - } - - fn execute( - &self, - _partition: usize, - _context: Arc, - ) -> Result { - Ok(Box::pin(UnboundedStream { - batch_produce: self.batch_produce, - count: 0, - batch: self.batch.clone(), - })) - } - } - - #[derive(Eq, PartialEq, Debug)] - pub enum SourceType { - Unbounded, - Bounded, - } - - /// A mock execution plan that simply returns the provided statistics - #[derive(Debug, Clone)] - pub struct StatisticsExec { - stats: Statistics, - schema: Arc, - cache: PlanProperties, - } - - impl StatisticsExec { - pub fn new(stats: Statistics, schema: Schema) -> Self { - assert_eq!( - stats.column_statistics.len(), schema.fields().len(), - "if defined, the column statistics vector length should be the number of fields" - ); - let cache = Self::compute_properties(Arc::new(schema.clone())); - Self { - stats, - schema: Arc::new(schema), - cache, - } - } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties(schema: SchemaRef) -> PlanProperties { - PlanProperties::new( - EquivalenceProperties::new(schema), - Partitioning::UnknownPartitioning(2), - EmissionType::Incremental, - Boundedness::Bounded, - ) - } - } - - 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 name(&self) -> &'static str { - Self::static_name() - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn properties(&self) -> &PlanProperties { - &self.cache - } - - fn children(&self) -> Vec<&Arc> { - vec![] - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - Ok(self) - } - - fn execute( - &self, - _partition: usize, - _context: Arc, - ) -> Result { - unimplemented!("This plan only serves for testing statistics") - } - - fn statistics(&self) -> Result { - Ok(self.stats.clone()) - } - } - - #[test] - fn check_expr_supported() { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, false), - Field::new("b", DataType::Utf8, false), - ])); - let supported_expr = Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )) as Arc; - assert!(check_support(&supported_expr, &schema)); - let supported_expr_2 = Arc::new(Column::new("a", 0)) as Arc; - assert!(check_support(&supported_expr_2, &schema)); - let unsupported_expr = Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Or, - Arc::new(Column::new("a", 0)), - )) as Arc; - assert!(!check_support(&unsupported_expr, &schema)); - let unsupported_expr_2 = Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Or, - Arc::new(NegativeExpr::new(Arc::new(Column::new("a", 0)))), - )) as Arc; - assert!(!check_support(&unsupported_expr_2, &schema)); - } -} - -#[cfg(test)] -mod hash_join_tests { - use super::*; - use util_tests::{SourceType, UnboundedExec}; - - use arrow::datatypes::{DataType, Field, Schema}; - use arrow::record_batch::RecordBatch; - use datafusion_physical_expr::expressions::col; - use datafusion_physical_plan::projection::ProjectionExec; - - struct TestCase { - case: String, - initial_sources_unbounded: (SourceType, SourceType), - initial_join_type: JoinType, - initial_mode: PartitionMode, - expected_sources_unbounded: (SourceType, SourceType), - expected_join_type: JoinType, - expected_mode: PartitionMode, - expecting_swap: bool, - } - - #[tokio::test] - async fn test_join_with_swap_full() -> Result<()> { - // NOTE: Currently, some initial conditions are not viable after join order selection. - // For example, full join always comes in partitioned mode. See the warning in - // function "swap". If this changes in the future, we should update these tests. - let cases = vec![ - TestCase { - case: "Bounded - Unbounded 1".to_string(), - initial_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - initial_join_type: JoinType::Full, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - expected_join_type: JoinType::Full, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }, - TestCase { - case: "Unbounded - Bounded 2".to_string(), - initial_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), - initial_join_type: JoinType::Full, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), - expected_join_type: JoinType::Full, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }, - TestCase { - case: "Bounded - Bounded 3".to_string(), - initial_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), - initial_join_type: JoinType::Full, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), - expected_join_type: JoinType::Full, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }, - TestCase { - case: "Unbounded - Unbounded 4".to_string(), - initial_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), - initial_join_type: JoinType::Full, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: ( - SourceType::Unbounded, - SourceType::Unbounded, - ), - expected_join_type: JoinType::Full, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }, - ]; - for case in cases.into_iter() { - test_join_with_maybe_swap_unbounded_case(case).await? - } - Ok(()) - } - - #[tokio::test] - async fn test_cases_without_collect_left_check() -> Result<()> { - let mut cases = vec![]; - let join_types = vec![JoinType::LeftSemi, JoinType::Inner]; - for join_type in join_types { - cases.push(TestCase { - case: "Unbounded - Bounded / CollectLeft".to_string(), - initial_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), - initial_join_type: join_type, - initial_mode: PartitionMode::CollectLeft, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - expected_join_type: join_type.swap(), - expected_mode: PartitionMode::CollectLeft, - expecting_swap: true, - }); - cases.push(TestCase { - case: "Bounded - Unbounded / CollectLeft".to_string(), - initial_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - initial_join_type: join_type, - initial_mode: PartitionMode::CollectLeft, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - expected_join_type: join_type, - expected_mode: PartitionMode::CollectLeft, - expecting_swap: false, - }); - cases.push(TestCase { - case: "Unbounded - Unbounded / CollectLeft".to_string(), - initial_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), - initial_join_type: join_type, - initial_mode: PartitionMode::CollectLeft, - expected_sources_unbounded: ( - SourceType::Unbounded, - SourceType::Unbounded, - ), - expected_join_type: join_type, - expected_mode: PartitionMode::CollectLeft, - expecting_swap: false, - }); - cases.push(TestCase { - case: "Bounded - Bounded / CollectLeft".to_string(), - initial_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), - initial_join_type: join_type, - initial_mode: PartitionMode::CollectLeft, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), - expected_join_type: join_type, - expected_mode: PartitionMode::CollectLeft, - expecting_swap: false, - }); - cases.push(TestCase { - case: "Unbounded - Bounded / Partitioned".to_string(), - initial_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), - initial_join_type: join_type, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - expected_join_type: join_type.swap(), - expected_mode: PartitionMode::Partitioned, - expecting_swap: true, - }); - cases.push(TestCase { - case: "Bounded - Unbounded / Partitioned".to_string(), - initial_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - initial_join_type: join_type, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - expected_join_type: join_type, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }); - cases.push(TestCase { - case: "Bounded - Bounded / Partitioned".to_string(), - initial_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), - initial_join_type: join_type, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), - expected_join_type: join_type, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }); - cases.push(TestCase { - case: "Unbounded - Unbounded / Partitioned".to_string(), - initial_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), - initial_join_type: join_type, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: ( - SourceType::Unbounded, - SourceType::Unbounded, - ), - expected_join_type: join_type, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }); - } - - for case in cases.into_iter() { - test_join_with_maybe_swap_unbounded_case(case).await? - } - Ok(()) - } - - #[tokio::test] - async fn test_not_support_collect_left() -> Result<()> { - let mut cases = vec![]; - // After [JoinSelection] optimization, these join types cannot run in CollectLeft mode except - // [JoinType::LeftSemi] - let the_ones_not_support_collect_left = vec![JoinType::Left, JoinType::LeftAnti]; - for join_type in the_ones_not_support_collect_left { - cases.push(TestCase { - case: "Unbounded - Bounded".to_string(), - initial_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), - initial_join_type: join_type, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - expected_join_type: join_type.swap(), - expected_mode: PartitionMode::Partitioned, - expecting_swap: true, - }); - cases.push(TestCase { - case: "Bounded - Unbounded".to_string(), - initial_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - initial_join_type: join_type, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - expected_join_type: join_type, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }); - cases.push(TestCase { - case: "Bounded - Bounded".to_string(), - initial_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), - initial_join_type: join_type, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), - expected_join_type: join_type, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }); - cases.push(TestCase { - case: "Unbounded - Unbounded".to_string(), - initial_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), - initial_join_type: join_type, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: ( - SourceType::Unbounded, - SourceType::Unbounded, - ), - expected_join_type: join_type, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }); - } - - for case in cases.into_iter() { - test_join_with_maybe_swap_unbounded_case(case).await? - } - Ok(()) - } - - #[tokio::test] - async fn test_not_supporting_swaps_possible_collect_left() -> Result<()> { - let mut cases = vec![]; - let the_ones_not_support_collect_left = - vec![JoinType::Right, JoinType::RightAnti, JoinType::RightSemi]; - for join_type in the_ones_not_support_collect_left { - // We expect that (SourceType::Unbounded, SourceType::Bounded) will change, regardless of the - // statistics. - cases.push(TestCase { - case: "Unbounded - Bounded / CollectLeft".to_string(), - initial_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), - initial_join_type: join_type, - initial_mode: PartitionMode::CollectLeft, - expected_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), - expected_join_type: join_type, - expected_mode: PartitionMode::CollectLeft, - expecting_swap: false, - }); - // We expect that (SourceType::Bounded, SourceType::Unbounded) will stay same, regardless of the - // statistics. - cases.push(TestCase { - case: "Bounded - Unbounded / CollectLeft".to_string(), - initial_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - initial_join_type: join_type, - initial_mode: PartitionMode::CollectLeft, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - expected_join_type: join_type, - expected_mode: PartitionMode::CollectLeft, - expecting_swap: false, - }); - cases.push(TestCase { - case: "Unbounded - Unbounded / CollectLeft".to_string(), - initial_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), - initial_join_type: join_type, - initial_mode: PartitionMode::CollectLeft, - expected_sources_unbounded: ( - SourceType::Unbounded, - SourceType::Unbounded, - ), - expected_join_type: join_type, - expected_mode: PartitionMode::CollectLeft, - expecting_swap: false, - }); - // - cases.push(TestCase { - case: "Bounded - Bounded / CollectLeft".to_string(), - initial_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), - initial_join_type: join_type, - initial_mode: PartitionMode::CollectLeft, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), - expected_join_type: join_type, - expected_mode: PartitionMode::CollectLeft, - expecting_swap: false, - }); - // If cases are partitioned, only unbounded & bounded check will affect the order. - cases.push(TestCase { - case: "Unbounded - Bounded / Partitioned".to_string(), - initial_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), - initial_join_type: join_type, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: (SourceType::Unbounded, SourceType::Bounded), - expected_join_type: join_type, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }); - cases.push(TestCase { - case: "Bounded - Unbounded / Partitioned".to_string(), - initial_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - initial_join_type: join_type, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Unbounded), - expected_join_type: join_type, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }); - cases.push(TestCase { - case: "Bounded - Bounded / Partitioned".to_string(), - initial_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), - initial_join_type: join_type, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: (SourceType::Bounded, SourceType::Bounded), - expected_join_type: join_type, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }); - cases.push(TestCase { - case: "Unbounded - Unbounded / Partitioned".to_string(), - initial_sources_unbounded: (SourceType::Unbounded, SourceType::Unbounded), - initial_join_type: join_type, - initial_mode: PartitionMode::Partitioned, - expected_sources_unbounded: ( - SourceType::Unbounded, - SourceType::Unbounded, - ), - expected_join_type: join_type, - expected_mode: PartitionMode::Partitioned, - expecting_swap: false, - }); - } - - for case in cases.into_iter() { - test_join_with_maybe_swap_unbounded_case(case).await? - } - Ok(()) - } - - async fn test_join_with_maybe_swap_unbounded_case(t: TestCase) -> Result<()> { - let left_unbounded = t.initial_sources_unbounded.0 == SourceType::Unbounded; - let right_unbounded = t.initial_sources_unbounded.1 == SourceType::Unbounded; - let left_exec = Arc::new(UnboundedExec::new( - (!left_unbounded).then_some(1), - RecordBatch::new_empty(Arc::new(Schema::new(vec![Field::new( - "a", - DataType::Int32, - false, - )]))), - 2, - )) as _; - let right_exec = Arc::new(UnboundedExec::new( - (!right_unbounded).then_some(1), - RecordBatch::new_empty(Arc::new(Schema::new(vec![Field::new( - "b", - DataType::Int32, - false, - )]))), - 2, - )) as _; - - let join = Arc::new(HashJoinExec::try_new( - Arc::clone(&left_exec), - Arc::clone(&right_exec), - vec![( - col("a", &left_exec.schema())?, - col("b", &right_exec.schema())?, - )], - None, - &t.initial_join_type, - None, - t.initial_mode, - false, - )?) as _; - - let optimized_join_plan = hash_join_swap_subrule(join, &ConfigOptions::new())?; - - // If swap did happen - let projection_added = optimized_join_plan.as_any().is::(); - let plan = if projection_added { - let proj = optimized_join_plan - .as_any() - .downcast_ref::() - .expect( - "A proj is required to swap columns back to their original order", - ); - Arc::::clone(proj.input()) - } else { - optimized_join_plan - }; - - if let Some(HashJoinExec { - left, - right, - join_type, - mode, - .. - }) = plan.as_any().downcast_ref::() - { - let left_changed = Arc::ptr_eq(left, &right_exec); - let right_changed = Arc::ptr_eq(right, &left_exec); - // If this is not equal, we have a bigger problem. - assert_eq!(left_changed, right_changed); - assert_eq!( - ( - t.case.as_str(), - if left.boundedness().is_unbounded() { - SourceType::Unbounded - } else { - SourceType::Bounded - }, - if right.boundedness().is_unbounded() { - SourceType::Unbounded - } else { - SourceType::Bounded - }, - join_type, - mode, - left_changed && right_changed - ), - ( - t.case.as_str(), - t.expected_sources_unbounded.0, - t.expected_sources_unbounded.1, - &t.expected_join_type, - &t.expected_mode, - t.expecting_swap - ) - ); - }; - Ok(()) - } -} +// See tests in datafusion/core/tests/physical_optimizer diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index bcb87944f5fd..c2beab032049 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -26,12 +26,13 @@ pub mod enforce_sorting; pub mod join_selection; pub mod limit_pushdown; pub mod limited_distinct_aggregation; -mod optimizer; +pub mod optimizer; pub mod output_requirements; +pub mod projection_pushdown; pub mod pruning; pub mod sanity_checker; pub mod topk_aggregation; pub mod update_aggr_exprs; -pub use optimizer::PhysicalOptimizerRule; -pub mod test_utils; pub mod utils; + +pub use optimizer::PhysicalOptimizerRule; diff --git a/datafusion/physical-optimizer/src/limit_pushdown.rs b/datafusion/physical-optimizer/src/limit_pushdown.rs index 7a44b2e90dde..1c7e4d3d4c3d 100644 --- a/datafusion/physical-optimizer/src/limit_pushdown.rs +++ b/datafusion/physical-optimizer/src/limit_pushdown.rs @@ -22,6 +22,7 @@ use std::fmt::Debug; use std::sync::Arc; use crate::PhysicalOptimizerRule; + use datafusion_common::config::ConfigOptions; use datafusion_common::error::Result; use datafusion_common::tree_node::{Transformed, TreeNodeRecursion}; @@ -339,480 +340,3 @@ fn add_global_limit( } // See tests in datafusion/core/tests/physical_optimizer - -#[cfg(test)] -mod test { - use super::*; - use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use datafusion_common::config::ConfigOptions; - use datafusion_execution::{SendableRecordBatchStream, TaskContext}; - use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::BinaryExpr; - use datafusion_physical_expr::expressions::{col, lit}; - use datafusion_physical_expr::{Partitioning, PhysicalSortExpr}; - use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; - use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; - use datafusion_physical_plan::empty::EmptyExec; - use datafusion_physical_plan::filter::FilterExec; - use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; - use datafusion_physical_plan::projection::ProjectionExec; - use datafusion_physical_plan::repartition::RepartitionExec; - use datafusion_physical_plan::sorts::sort::SortExec; - use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; - use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; - use datafusion_physical_plan::{ - get_plan_string, ExecutionPlan, ExecutionPlanProperties, - }; - use std::sync::Arc; - - #[derive(Debug)] - struct DummyStreamPartition { - schema: SchemaRef, - } - impl PartitionStream for DummyStreamPartition { - fn schema(&self) -> &SchemaRef { - &self.schema - } - fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { - unreachable!() - } - } - - #[test] - fn transforms_streaming_table_exec_into_fetching_version_when_skip_is_zero( - ) -> Result<()> { - let schema = create_schema(); - let streaming_table = streaming_table_exec(schema)?; - let global_limit = global_limit_exec(streaming_table, 0, Some(5)); - - let initial = get_plan_string(&global_limit); - let expected_initial = [ - "GlobalLimitExec: skip=0, fetch=5", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; - - let expected = [ - "StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true, fetch=5" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn transforms_streaming_table_exec_into_fetching_version_and_keeps_the_global_limit_when_skip_is_nonzero( - ) -> Result<()> { - let schema = create_schema(); - let streaming_table = streaming_table_exec(schema)?; - let global_limit = global_limit_exec(streaming_table, 2, Some(5)); - - let initial = get_plan_string(&global_limit); - let expected_initial = [ - "GlobalLimitExec: skip=2, fetch=5", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; - - let expected = [ - "GlobalLimitExec: skip=2, fetch=5", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true, fetch=7" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn transforms_coalesce_batches_exec_into_fetching_version_and_removes_local_limit( - ) -> Result<()> { - let schema = create_schema(); - let streaming_table = streaming_table_exec(Arc::clone(&schema))?; - let repartition = repartition_exec(streaming_table)?; - let filter = filter_exec(schema, repartition)?; - let coalesce_batches = coalesce_batches_exec(filter); - let local_limit = local_limit_exec(coalesce_batches, 5); - let coalesce_partitions = coalesce_partitions_exec(local_limit); - let global_limit = global_limit_exec(coalesce_partitions, 0, Some(5)); - - let initial = get_plan_string(&global_limit); - let expected_initial = [ - "GlobalLimitExec: skip=0, fetch=5", - " CoalescePartitionsExec", - " LocalLimitExec: fetch=5", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c3@2 > 0", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; - - let expected = [ - "GlobalLimitExec: skip=0, fetch=5", - " CoalescePartitionsExec", - " CoalesceBatchesExec: target_batch_size=8192, fetch=5", - " FilterExec: c3@2 > 0", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn pushes_global_limit_exec_through_projection_exec() -> Result<()> { - let schema = create_schema(); - let streaming_table = streaming_table_exec(Arc::clone(&schema))?; - let filter = filter_exec(Arc::clone(&schema), streaming_table)?; - let projection = projection_exec(schema, filter)?; - let global_limit = global_limit_exec(projection, 0, Some(5)); - - let initial = get_plan_string(&global_limit); - let expected_initial = [ - "GlobalLimitExec: skip=0, fetch=5", - " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", - " FilterExec: c3@2 > 0", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; - - let expected = [ - "ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", - " GlobalLimitExec: skip=0, fetch=5", - " FilterExec: c3@2 > 0", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn pushes_global_limit_exec_through_projection_exec_and_transforms_coalesce_batches_exec_into_fetching_version( - ) -> Result<()> { - let schema = create_schema(); - let streaming_table = streaming_table_exec(Arc::clone(&schema)).unwrap(); - let coalesce_batches = coalesce_batches_exec(streaming_table); - let projection = projection_exec(schema, coalesce_batches)?; - let global_limit = global_limit_exec(projection, 0, Some(5)); - - let initial = get_plan_string(&global_limit); - let expected_initial = [ - "GlobalLimitExec: skip=0, fetch=5", - " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", - " CoalesceBatchesExec: target_batch_size=8192", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" - ]; - - assert_eq!(initial, expected_initial); - - let after_optimize = - LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; - - let expected = [ - "ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", - " CoalesceBatchesExec: target_batch_size=8192, fetch=5", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn pushes_global_limit_into_multiple_fetch_plans() -> Result<()> { - let schema = create_schema(); - let streaming_table = streaming_table_exec(Arc::clone(&schema)).unwrap(); - let coalesce_batches = coalesce_batches_exec(streaming_table); - let projection = projection_exec(Arc::clone(&schema), coalesce_batches)?; - let repartition = repartition_exec(projection)?; - let sort = sort_exec( - vec![PhysicalSortExpr { - expr: col("c1", &schema)?, - options: SortOptions::default(), - }], - repartition, - ); - let spm = - sort_preserving_merge_exec(sort.output_ordering().unwrap().to_vec(), sort); - let global_limit = global_limit_exec(spm, 0, Some(5)); - - let initial = get_plan_string(&global_limit); - let expected_initial = [ - "GlobalLimitExec: skip=0, fetch=5", - " SortPreservingMergeExec: [c1@0 ASC]", - " SortExec: expr=[c1@0 ASC], preserve_partitioning=[false]", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", - " CoalesceBatchesExec: target_batch_size=8192", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" - ]; - - assert_eq!(initial, expected_initial); - - let after_optimize = - LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; - - let expected = [ - "SortPreservingMergeExec: [c1@0 ASC], fetch=5", - " SortExec: TopK(fetch=5), expr=[c1@0 ASC], preserve_partitioning=[false]", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", - " CoalesceBatchesExec: target_batch_size=8192", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn keeps_pushed_local_limit_exec_when_there_are_multiple_input_partitions( - ) -> Result<()> { - let schema = create_schema(); - let streaming_table = streaming_table_exec(Arc::clone(&schema))?; - let repartition = repartition_exec(streaming_table)?; - let filter = filter_exec(schema, repartition)?; - let coalesce_partitions = coalesce_partitions_exec(filter); - let global_limit = global_limit_exec(coalesce_partitions, 0, Some(5)); - - let initial = get_plan_string(&global_limit); - let expected_initial = [ - "GlobalLimitExec: skip=0, fetch=5", - " CoalescePartitionsExec", - " FilterExec: c3@2 > 0", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; - - let expected = [ - "GlobalLimitExec: skip=0, fetch=5", - " CoalescePartitionsExec", - " FilterExec: c3@2 > 0", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn merges_local_limit_with_local_limit() -> Result<()> { - let schema = create_schema(); - let empty_exec = empty_exec(schema); - let child_local_limit = local_limit_exec(empty_exec, 10); - let parent_local_limit = local_limit_exec(child_local_limit, 20); - - let initial = get_plan_string(&parent_local_limit); - let expected_initial = [ - "LocalLimitExec: fetch=20", - " LocalLimitExec: fetch=10", - " EmptyExec", - ]; - - assert_eq!(initial, expected_initial); - - let after_optimize = - LimitPushdown::new().optimize(parent_local_limit, &ConfigOptions::new())?; - - let expected = ["GlobalLimitExec: skip=0, fetch=10", " EmptyExec"]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn merges_global_limit_with_global_limit() -> Result<()> { - let schema = create_schema(); - let empty_exec = empty_exec(schema); - let child_global_limit = global_limit_exec(empty_exec, 10, Some(30)); - let parent_global_limit = global_limit_exec(child_global_limit, 10, Some(20)); - - let initial = get_plan_string(&parent_global_limit); - let expected_initial = [ - "GlobalLimitExec: skip=10, fetch=20", - " GlobalLimitExec: skip=10, fetch=30", - " EmptyExec", - ]; - - assert_eq!(initial, expected_initial); - - let after_optimize = - LimitPushdown::new().optimize(parent_global_limit, &ConfigOptions::new())?; - - let expected = ["GlobalLimitExec: skip=20, fetch=20", " EmptyExec"]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn merges_global_limit_with_local_limit() -> Result<()> { - let schema = create_schema(); - let empty_exec = empty_exec(schema); - let local_limit = local_limit_exec(empty_exec, 40); - let global_limit = global_limit_exec(local_limit, 20, Some(30)); - - let initial = get_plan_string(&global_limit); - let expected_initial = [ - "GlobalLimitExec: skip=20, fetch=30", - " LocalLimitExec: fetch=40", - " EmptyExec", - ]; - - assert_eq!(initial, expected_initial); - - let after_optimize = - LimitPushdown::new().optimize(global_limit, &ConfigOptions::new())?; - - let expected = ["GlobalLimitExec: skip=20, fetch=20", " EmptyExec"]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn merges_local_limit_with_global_limit() -> Result<()> { - let schema = create_schema(); - let empty_exec = empty_exec(schema); - let global_limit = global_limit_exec(empty_exec, 20, Some(30)); - let local_limit = local_limit_exec(global_limit, 20); - - let initial = get_plan_string(&local_limit); - let expected_initial = [ - "LocalLimitExec: fetch=20", - " GlobalLimitExec: skip=20, fetch=30", - " EmptyExec", - ]; - - assert_eq!(initial, expected_initial); - - let after_optimize = - LimitPushdown::new().optimize(local_limit, &ConfigOptions::new())?; - - let expected = ["GlobalLimitExec: skip=20, fetch=20", " EmptyExec"]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - fn create_schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("c1", DataType::Int32, true), - Field::new("c2", DataType::Int32, true), - Field::new("c3", DataType::Int32, true), - ])) - } - - fn streaming_table_exec(schema: SchemaRef) -> Result> { - Ok(Arc::new(StreamingTableExec::try_new( - Arc::clone(&schema), - vec![Arc::new(DummyStreamPartition { schema }) as _], - None, - None, - true, - None, - )?)) - } - - fn global_limit_exec( - input: Arc, - skip: usize, - fetch: Option, - ) -> Arc { - Arc::new(GlobalLimitExec::new(input, skip, fetch)) - } - - fn local_limit_exec( - input: Arc, - fetch: usize, - ) -> Arc { - Arc::new(LocalLimitExec::new(input, fetch)) - } - - fn sort_exec( - sort_exprs: impl IntoIterator, - input: Arc, - ) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new(SortExec::new(sort_exprs, input)) - } - - fn sort_preserving_merge_exec( - sort_exprs: impl IntoIterator, - input: Arc, - ) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new(SortPreservingMergeExec::new(sort_exprs, input)) - } - - fn projection_exec( - schema: SchemaRef, - input: Arc, - ) -> Result> { - Ok(Arc::new(ProjectionExec::try_new( - vec![ - (col("c1", schema.as_ref()).unwrap(), "c1".to_string()), - (col("c2", schema.as_ref()).unwrap(), "c2".to_string()), - (col("c3", schema.as_ref()).unwrap(), "c3".to_string()), - ], - input, - )?)) - } - - fn filter_exec( - schema: SchemaRef, - input: Arc, - ) -> Result> { - Ok(Arc::new(FilterExec::try_new( - Arc::new(BinaryExpr::new( - col("c3", schema.as_ref()).unwrap(), - Operator::Gt, - lit(0), - )), - input, - )?)) - } - - fn coalesce_batches_exec(input: Arc) -> Arc { - Arc::new(CoalesceBatchesExec::new(input, 8192)) - } - - fn coalesce_partitions_exec( - local_limit: Arc, - ) -> Arc { - Arc::new(CoalescePartitionsExec::new(local_limit)) - } - - fn repartition_exec( - streaming_table: Arc, - ) -> Result> { - Ok(Arc::new(RepartitionExec::try_new( - streaming_table, - Partitioning::RoundRobinBatch(8), - )?)) - } - - fn empty_exec(schema: SchemaRef) -> Arc { - Arc::new(EmptyExec::new(schema)) - } -} diff --git a/datafusion/physical-optimizer/src/optimizer.rs b/datafusion/physical-optimizer/src/optimizer.rs index 609890e2d43f..88f11f53491e 100644 --- a/datafusion/physical-optimizer/src/optimizer.rs +++ b/datafusion/physical-optimizer/src/optimizer.rs @@ -17,11 +17,26 @@ //! Physical optimizer traits +use std::fmt::Debug; +use std::sync::Arc; + +use crate::aggregate_statistics::AggregateStatistics; +use crate::coalesce_batches::CoalesceBatches; +use crate::combine_partial_final_agg::CombinePartialFinalAggregate; +use crate::enforce_distribution::EnforceDistribution; +use crate::enforce_sorting::EnforceSorting; +use crate::join_selection::JoinSelection; +use crate::limit_pushdown::LimitPushdown; +use crate::limited_distinct_aggregation::LimitedDistinctAggregation; +use crate::output_requirements::OutputRequirements; +use crate::projection_pushdown::ProjectionPushdown; +use crate::sanity_checker::SanityCheckPlan; +use crate::topk_aggregation::TopKAggregation; +use crate::update_aggr_exprs::OptimizeAggregateOrder; + use datafusion_common::config::ConfigOptions; use datafusion_common::Result; use datafusion_physical_plan::ExecutionPlan; -use std::fmt::Debug; -use std::sync::Arc; /// `PhysicalOptimizerRule` transforms one ['ExecutionPlan'] into another which /// computes the same results, but in a potentially more efficient way. @@ -47,3 +62,92 @@ pub trait PhysicalOptimizerRule: Debug { /// and should disable the schema check. fn schema_check(&self) -> bool; } + +/// A rule-based physical optimizer. +#[derive(Clone, Debug)] +pub struct PhysicalOptimizer { + /// All rules to apply + pub rules: Vec>, +} + +impl Default for PhysicalOptimizer { + fn default() -> Self { + Self::new() + } +} + +impl PhysicalOptimizer { + /// Create a new optimizer using the recommended list of rules + pub fn new() -> Self { + let rules: Vec> = vec![ + // If there is a output requirement of the query, make sure that + // this information is not lost across different rules during optimization. + Arc::new(OutputRequirements::new_add_mode()), + Arc::new(AggregateStatistics::new()), + // Statistics-based join selection will change the Auto mode to a real join implementation, + // like collect left, or hash join, or future sort merge join, which will influence the + // EnforceDistribution and EnforceSorting rules as they decide whether to add additional + // repartitioning and local sorting steps to meet distribution and ordering requirements. + // Therefore, it should run before EnforceDistribution and EnforceSorting. + Arc::new(JoinSelection::new()), + // The LimitedDistinctAggregation rule should be applied before the EnforceDistribution rule, + // as that rule may inject other operations in between the different AggregateExecs. + // Applying the rule early means only directly-connected AggregateExecs must be examined. + Arc::new(LimitedDistinctAggregation::new()), + // The EnforceDistribution rule is for adding essential repartitioning to satisfy distribution + // requirements. Please make sure that the whole plan tree is determined before this rule. + // This rule increases parallelism if doing so is beneficial to the physical plan; i.e. at + // least one of the operators in the plan benefits from increased parallelism. + Arc::new(EnforceDistribution::new()), + // The CombinePartialFinalAggregate rule should be applied after the EnforceDistribution rule + Arc::new(CombinePartialFinalAggregate::new()), + // The EnforceSorting rule is for adding essential local sorting to satisfy the required + // ordering. Please make sure that the whole plan tree is determined before this rule. + // Note that one should always run this rule after running the EnforceDistribution rule + // as the latter may break local sorting requirements. + Arc::new(EnforceSorting::new()), + // Run once after the local sorting requirement is changed + Arc::new(OptimizeAggregateOrder::new()), + // TODO: `try_embed_to_hash_join` in the ProjectionPushdown rule would be block by the CoalesceBatches, so add it before CoalesceBatches. Maybe optimize it in the future. + Arc::new(ProjectionPushdown::new()), + // The CoalesceBatches rule will not influence the distribution and ordering of the + // whole plan tree. Therefore, to avoid influencing other rules, it should run last. + Arc::new(CoalesceBatches::new()), + // Remove the ancillary output requirement operator since we are done with the planning + // phase. + Arc::new(OutputRequirements::new_remove_mode()), + // The aggregation limiter will try to find situations where the accumulator count + // is not tied to the cardinality, i.e. when the output of the aggregation is passed + // into an `order by max(x) limit y`. In this case it will copy the limit value down + // to the aggregation, allowing it to use only y number of accumulators. + Arc::new(TopKAggregation::new()), + // The ProjectionPushdown rule tries to push projections towards + // the sources in the execution plan. As a result of this process, + // a projection can disappear if it reaches the source providers, and + // sequential projections can merge into one. Even if these two cases + // are not present, the load of executors such as join or union will be + // reduced by narrowing their input tables. + Arc::new(ProjectionPushdown::new()), + // The LimitPushdown rule tries to push limits down as far as possible, + // replacing operators with fetching variants, or adding limits + // past operators that support limit pushdown. + Arc::new(LimitPushdown::new()), + // The SanityCheckPlan rule checks whether the order and + // distribution requirements of each node in the plan + // is satisfied. It will also reject non-runnable query + // plans that use pipeline-breaking operators on infinite + // input(s). The rule generates a diagnostic error + // message for invalid plans. It makes no changes to the + // given query plan; i.e. it only acts as a final + // gatekeeping rule. + Arc::new(SanityCheckPlan::new()), + ]; + + Self::with_rules(rules) + } + + /// Create a new optimizer with the given rules + pub fn with_rules(rules: Vec>) -> Self { + Self { rules } + } +} diff --git a/datafusion/physical-optimizer/src/projection_pushdown.rs b/datafusion/physical-optimizer/src/projection_pushdown.rs new file mode 100644 index 000000000000..34affcbd4a19 --- /dev/null +++ b/datafusion/physical-optimizer/src/projection_pushdown.rs @@ -0,0 +1,61 @@ +// 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. + +//! This file implements the `ProjectionPushdown` physical optimization rule. +//! The function [`remove_unnecessary_projections`] tries to push down all +//! projections one by one if the operator below is amenable to this. If a +//! projection reaches a source, it can even disappear from the plan entirely. + +use std::sync::Arc; + +use crate::PhysicalOptimizerRule; + +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{TransformedResult, TreeNode}; +use datafusion_common::Result; +use datafusion_physical_plan::projection::remove_unnecessary_projections; +use datafusion_physical_plan::ExecutionPlan; + +/// This rule inspects `ProjectionExec`'s in the given physical plan and tries to +/// remove or swap with its child. +#[derive(Default, Debug)] +pub struct ProjectionPushdown {} + +impl ProjectionPushdown { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +impl PhysicalOptimizerRule for ProjectionPushdown { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + plan.transform_down(remove_unnecessary_projections).data() + } + + fn name(&self) -> &str { + "ProjectionPushdown" + } + + fn schema_check(&self) -> bool { + true + } +} diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes.rs index 8e975e10180f..e75c75a235b7 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes.rs @@ -405,7 +405,7 @@ mod tests { use crate::aggregates::group_values::multi_group_by::bytes::ByteGroupValueBuilder; use arrow_array::{ArrayRef, StringArray}; - use arrow_buffer::{BooleanBufferBuilder, NullBuffer}; + use arrow_buffer::NullBufferBuilder; use datafusion_physical_expr::binary_map::OutputType; use super::GroupColumn; @@ -602,16 +602,15 @@ mod tests { .into_parts(); // explicitly build a boolean buffer where one of the null values also happens to match - let mut boolean_buffer_builder = BooleanBufferBuilder::new(6); - boolean_buffer_builder.append(true); - boolean_buffer_builder.append(false); // this sets Some("bar") to null above - boolean_buffer_builder.append(false); - boolean_buffer_builder.append(false); - boolean_buffer_builder.append(true); - boolean_buffer_builder.append(true); - let nulls = NullBuffer::new(boolean_buffer_builder.finish()); + let mut nulls = NullBufferBuilder::new(6); + nulls.append_non_null(); + nulls.append_null(); // this sets Some("bar") to null above + nulls.append_null(); + nulls.append_null(); + nulls.append_non_null(); + nulls.append_non_null(); let input_array = - Arc::new(StringArray::new(offsets, buffer, Some(nulls))) as ArrayRef; + Arc::new(StringArray::new(offsets, buffer, nulls.finish())) as ArrayRef; // Check let mut equal_to_results = vec![true; builder.len()]; diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs index 811790f4e588..c3d88b894999 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/bytes_view.rs @@ -548,7 +548,7 @@ mod tests { use arrow::array::AsArray; use arrow::datatypes::StringViewType; use arrow_array::{ArrayRef, StringViewArray}; - use arrow_buffer::{BooleanBufferBuilder, NullBuffer}; + use arrow_buffer::NullBufferBuilder; use super::GroupColumn; @@ -751,22 +751,21 @@ mod tests { .into_parts(); // explicitly build a boolean buffer where one of the null values also happens to match - let mut boolean_buffer_builder = BooleanBufferBuilder::new(9); - boolean_buffer_builder.append(true); - boolean_buffer_builder.append(false); // this sets Some("bar") to null above - boolean_buffer_builder.append(false); - boolean_buffer_builder.append(false); - boolean_buffer_builder.append(true); - boolean_buffer_builder.append(true); - boolean_buffer_builder.append(true); - boolean_buffer_builder.append(true); - boolean_buffer_builder.append(true); - boolean_buffer_builder.append(true); - boolean_buffer_builder.append(true); - boolean_buffer_builder.append(true); - let nulls = NullBuffer::new(boolean_buffer_builder.finish()); + let mut nulls = NullBufferBuilder::new(9); + nulls.append_non_null(); + nulls.append_null(); // this sets Some("bar") to null above + nulls.append_null(); + nulls.append_null(); + nulls.append_non_null(); + nulls.append_non_null(); + nulls.append_non_null(); + nulls.append_non_null(); + nulls.append_non_null(); + nulls.append_non_null(); + nulls.append_non_null(); + nulls.append_non_null(); let input_array = - Arc::new(StringViewArray::new(views, buffer, Some(nulls))) as ArrayRef; + Arc::new(StringViewArray::new(views, buffer, nulls.finish())) as ArrayRef; // Check let mut equal_to_results = vec![true; input_array.len()]; diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/primitive.rs index 4ceeb634bad2..cd5dfae86ee9 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/primitive.rs @@ -214,7 +214,7 @@ mod tests { use crate::aggregates::group_values::multi_group_by::primitive::PrimitiveGroupValueBuilder; use arrow::datatypes::Int64Type; use arrow_array::{ArrayRef, Int64Array}; - use arrow_buffer::{BooleanBufferBuilder, NullBuffer}; + use arrow_buffer::NullBufferBuilder; use arrow_schema::DataType; use super::GroupColumn; @@ -304,16 +304,15 @@ mod tests { Int64Array::from(vec![Some(1), Some(2), None, None, Some(1), Some(3)]) .into_parts(); - // explicitly build a boolean buffer where one of the null values also happens to match - let mut boolean_buffer_builder = BooleanBufferBuilder::new(6); - boolean_buffer_builder.append(true); - boolean_buffer_builder.append(false); // this sets Some(2) to null above - boolean_buffer_builder.append(false); - boolean_buffer_builder.append(false); - boolean_buffer_builder.append(true); - boolean_buffer_builder.append(true); - let nulls = NullBuffer::new(boolean_buffer_builder.finish()); - let input_array = Arc::new(Int64Array::new(values, Some(nulls))) as ArrayRef; + // explicitly build a null buffer where one of the null values also happens to match + let mut nulls = NullBufferBuilder::new(6); + nulls.append_non_null(); + nulls.append_null(); // this sets Some(2) to null above + nulls.append_null(); + nulls.append_null(); + nulls.append_non_null(); + nulls.append_non_null(); + let input_array = Arc::new(Int64Array::new(values, nulls.finish())) as ArrayRef; // Check let mut equal_to_results = vec![true; builder.len()]; diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index c1dc41196b36..38fe14ab90b7 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -53,7 +53,7 @@ object_store = { workspace = true } postgres-protocol = { version = "0.6.7", optional = true } postgres-types = { version = "0.2.8", features = ["derive", "with-chrono-0_4"], optional = true } rust_decimal = { version = "1.36.0", features = ["tokio-pg"] } -sqllogictest = "0.26.0" +sqllogictest = "0.26.4" sqlparser = { workspace = true } tempfile = { workspace = true } testcontainers = { version = "0.23", features = ["default"], optional = true } diff --git a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs index 7421edb87b11..09cf70280e7c 100644 --- a/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs +++ b/datafusion/sqllogictest/src/engines/datafusion_engine/normalize.rs @@ -250,7 +250,7 @@ pub fn cell_to_string(col: &ArrayRef, row: usize) -> Result { } /// Converts columns to a result as expected by sqllogicteset. -pub(crate) fn convert_schema_to_types(columns: &Fields) -> Vec { +pub fn convert_schema_to_types(columns: &Fields) -> Vec { columns .iter() .map(|f| f.data_type()) diff --git a/datafusion/sqllogictest/src/engines/mod.rs b/datafusion/sqllogictest/src/engines/mod.rs index 7b65c0aa77cb..3569dea70176 100644 --- a/datafusion/sqllogictest/src/engines/mod.rs +++ b/datafusion/sqllogictest/src/engines/mod.rs @@ -21,6 +21,8 @@ mod datafusion_engine; mod output; pub use datafusion_engine::convert_batches; +pub use datafusion_engine::convert_schema_to_types; +pub use datafusion_engine::DFSqlLogicTestError; pub use datafusion_engine::DataFusion; pub use output::DFColumnType; pub use output::DFOutput; diff --git a/datafusion/sqllogictest/src/lib.rs b/datafusion/sqllogictest/src/lib.rs index 82f194321a8e..0ea55782d34e 100644 --- a/datafusion/sqllogictest/src/lib.rs +++ b/datafusion/sqllogictest/src/lib.rs @@ -20,8 +20,10 @@ mod engines; pub use engines::convert_batches; +pub use engines::convert_schema_to_types; pub use engines::DFColumnType; pub use engines::DFOutput; +pub use engines::DFSqlLogicTestError; pub use engines::DataFusion; #[cfg(feature = "postgres")] diff --git a/datafusion/sqllogictest/test_files/join.slt b/datafusion/sqllogictest/test_files/join.slt.part similarity index 100% rename from datafusion/sqllogictest/test_files/join.slt rename to datafusion/sqllogictest/test_files/join.slt.part diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index cf897d628da5..0bdf223a11b7 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -23,7 +23,7 @@ statement ok set datafusion.optimizer.repartition_joins = false; -include ./join.slt +include ./join.slt.part statement ok CREATE EXTERNAL TABLE annotated_data ( diff --git a/datafusion/sqllogictest/test_files/join_only.slt b/datafusion/sqllogictest/test_files/join_only.slt new file mode 100644 index 000000000000..b2b6a1fa9b9d --- /dev/null +++ b/datafusion/sqllogictest/test_files/join_only.slt @@ -0,0 +1,18 @@ +# 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. + +include ./join.slt.part diff --git a/datafusion/sqllogictest/test_files/operator.slt b/datafusion/sqllogictest/test_files/operator.slt index ac977a0c514a..83bd1d7ee3cd 100644 --- a/datafusion/sqllogictest/test_files/operator.slt +++ b/datafusion/sqllogictest/test_files/operator.slt @@ -33,6 +33,8 @@ SELECT arrow_cast(1.25, 'Decimal128(5, 2)') as decimal ; +############### Addition ############### + # Plus with the same operand type, expect the same output type # except for decimal which is promoted to the highest precision query TTTTTTTTTTT @@ -52,6 +54,43 @@ from numeric_types; ---- Int8 Int16 Int32 Int64 UInt8 UInt16 UInt32 UInt64 Float32 Float64 Decimal128(6, 2) +# Plus with literal integer +query TTTTTTTTTTT +select + arrow_typeof(int8 + 2), + arrow_typeof(int16 + 2), + arrow_typeof(int32 + 2), + arrow_typeof(int64 + 2), + arrow_typeof(uint8 + 2), + arrow_typeof(uint16 + 2), + arrow_typeof(uint32 + 2), + arrow_typeof(uint64 + 2), + arrow_typeof(float32 + 2), + arrow_typeof(float64 + 2), + arrow_typeof(decimal + 2) +from numeric_types; +---- +Int64 Int64 Int64 Int64 Int64 Int64 Int64 Int64 Float32 Float64 Decimal128(23, 2) + +# Plus with literal decimal +query TTTTTTTTTTT +select + arrow_typeof(int8 + 2.0), + arrow_typeof(int16 + 2.0), + arrow_typeof(int32 + 2.0), + arrow_typeof(int64 + 2.0), + arrow_typeof(uint8 + 2.0), + arrow_typeof(uint16 + 2.0), + arrow_typeof(uint32 + 2.0), + arrow_typeof(uint64 + 2.0), + arrow_typeof(float32 + 2.0), + arrow_typeof(float64 + 2.0), + arrow_typeof(decimal + 2.0) +from numeric_types; +---- +Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 + +############### Subtraction ############### # Minus with the same operand type, expect the same output type # except for decimal which is promoted to the highest precision @@ -72,6 +111,44 @@ from numeric_types; ---- Int8 Int16 Int32 Int64 UInt8 UInt16 UInt32 UInt64 Float32 Float64 Decimal128(6, 2) +# Minus with literal integer +query TTTTTTTTTTT +select + arrow_typeof(int8 - 2), + arrow_typeof(int16 - 2), + arrow_typeof(int32 - 2), + arrow_typeof(int64 - 2), + arrow_typeof(uint8 - 2), + arrow_typeof(uint16 - 2), + arrow_typeof(uint32 - 2), + arrow_typeof(uint64 - 2), + arrow_typeof(float32 - 2), + arrow_typeof(float64 - 2), + arrow_typeof(decimal - 2) +from numeric_types; +---- +Int64 Int64 Int64 Int64 Int64 Int64 Int64 Int64 Float32 Float64 Decimal128(23, 2) + +# Minus with literal decimal +query TTTTTTTTTTT +select + arrow_typeof(int8 - 2.0), + arrow_typeof(int16 - 2.0), + arrow_typeof(int32 - 2.0), + arrow_typeof(int64 - 2.0), + arrow_typeof(uint8 - 2.0), + arrow_typeof(uint16 - 2.0), + arrow_typeof(uint32 - 2.0), + arrow_typeof(uint64 - 2.0), + arrow_typeof(float32 - 2.0), + arrow_typeof(float64 - 2.0), + arrow_typeof(decimal - 2.0) +from numeric_types; +---- +Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 + +############### Multiplication ############### + # Multiply with the same operand type, expect the same output type # except for decimal which is promoted to the highest precision query TTTTTTTTTTT @@ -91,6 +168,45 @@ from numeric_types; ---- Int8 Int16 Int32 Int64 UInt8 UInt16 UInt32 UInt64 Float32 Float64 Decimal128(11, 4) +# Multiply with literal integer +query TTTTTTTTTTT +select + arrow_typeof(int8 * 2), + arrow_typeof(int16 * 2), + arrow_typeof(int32 * 2), + arrow_typeof(int64 * 2), + arrow_typeof(uint8 * 2), + arrow_typeof(uint16 * 2), + arrow_typeof(uint32 * 2), + arrow_typeof(uint64 * 2), + arrow_typeof(float32 * 2), + arrow_typeof(float64 * 2), + arrow_typeof(decimal * 2) +from numeric_types; +---- +Int64 Int64 Int64 Int64 Int64 Int64 Int64 Int64 Float32 Float64 Decimal128(26, 2) + +# Multiply with literal decimal +query TTTTTTTTTTT +select + arrow_typeof(int8 * 2.0), + arrow_typeof(int16 * 2.0), + arrow_typeof(int32 * 2.0), + arrow_typeof(int64 * 2.0), + arrow_typeof(uint8 * 2.0), + arrow_typeof(uint16 * 2.0), + arrow_typeof(uint32 * 2.0), + arrow_typeof(uint64 * 2.0), + arrow_typeof(float32 * 2.0), + arrow_typeof(float64 * 2.0), + arrow_typeof(decimal * 2.0) +from numeric_types; +---- +Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 + +############### Division ############### + + # Divide with the same operand type, expect the same output type # except for decimal which is promoted to the highest precision query TTTTTTTTTTT @@ -110,5 +226,139 @@ from numeric_types; ---- Int8 Int16 Int32 Int64 UInt8 UInt16 UInt32 UInt64 Float32 Float64 Decimal128(11, 6) +# Divide with literal integer +query TTTTTTTTTTT +select + arrow_typeof(int8 / 2), + arrow_typeof(int16 / 2), + arrow_typeof(int32 / 2), + arrow_typeof(int64 / 2), + arrow_typeof(uint8 / 2), + arrow_typeof(uint16 / 2), + arrow_typeof(uint32 / 2), + arrow_typeof(uint64 / 2), + arrow_typeof(float32 / 2), + arrow_typeof(float64 / 2), + arrow_typeof(decimal / 2) +from numeric_types; +---- +Int64 Int64 Int64 Int64 Int64 Int64 Int64 Int64 Float32 Float64 Decimal128(9, 6) + +# Divide with literal decimal +query TTTTTTTTTTT +select + arrow_typeof(int8 / 2.0), + arrow_typeof(int16 / 2.0), + arrow_typeof(int32 / 2.0), + arrow_typeof(int64 / 2.0), + arrow_typeof(uint8 / 2.0), + arrow_typeof(uint16 / 2.0), + arrow_typeof(uint32 / 2.0), + arrow_typeof(uint64 / 2.0), + arrow_typeof(float32 / 2.0), + arrow_typeof(float64 / 2.0), + arrow_typeof(decimal / 2.0) +from numeric_types; +---- +Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 Float64 + +############### +# Test for comparison with constants uses efficient types +# Expect the physical plans to compare with constants of the same type +# should have no casts of the column to a different type + +statement ok +set datafusion.explain.physical_plan_only = true; + +############### Less Than ############### + +## < positive integer (expect no casts) +query TT +EXPLAIN SELECT * FROM numeric_types +WHERE int64 < 5 AND uint64 < 5 AND float64 < 5 AND decimal < 5; +---- +physical_plan +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: int64@3 < 5 AND uint64@7 < 5 AND float64@9 < 5 AND decimal@10 < Some(500),5,2 +03)----MemoryExec: partitions=1, partition_sizes=[1] + +## < negative integer (expect no casts) +query TT +EXPLAIN SELECT * FROM numeric_types +WHERE int64 < -5 AND uint64 < -5 AND float64 < -5 AND decimal < -5; +---- +physical_plan +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: int64@3 < -5 AND CAST(uint64@7 AS Decimal128(20, 0)) < Some(-5),20,0 AND float64@9 < -5 AND decimal@10 < Some(-500),5,2 +03)----MemoryExec: partitions=1, partition_sizes=[1] + +## < decimal (expect casts for integers to float) +query TT +EXPLAIN SELECT * FROM numeric_types +WHERE int64 < 5.1 AND uint64 < 5.1 AND float64 < 5.1 AND decimal < 5.1; +---- +physical_plan +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: CAST(int64@3 AS Float64) < 5.1 AND CAST(uint64@7 AS Float64) < 5.1 AND float64@9 < 5.1 AND decimal@10 < Some(510),5,2 +03)----MemoryExec: partitions=1, partition_sizes=[1] + +## < negative decimal (expect casts for integers to float) +query TT +EXPLAIN SELECT * FROM numeric_types +WHERE int64 < -5.1 AND uint64 < -5.1 AND float64 < -5.1 AND decimal < -5.1; +---- +physical_plan +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: CAST(int64@3 AS Float64) < -5.1 AND CAST(uint64@7 AS Float64) < -5.1 AND float64@9 < -5.1 AND decimal@10 < Some(-510),5,2 +03)----MemoryExec: partitions=1, partition_sizes=[1] + + +############### Equality ############### + +## = positive integer (expect no casts) +query TT +EXPLAIN SELECT * FROM numeric_types +WHERE int64 = 5 AND uint64 = 5 AND float64 = 5 AND decimal = 5; +---- +physical_plan +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: int64@3 = 5 AND uint64@7 = 5 AND float64@9 = 5 AND decimal@10 = Some(500),5,2 +03)----MemoryExec: partitions=1, partition_sizes=[1] + +## = negative integer (expect no casts) +query TT +EXPLAIN SELECT * FROM numeric_types +WHERE int64 = -5 AND uint64 = -5 AND float64 = -5 AND decimal = -5; +---- +physical_plan +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: int64@3 = -5 AND CAST(uint64@7 AS Decimal128(20, 0)) = Some(-5),20,0 AND float64@9 = -5 AND decimal@10 = Some(-500),5,2 +03)----MemoryExec: partitions=1, partition_sizes=[1] + +## = decimal (expect casts for integers to float) +query TT +EXPLAIN SELECT * FROM numeric_types +WHERE int64 = 5.1 AND uint64 = 5.1 AND float64 = 5.1 AND decimal = 5.1; +---- +physical_plan +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: CAST(int64@3 AS Float64) = 5.1 AND CAST(uint64@7 AS Float64) = 5.1 AND float64@9 = 5.1 AND decimal@10 = Some(510),5,2 +03)----MemoryExec: partitions=1, partition_sizes=[1] + +## = negative decimal (expect casts for integers to float) +query TT +EXPLAIN SELECT * FROM numeric_types +WHERE int64 = -5.1 AND uint64 = -5.1 AND float64 = -5.1 AND decimal = -5.1; +---- +physical_plan +01)CoalesceBatchesExec: target_batch_size=8192 +02)--FilterExec: CAST(int64@3 AS Float64) = -5.1 AND CAST(uint64@7 AS Float64) = -5.1 AND float64@9 = -5.1 AND decimal@10 = Some(-510),5,2 +03)----MemoryExec: partitions=1, partition_sizes=[1] + + +statement ok +set datafusion.explain.physical_plan_only = false; + + statement ok drop table numeric_types diff --git a/docs/source/user-guide/introduction.md b/docs/source/user-guide/introduction.md index 9bdb22b7145f..c97042fdc525 100644 --- a/docs/source/user-guide/introduction.md +++ b/docs/source/user-guide/introduction.md @@ -103,6 +103,7 @@ Here are some active projects using DataFusion: - [Dask SQL](https://github.com/dask-contrib/dask-sql) Distributed SQL query engine in Python - [delta-rs](https://github.com/delta-io/delta-rs) Native Rust implementation of Delta Lake - [Exon](https://github.com/wheretrue/exon) Analysis toolkit for life-science applications +- [Funnel](https://funnel.io/) Data Platform powering Marketing Intelligence applications. - [GlareDB](https://github.com/GlareDB/glaredb) Fast SQL database for querying and analyzing distributed data. - [GreptimeDB](https://github.com/GreptimeTeam/greptimedb) Open Source & Cloud Native Distributed Time Series Database - [HoraeDB](https://github.com/apache/incubator-horaedb) Distributed Time-Series Database