From 2fb9085d502f37cb833d270f3a8a1b8e006dd92c Mon Sep 17 00:00:00 2001 From: logan-keede Date: Fri, 14 Feb 2025 23:50:40 +0530 Subject: [PATCH 01/18] build moves only tests+benches pending --- Cargo.lock | 8 ++++++++ datafusion-cli/Cargo.toml | 1 + datafusion-cli/src/functions.rs | 2 +- .../examples/parquet_exec_visitor.rs | 2 +- .../core/src/datasource/cte_worktable.rs | 2 +- datafusion/core/src/datasource/empty.rs | 3 ++- .../core/src/datasource/listing/table.rs | 3 ++- datafusion/core/src/datasource/memory.rs | 4 ++-- .../datasource/physical_plan/arrow_file.rs | 2 +- .../core/src/datasource/physical_plan/avro.rs | 2 +- .../core/src/datasource/physical_plan/csv.rs | 2 +- .../physical_plan/file_scan_config.rs | 2 +- .../core/src/datasource/physical_plan/json.rs | 2 +- .../datasource/physical_plan/parquet/mod.rs | 2 +- datafusion/core/src/physical_planner.rs | 8 ++++---- datafusion/core/src/test/mod.rs | 2 +- datafusion/core/src/test_util/parquet.rs | 2 +- datafusion/datasource/Cargo.toml | 3 +++ .../src/empty.rs | 8 +++++--- .../src/memory.rs | 12 ++++++------ datafusion/datasource/src/mod.rs | 7 +++++++ .../src/placeholder_row.rs | 10 ++++++---- .../src/recursive_query.rs | 14 +++++++------- .../src/source.rs | 10 ++++++---- .../src/values.rs | 19 ++++++++++--------- .../src/work_table.rs | 10 ++++++---- datafusion/functions-table/Cargo.toml | 1 + .../functions-table/src/generate_series.rs | 2 +- datafusion/physical-optimizer/Cargo.toml | 1 + .../src/aggregate_statistics.rs | 2 +- datafusion/physical-plan/src/lib.rs | 14 +++++++------- datafusion/proto/Cargo.toml | 1 + datafusion/proto/src/physical_plan/mod.rs | 6 +++--- .../tests/cases/roundtrip_physical_plan.rs | 4 ++-- datafusion/substrait/Cargo.toml | 1 + .../substrait/src/physical_plan/producer.rs | 2 +- 36 files changed, 105 insertions(+), 71 deletions(-) rename datafusion/{physical-plan => datasource}/src/empty.rs (97%) rename datafusion/{physical-plan => datasource}/src/memory.rs (99%) rename datafusion/{physical-plan => datasource}/src/placeholder_row.rs (96%) rename datafusion/{physical-plan => datasource}/src/recursive_query.rs (97%) rename datafusion/{physical-plan => datasource}/src/source.rs (95%) rename datafusion/{physical-plan => datasource}/src/values.rs (95%) rename datafusion/{physical-plan => datasource}/src/work_table.rs (97%) diff --git a/Cargo.lock b/Cargo.lock index a03b295dabfc..fdfdfd1355ca 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1852,6 +1852,7 @@ dependencies = [ "clap 4.5.29", "ctor", "datafusion", + "datafusion-datasource", "dirs", "env_logger", "futures", @@ -1916,6 +1917,8 @@ dependencies = [ "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-plan", "flate2", "futures", @@ -1923,6 +1926,7 @@ dependencies = [ "itertools 0.14.0", "log", "object_store", + "parking_lot", "rand 0.8.5", "tempfile", "tokio", @@ -2122,6 +2126,7 @@ dependencies = [ "async-trait", "datafusion-catalog", "datafusion-common", + "datafusion-datasource", "datafusion-expr", "datafusion-physical-plan", "parking_lot", @@ -2227,6 +2232,7 @@ version = "45.0.0" dependencies = [ "arrow", "datafusion-common", + "datafusion-datasource", "datafusion-execution", "datafusion-expr", "datafusion-expr-common", @@ -2281,6 +2287,7 @@ dependencies = [ "chrono", "datafusion", "datafusion-common", + "datafusion-datasource", "datafusion-expr", "datafusion-functions", "datafusion-functions-aggregate", @@ -2371,6 +2378,7 @@ dependencies = [ "async-trait", "chrono", "datafusion", + "datafusion-datasource", "datafusion-functions-aggregate", "itertools 0.14.0", "object_store", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index cef32279371e..197a12c803c6 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -45,6 +45,7 @@ datafusion = { workspace = true, features = [ "unicode_expressions", "compression", ] } +datafusion-datasource = { workspace = true } dirs = "6.0.0" env_logger = { workspace = true } futures = { workspace = true } diff --git a/datafusion-cli/src/functions.rs b/datafusion-cli/src/functions.rs index b5bcb8243ea9..b6f7fcdc51a5 100644 --- a/datafusion-cli/src/functions.rs +++ b/datafusion-cli/src/functions.rs @@ -31,9 +31,9 @@ use datafusion::common::{plan_err, Column}; use datafusion::datasource::TableProvider; use datafusion::error::Result; use datafusion::logical_expr::Expr; -use datafusion::physical_plan::memory::MemorySourceConfig; use datafusion::physical_plan::ExecutionPlan; use datafusion::scalar::ScalarValue; +use datafusion_datasource::memory::MemorySourceConfig; use async_trait::async_trait; use parquet::basic::ConvertedType; diff --git a/datafusion-examples/examples/parquet_exec_visitor.rs b/datafusion-examples/examples/parquet_exec_visitor.rs index 20809a1121c1..3d3067ada01b 100644 --- a/datafusion-examples/examples/parquet_exec_visitor.rs +++ b/datafusion-examples/examples/parquet_exec_visitor.rs @@ -23,10 +23,10 @@ use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion::error::DataFusionError; use datafusion::execution::context::SessionContext; use datafusion::physical_plan::metrics::MetricValue; -use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::{ execute_stream, visit_execution_plan, ExecutionPlan, ExecutionPlanVisitor, }; +use datafusion_datasource::source::DataSourceExec; use futures::StreamExt; /// Example of collecting metrics after execution by visiting the `ExecutionPlan` diff --git a/datafusion/core/src/datasource/cte_worktable.rs b/datafusion/core/src/datasource/cte_worktable.rs index b63755f644a8..6ed4fb7781ce 100644 --- a/datafusion/core/src/datasource/cte_worktable.rs +++ b/datafusion/core/src/datasource/cte_worktable.rs @@ -23,7 +23,7 @@ use std::{any::Any, borrow::Cow}; use arrow::datatypes::SchemaRef; use async_trait::async_trait; use datafusion_catalog::Session; -use datafusion_physical_plan::work_table::WorkTableExec; +use datafusion_datasource::work_table::WorkTableExec; use crate::{ error::Result, diff --git a/datafusion/core/src/datasource/empty.rs b/datafusion/core/src/datasource/empty.rs index abda7fa9ec4b..e521b0e2e85f 100644 --- a/datafusion/core/src/datasource/empty.rs +++ b/datafusion/core/src/datasource/empty.rs @@ -28,7 +28,8 @@ use datafusion_common::project_schema; use crate::datasource::{TableProvider, TableType}; use crate::error::Result; use crate::logical_expr::Expr; -use crate::physical_plan::{empty::EmptyExec, ExecutionPlan}; +use datafusion_datasource::empty::EmptyExec; +use datafusion_physical_plan::ExecutionPlan; /// An empty plan that is useful for testing and generating plans /// without mapping them to actual data. diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 642ec93f3671..8a42627aab08 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -34,10 +34,11 @@ use crate::datasource::{ use crate::execution::context::SessionState; use datafusion_catalog::TableProvider; use datafusion_common::{config_err, DataFusionError, Result}; +use datafusion_datasource::empty::EmptyExec; use datafusion_expr::dml::InsertOp; use datafusion_expr::{utils::conjunction, Expr, TableProviderFilterPushDown}; use datafusion_expr::{SortExpr, TableType}; -use datafusion_physical_plan::{empty::EmptyExec, ExecutionPlan, Statistics}; +use datafusion_physical_plan::{ExecutionPlan, Statistics}; use arrow::datatypes::{DataType, Field, Schema, SchemaBuilder, SchemaRef}; use datafusion_common::{ diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index a996990105b3..12236945c2d0 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -38,11 +38,11 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_catalog::Session; use datafusion_common::{not_impl_err, plan_err, Constraints, DFSchema, SchemaExt}; +use datafusion_datasource::memory::MemorySourceConfig; +use datafusion_datasource::source::DataSourceExec; use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; use datafusion_expr::SortExpr; -use datafusion_physical_plan::memory::MemorySourceConfig; -use datafusion_physical_plan::source::DataSourceExec; use async_trait::async_trait; use futures::StreamExt; diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 4a7cdc192cd3..c6e05893a979 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -32,12 +32,12 @@ use arrow::datatypes::SchemaRef; use arrow_ipc::reader::FileDecoder; use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Statistics}; +use datafusion_datasource::source::DataSourceExec; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, }; diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 6aa330caffab..355286c082bb 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -29,12 +29,12 @@ use crate::error::Result; use arrow::datatypes::SchemaRef; use datafusion_common::{Constraints, Statistics}; +use datafusion_datasource::source::DataSourceExec; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::{ DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, }; diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 5e017b992581..936d5c0cac0d 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -37,13 +37,13 @@ use arrow::csv; use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Statistics}; +use datafusion_datasource::source::DataSourceExec; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use datafusion_physical_plan::projection::ProjectionExec; -use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, PlanProperties}; use futures::{StreamExt, TryStreamExt}; diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index e979eb49d0f6..8e177949c991 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -36,13 +36,13 @@ use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, Partitioning} use crate::datasource::data_source::FileSource; pub use datafusion_datasource::file_scan_config::*; +use datafusion_datasource::source::{DataSource, DataSourceExec}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_plan::display::{display_orderings, ProjectSchemaDisplay}; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::projection::{ all_alias_free_columns, new_projections_for_columns, ProjectionExec, }; -use datafusion_physical_plan::source::{DataSource, DataSourceExec}; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; /// Convert type to a type suitable for use as a [`ListingTable`] diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 51e0a46d942e..df475571a7a4 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -35,12 +35,12 @@ use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; use datafusion_common::{Constraints, Statistics}; +use datafusion_datasource::source::DataSourceExec; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::{DisplayAs, DisplayFormatType, PlanProperties}; use futures::{StreamExt, TryStreamExt}; diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index a1c2bb4207ef..d5fcd1035b9e 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -50,10 +50,10 @@ pub use access_plan::{ParquetAccessPlan, RowGroupAccess}; use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_common::Constraints; +use datafusion_datasource::source::DataSourceExec; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; use datafusion_physical_optimizer::pruning::PruningPredicate; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::source::DataSourceExec; pub use metrics::ParquetFileMetrics; pub use page_filter::PagePruningAccessPlanFilter; pub use reader::{DefaultParquetFileReaderFactory, ParquetFileReaderFactory}; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 47dee391c751..7c56e5531f23 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -38,7 +38,6 @@ use crate::logical_expr::{ use crate::physical_expr::{create_physical_expr, create_physical_exprs}; use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use crate::physical_plan::analyze::AnalyzeExec; -use crate::physical_plan::empty::EmptyExec; use crate::physical_plan::explain::ExplainExec; use crate::physical_plan::expressions::PhysicalSortExpr; use crate::physical_plan::filter::FilterExec; @@ -48,7 +47,6 @@ use crate::physical_plan::joins::{ }; use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use crate::physical_plan::projection::ProjectionExec; -use crate::physical_plan::recursive_query::RecursiveQueryExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::union::UnionExec; @@ -58,6 +56,8 @@ use crate::physical_plan::{ displayable, windows, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, Partitioning, PhysicalExpr, WindowExpr, }; +use datafusion_datasource::empty::EmptyExec; +use datafusion_datasource::recursive_query::RecursiveQueryExec; use arrow::array::{builder::StringBuilder, RecordBatch}; use arrow::compute::SortOptions; @@ -68,6 +68,8 @@ use datafusion_common::{ exec_err, internal_datafusion_err, internal_err, not_impl_err, plan_err, DFSchema, ScalarValue, }; +use datafusion_datasource::memory::MemorySourceConfig; +use datafusion_datasource::placeholder_row::PlaceholderRowExec; use datafusion_expr::dml::{CopyTo, InsertOp}; use datafusion_expr::expr::{ physical_name, AggregateFunction, Alias, GroupingSet, WindowFunction, @@ -83,8 +85,6 @@ use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::LexOrdering; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::execution_plan::InvariantLevel; -use datafusion_physical_plan::memory::MemorySourceConfig; -use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::unnest::ListUnnest; use crate::schema_equivalence::schema_satisfied_by; diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 5b7a9d8a16eb..fcd9e9145999 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -42,7 +42,7 @@ use arrow::array::{self, Array, ArrayRef, Decimal128Builder, Int32Array}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::DataFusionError; -use datafusion_physical_plan::source::DataSourceExec; +use datafusion_datasource::source::DataSourceExec; #[cfg(feature = "compression")] use bzip2::write::BzEncoder; diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index 67e0e1726917..f0ac6bb0df2e 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -37,7 +37,7 @@ use crate::physical_plan::metrics::MetricsSet; use crate::physical_plan::ExecutionPlan; use crate::prelude::{Expr, SessionConfig, SessionContext}; -use datafusion_physical_plan::source::DataSourceExec; +use datafusion_datasource::source::DataSourceExec; use object_store::path::Path; use object_store::ObjectMeta; use parquet::arrow::ArrowWriter; diff --git a/datafusion/datasource/Cargo.toml b/datafusion/datasource/Cargo.toml index caf1c60a785d..953b5ba2fd54 100644 --- a/datafusion/datasource/Cargo.toml +++ b/datafusion/datasource/Cargo.toml @@ -49,6 +49,8 @@ datafusion-common = { workspace = true, features = ["object_store"] } datafusion-common-runtime = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } +datafusion-physical-expr = { workspace = true } +datafusion-physical-expr-common = { workspace = true } datafusion-physical-plan = { workspace = true } flate2 = { version = "1.0.24", optional = true } futures = { workspace = true } @@ -56,6 +58,7 @@ glob = "0.3.0" itertools = { workspace = true } log = { workspace = true } object_store = { workspace = true } +parking_lot = { workspace = true } rand = { workspace = true } tokio = { workspace = true } tokio-util = { version = "0.7.4", features = ["io"], optional = true } diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/datasource/src/empty.rs similarity index 97% rename from datafusion/physical-plan/src/empty.rs rename to datafusion/datasource/src/empty.rs index 5168c3cc101f..f212a3548dd5 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/datasource/src/empty.rs @@ -20,10 +20,12 @@ use std::any::Any; use std::sync::Arc; -use super::{common, DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics}; -use crate::{ +use crate::memory::MemoryStream; +use datafusion_physical_plan::{ + common, DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics, +}; +use datafusion_physical_plan::{ execution_plan::{Boundedness, EmissionType}, - memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning, }; diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/datasource/src/memory.rs similarity index 99% rename from datafusion/physical-plan/src/memory.rs rename to datafusion/datasource/src/memory.rs index 0077804bdfc9..5ebf4de8851b 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -22,16 +22,16 @@ use std::fmt; use std::sync::Arc; use std::task::{Context, Poll}; -use super::{ +use crate::source::{DataSource, DataSourceExec}; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::projection::{ + all_alias_free_columns, new_projections_for_columns, ProjectionExec, +}; +use datafusion_physical_plan::{ common, ColumnarValue, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use crate::execution_plan::{Boundedness, EmissionType}; -use crate::projection::{ - all_alias_free_columns, new_projections_for_columns, ProjectionExec, -}; -use crate::source::{DataSource, DataSourceExec}; use arrow::array::{RecordBatch, RecordBatchOptions}; use arrow::datatypes::{Schema, SchemaRef}; diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index c735c3108b3d..b6975049a686 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -18,13 +18,20 @@ //! A table that uses the `ObjectStore` listing capability //! to get the list of files to process. +pub mod empty; pub mod file_compression_type; pub mod file_groups; pub mod file_meta; pub mod file_scan_config; pub mod file_sink_config; pub mod file_stream; +pub mod memory; +pub mod placeholder_row; +pub mod recursive_query; +pub mod source; pub mod url; +pub mod values; +pub mod work_table; pub mod write; use chrono::TimeZone; use datafusion_common::Result; diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/datasource/src/placeholder_row.rs similarity index 96% rename from datafusion/physical-plan/src/placeholder_row.rs rename to datafusion/datasource/src/placeholder_row.rs index 6a8f247ec0e6..4f3cd87bb222 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/datasource/src/placeholder_row.rs @@ -20,16 +20,18 @@ use std::any::Any; use std::sync::Arc; -use super::{common, DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics}; -use crate::execution_plan::{Boundedness, EmissionType}; -use crate::{memory::MemoryStream, DisplayFormatType, ExecutionPlan, Partitioning}; - +use crate::memory::MemoryStream; use arrow::array::{ArrayRef, NullArray}; use arrow::array::{RecordBatch, RecordBatchOptions}; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::{ + common, DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics, +}; +use datafusion_physical_plan::{DisplayFormatType, ExecutionPlan, Partitioning}; use log::trace; diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/datasource/src/recursive_query.rs similarity index 97% rename from datafusion/physical-plan/src/recursive_query.rs rename to datafusion/datasource/src/recursive_query.rs index bf7d2c7f275c..4b3b39d003ee 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/datasource/src/recursive_query.rs @@ -21,13 +21,13 @@ use std::any::Any; use std::sync::Arc; use std::task::{Context, Poll}; -use super::{ +use super::work_table::{ReservedBatches, WorkTable, WorkTableExec}; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, - work_table::{ReservedBatches, WorkTable, WorkTableExec}, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use crate::execution_plan::{Boundedness, EmissionType}; -use crate::{DisplayAs, DisplayFormatType, ExecutionPlan}; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; @@ -156,10 +156,10 @@ impl ExecutionPlan for RecursiveQueryExec { vec![false, false] } - fn required_input_distribution(&self) -> Vec { + fn required_input_distribution(&self) -> Vec { vec![ - datafusion_physical_expr::Distribution::SinglePartition, - datafusion_physical_expr::Distribution::SinglePartition, + datafusion_physical_plan::Distribution::SinglePartition, + datafusion_physical_plan::Distribution::SinglePartition, ] } diff --git a/datafusion/physical-plan/src/source.rs b/datafusion/datasource/src/source.rs similarity index 95% rename from datafusion/physical-plan/src/source.rs rename to datafusion/datasource/src/source.rs index 0c1dfddd2678..1b0d76902972 100644 --- a/datafusion/physical-plan/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -20,10 +20,12 @@ use std::fmt; use std::fmt::{Debug, Formatter}; use std::sync::Arc; -use crate::execution_plan::{Boundedness, EmissionType}; -use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::projection::ProjectionExec; -use crate::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, +}; use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Statistics}; diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/datasource/src/values.rs similarity index 95% rename from datafusion/physical-plan/src/values.rs rename to datafusion/datasource/src/values.rs index 6ab5cc84a21f..ae677b4a768a 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/datasource/src/values.rs @@ -20,18 +20,19 @@ use std::any::Any; use std::sync::Arc; -use super::{common, DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics}; -use crate::execution_plan::{Boundedness, EmissionType}; -use crate::{ - memory::MemoryStream, ColumnarValue, DisplayFormatType, ExecutionPlan, Partitioning, - PhysicalExpr, -}; - +use crate::memory::MemoryStream; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::{internal_err, plan_err, Result, ScalarValue}; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::{ + common, DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics, +}; +use datafusion_physical_plan::{ + ColumnarValue, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, +}; /// Execution plan for values list based relation (produces constant rows) #[deprecated( @@ -230,8 +231,8 @@ impl ExecutionPlan for ValuesExec { #[cfg(test)] mod tests { use super::*; - use crate::expressions::lit; - use crate::test::{self, make_partition}; + use datafusion_physical_plan::expressions::lit; + use datafusion_physical_plan::test::{self, make_partition}; use arrow::datatypes::{DataType, Field}; use datafusion_common::stats::{ColumnStatistics, Precision}; diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/datasource/src/work_table.rs similarity index 97% rename from datafusion/physical-plan/src/work_table.rs rename to datafusion/datasource/src/work_table.rs index a6ced527cbb2..6614b1d59f59 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/datasource/src/work_table.rs @@ -20,13 +20,15 @@ use std::any::Any; use std::sync::{Arc, Mutex}; -use super::{ +use crate::memory::MemoryStream; +use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::{ metrics::{ExecutionPlanMetricsSet, MetricsSet}, SendableRecordBatchStream, Statistics, }; -use crate::execution_plan::{Boundedness, EmissionType}; -use crate::memory::MemoryStream; -use crate::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; +use datafusion_physical_plan::{ + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, +}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; diff --git a/datafusion/functions-table/Cargo.toml b/datafusion/functions-table/Cargo.toml index f722d698f3d3..3020d7d131c5 100644 --- a/datafusion/functions-table/Cargo.toml +++ b/datafusion/functions-table/Cargo.toml @@ -42,6 +42,7 @@ arrow = { workspace = true } async-trait = { workspace = true } datafusion-catalog = { workspace = true } datafusion-common = { workspace = true } +datafusion-datasource = { workspace = true } datafusion-expr = { workspace = true } datafusion-physical-plan = { workspace = true } parking_lot = { workspace = true } diff --git a/datafusion/functions-table/src/generate_series.rs b/datafusion/functions-table/src/generate_series.rs index 887daa71ec55..8c460b08ef6a 100644 --- a/datafusion/functions-table/src/generate_series.rs +++ b/datafusion/functions-table/src/generate_series.rs @@ -23,8 +23,8 @@ use datafusion_catalog::Session; use datafusion_catalog::TableFunctionImpl; use datafusion_catalog::TableProvider; use datafusion_common::{plan_err, Result, ScalarValue}; +use datafusion_datasource::memory::{LazyBatchGenerator, LazyMemoryExec}; use datafusion_expr::{Expr, TableType}; -use datafusion_physical_plan::memory::{LazyBatchGenerator, LazyMemoryExec}; use datafusion_physical_plan::ExecutionPlan; use parking_lot::RwLock; use std::fmt; diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index c9c86e9c8d5c..ee9cdf052093 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -37,6 +37,7 @@ recursive_protection = ["dep:recursive"] [dependencies] arrow = { workspace = true } datafusion-common = { workspace = true, default-features = true } +datafusion-datasource = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-expr-common = { workspace = true, default-features = true } diff --git a/datafusion/physical-optimizer/src/aggregate_statistics.rs b/datafusion/physical-optimizer/src/aggregate_statistics.rs index a9b02188a7a2..a10f7de96460 100644 --- a/datafusion/physical-optimizer/src/aggregate_statistics.rs +++ b/datafusion/physical-optimizer/src/aggregate_statistics.rs @@ -20,8 +20,8 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::scalar::ScalarValue; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::Result; +use datafusion_datasource::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::aggregates::AggregateExec; -use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::udaf::{AggregateFunctionExpr, StatisticsArgs}; use datafusion_physical_plan::{expressions, ExecutionPlan}; diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 9210e3b0273c..3acecf9d9502 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -56,18 +56,18 @@ pub mod coalesce_batches; pub mod coalesce_partitions; pub mod common; pub mod display; -pub mod empty; +// pub mod empty; pub mod execution_plan; pub mod explain; pub mod filter; pub mod insert; pub mod joins; pub mod limit; -pub mod memory; +// pub mod memory; pub mod metrics; -pub mod placeholder_row; +// pub mod placeholder_row; pub mod projection; -pub mod recursive_query; +// pub mod recursive_query; pub mod repartition; pub mod sorts; pub mod spill; @@ -76,9 +76,9 @@ pub mod streaming; pub mod tree_node; pub mod union; pub mod unnest; -pub mod values; +// pub mod values; pub mod windows; -pub mod work_table; +// pub mod work_table; pub mod udaf { pub use datafusion_expr::StatisticsArgs; @@ -86,6 +86,6 @@ pub mod udaf { } pub mod coalesce; -pub mod source; +// pub mod source; #[cfg(test)] pub mod test; diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index 16bb8b86b78e..e4d3ab2e274a 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -45,6 +45,7 @@ arrow = { workspace = true } chrono = { workspace = true } datafusion = { workspace = true, default-features = true } datafusion-common = { workspace = true, default-features = true } +datafusion-datasource = { workspace = true } datafusion-expr = { workspace = true } datafusion-proto-common = { workspace = true } object_store = { workspace = true } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 84b952965958..356fc18f84e3 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -41,7 +41,6 @@ use datafusion::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; use datafusion::physical_plan::analyze::AnalyzeExec; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::explain::ExplainExec; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::filter::FilterExec; @@ -52,12 +51,10 @@ use datafusion::physical_plan::joins::{ }; use datafusion::physical_plan::joins::{HashJoinExec, PartitionMode}; use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use datafusion::physical_plan::placeholder_row::PlaceholderRowExec; 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::source::DataSourceExec; use datafusion::physical_plan::union::{InterleaveExec, UnionExec}; use datafusion::physical_plan::unnest::{ListUnnest, UnnestExec}; use datafusion::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; @@ -66,6 +63,9 @@ use datafusion::physical_plan::{ }; use datafusion_common::config::TableParquetOptions; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; +use datafusion_datasource::empty::EmptyExec; +use datafusion_datasource::placeholder_row::PlaceholderRowExec; +use datafusion_datasource::source::DataSourceExec; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; use crate::common::{byte_to_string, str_to_byte}; diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 7418184fcac1..eabfb4f87745 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -64,7 +64,6 @@ use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; use datafusion::physical_plan::analyze::AnalyzeExec; -use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::{ binary, cast, col, in_list, like, lit, BinaryExpr, Column, NotExpr, PhysicalSortExpr, }; @@ -74,7 +73,6 @@ use datafusion::physical_plan::joins::{ HashJoinExec, NestedLoopJoinExec, PartitionMode, StreamJoinPartitionMode, }; use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use datafusion::physical_plan::placeholder_row::PlaceholderRowExec; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; @@ -97,6 +95,8 @@ use datafusion_common::stats::Precision; use datafusion_common::{ internal_err, not_impl_err, Constraints, DataFusionError, Result, UnnestOptions, }; +use datafusion_datasource::empty::EmptyExec; +use datafusion_datasource::placeholder_row::PlaceholderRowExec; use datafusion_expr::{ Accumulator, AccumulatorFactoryFunction, AggregateUDF, ColumnarValue, ScalarUDF, Signature, SimpleAggregateUDF, WindowFrame, WindowFrameBound, WindowUDF, diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index f13d2b77a787..6271f2992504 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -35,6 +35,7 @@ async-recursion = "1.0" async-trait = { workspace = true } chrono = { workspace = true } datafusion = { workspace = true } +datafusion-datasource = { workspace = true } itertools = { workspace = true } object_store = { workspace = true } pbjson-types = { workspace = true } diff --git a/datafusion/substrait/src/physical_plan/producer.rs b/datafusion/substrait/src/physical_plan/producer.rs index 3fc94a33442b..5bf1adf4fc16 100644 --- a/datafusion/substrait/src/physical_plan/producer.rs +++ b/datafusion/substrait/src/physical_plan/producer.rs @@ -24,8 +24,8 @@ use crate::variation_const::{ use datafusion::arrow::datatypes::DataType; use datafusion::error::{DataFusionError, Result}; -use datafusion::physical_plan::source::DataSourceExec; use datafusion::physical_plan::{displayable, ExecutionPlan}; +use datafusion_datasource::source::DataSourceExec; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use substrait::proto::expression::mask_expression::{StructItem, StructSelect}; From 2d135a9617d25eb3819bcb2d45d69a6d82c20b5a Mon Sep 17 00:00:00 2001 From: logan-keede Date: Sat, 15 Feb 2025 16:54:33 +0530 Subject: [PATCH 02/18] unstable --- Cargo.lock | 2 ++ datafusion/core/src/datasource/physical_plan/csv.rs | 2 +- .../core/src/datasource/physical_plan/parquet/mod.rs | 2 +- datafusion/core/tests/custom_sources_cases/mod.rs | 2 +- datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs | 4 ++-- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 4 ++-- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 2 +- .../tests/fuzz_cases/sort_preserving_repartition_fuzz.rs | 4 ++-- datafusion/core/tests/fuzz_cases/window_fuzz.rs | 4 ++-- datafusion/core/tests/memory_limit/mod.rs | 4 ++-- datafusion/core/tests/parquet/file_statistics.rs | 2 +- datafusion/core/tests/parquet/page_pruning.rs | 2 +- datafusion/core/tests/parquet/utils.rs | 2 +- .../core/tests/physical_optimizer/aggregate_statistics.rs | 4 ++-- .../core/tests/physical_optimizer/enforce_distribution.rs | 2 +- datafusion/core/tests/physical_optimizer/limit_pushdown.rs | 2 +- .../core/tests/physical_optimizer/projection_pushdown.rs | 4 ++-- .../replace_with_order_preserving_variants.rs | 4 ++-- datafusion/core/tests/physical_optimizer/test_utils.rs | 4 ++-- datafusion/core/tests/sql/path_partition.rs | 2 +- .../tests/user_defined/user_defined_table_functions.rs | 2 +- datafusion/physical-plan/Cargo.toml | 7 +++++++ datafusion/physical-plan/benches/spm.rs | 2 +- 23 files changed, 39 insertions(+), 30 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index fdfdfd1355ca..9249867c318c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2258,6 +2258,7 @@ dependencies = [ "criterion", "datafusion-common", "datafusion-common-runtime", + "datafusion-datasource", "datafusion-execution", "datafusion-expr", "datafusion-functions-aggregate", @@ -2265,6 +2266,7 @@ dependencies = [ "datafusion-functions-window-common", "datafusion-physical-expr", "datafusion-physical-expr-common", + "datafusion-physical-plan", "futures", "half", "hashbrown 0.14.5", diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 936d5c0cac0d..8b710e696347 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -409,7 +409,7 @@ impl ExecutionPlan for CsvExec { /// # }; /// # use datafusion::datasource::physical_plan::CsvSource; /// # use datafusion_execution::object_store::ObjectStoreUrl; -/// # use datafusion_physical_plan::source::DataSourceExec; +/// # use datafusion_datasource::source::DataSourceExec; /// /// # let object_store_url = ObjectStoreUrl::local_filesystem(); /// # let file_schema = Arc::new(Schema::empty()); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index d5fcd1035b9e..47b0bbf49750 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -582,7 +582,7 @@ mod tests { use datafusion_expr::{col, lit, when, Expr}; use datafusion_physical_expr::planner::logical2physical; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; - use datafusion_physical_plan::source::DataSourceExec; + use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use crate::datasource::physical_plan::parquet::source::ParquetSource; diff --git a/datafusion/core/tests/custom_sources_cases/mod.rs b/datafusion/core/tests/custom_sources_cases/mod.rs index aafefac04e32..c4de715a811e 100644 --- a/datafusion/core/tests/custom_sources_cases/mod.rs +++ b/datafusion/core/tests/custom_sources_cases/mod.rs @@ -41,7 +41,7 @@ use datafusion_common::project_schema; use datafusion_common::stats::Precision; use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; +use datafusion_datasource::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::PlanProperties; use async_trait::async_trait; diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 5e1f263b4c76..f6286039a7dd 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -43,8 +43,8 @@ use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_plan::memory::MemorySourceConfig; -use datafusion_physical_plan::source::DataSourceExec; +use datafusion_datasource::memory::MemorySourceConfig; +use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::InputOrderMode; use test_utils::{add_empty_batches, StringBatchGenerator}; diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index 5dd29f90ef83..ef9e4a497776 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -38,8 +38,8 @@ use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::ScalarValue; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::PhysicalExprRef; -use datafusion_physical_plan::memory::MemorySourceConfig; -use datafusion_physical_plan::source::DataSourceExec; +use datafusion_datasource::memory::MemorySourceConfig; +use datafusion_datasource::source::DataSourceExec; use itertools::Itertools; use rand::Rng; diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index ecc077261acc..fefb8f1c188e 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -32,7 +32,7 @@ use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_execution::memory_pool::GreedyMemoryPool; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_datasource::memory::MemorySourceConfig; use rand::Rng; use test_utils::{batches_to_vec, partitions_to_sorted_vec}; diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index d23408743f9f..7bf5ceb6ad53 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -45,8 +45,8 @@ mod sp_repartition_fuzz_tests { use test_utils::add_empty_batches; use datafusion_physical_expr_common::sort_expr::LexOrdering; - use datafusion_physical_plan::memory::MemorySourceConfig; - use datafusion_physical_plan::source::DataSourceExec; + use datafusion_datasource::memory::MemorySourceConfig; + use datafusion_datasource::source::DataSourceExec; use itertools::izip; use rand::{rngs::StdRng, seq::SliceRandom, Rng, SeedableRng}; diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 4a484221a88a..1c07e5759db6 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -48,8 +48,8 @@ use datafusion_functions_window::rank::{dense_rank_udwf, rank_udwf}; use datafusion_physical_expr::expressions::{cast, col, lit}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_plan::memory::MemorySourceConfig; -use datafusion_physical_plan::source::DataSourceExec; +use datafusion_datasource::memory::MemorySourceConfig; +use datafusion_datasource::source::DataSourceExec; use rand::distributions::Alphanumeric; use rand::rngs::StdRng; diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index 719faed4e454..c3255a25b419 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -46,8 +46,8 @@ 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::memory::MemorySourceConfig; -use datafusion_physical_plan::source::DataSourceExec; +use datafusion_datasource::memory::MemorySourceConfig; +use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::spill::get_record_batch_memory_size; use test_utils::AccessLogGenerator; diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index 82024a731ed3..61d51aeefefe 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -34,7 +34,7 @@ use datafusion_execution::cache::cache_unit::{ use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_expr::{col, lit, Expr}; -use datafusion_physical_plan::source::DataSourceExec; +use datafusion_datasource::source::DataSourceExec; use datafusion::datasource::physical_plan::FileScanConfig; use tempfile::tempdir; diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index ea86bf3685bb..13c5fbee4ccf 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -33,7 +33,7 @@ use datafusion_common::{ScalarValue, ToDFSchema}; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::{col, lit, Expr}; use datafusion_physical_expr::create_physical_expr; -use datafusion_physical_plan::source::DataSourceExec; +use datafusion_datasource::source::DataSourceExec; use futures::StreamExt; use object_store::path::Path; diff --git a/datafusion/core/tests/parquet/utils.rs b/datafusion/core/tests/parquet/utils.rs index dd5541461ff6..10b93eddfede 100644 --- a/datafusion/core/tests/parquet/utils.rs +++ b/datafusion/core/tests/parquet/utils.rs @@ -19,7 +19,7 @@ use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use datafusion_physical_plan::metrics::MetricsSet; -use datafusion_physical_plan::source::DataSourceExec; +use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::{accept, ExecutionPlan, ExecutionPlanVisitor}; /// Find the metrics from the first DataSourceExec encountered in the plan diff --git a/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs b/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs index 1757c7150bfe..fd8e9d21e7e4 100644 --- a/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs @@ -36,9 +36,9 @@ 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::MemorySourceConfig; +use datafusion_datasource::memory::MemorySourceConfig; use datafusion_physical_plan::projection::ProjectionExec; -use datafusion_physical_plan::source::DataSourceExec; +use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::ExecutionPlan; /// Mock data using a MemorySourceConfig which has an exact count statistic diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 855550dc748a..112524dd3c95 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -57,7 +57,7 @@ use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use datafusion_physical_plan::source::DataSourceExec; +use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::ExecutionPlanProperties; use datafusion_physical_plan::PlanProperties; diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index dd2c1960a658..4df884ec64e0 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -30,7 +30,7 @@ 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_datasource::empty::EmptyExec; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::projection::ProjectionExec; diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index c9eadf009130..3087998b7867 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -48,12 +48,12 @@ use datafusion_physical_plan::joins::{ HashJoinExec, NestedLoopJoinExec, PartitionMode, StreamJoinPartitionMode, SymmetricHashJoinExec, }; -use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_datasource::memory::MemorySourceConfig; 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::source::DataSourceExec; +use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::streaming::PartitionStream; use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; 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 d9b569dfa611..30cfda2d319a 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 @@ -32,13 +32,13 @@ use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::collect; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; -use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_datasource::memory::MemorySourceConfig; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::{ displayable, get_plan_string, ExecutionPlan, Partitioning, }; -use datafusion_physical_plan::source::DataSourceExec; +use datafusion_datasource::source::DataSourceExec; use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_common::Result; use datafusion_expr::{JoinType, Operator}; diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 5e486a715b41..d04485ede760 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -52,11 +52,11 @@ use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::{JoinFilter, JoinOn}; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode, SortMergeJoinExec}; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_datasource::memory::MemorySourceConfig; 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::source::DataSourceExec; +use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::union::UnionExec; diff --git a/datafusion/core/tests/sql/path_partition.rs b/datafusion/core/tests/sql/path_partition.rs index 6345f5e4352f..6ec757fd2212 100644 --- a/datafusion/core/tests/sql/path_partition.rs +++ b/datafusion/core/tests/sql/path_partition.rs @@ -43,7 +43,7 @@ use datafusion_common::ScalarValue; use datafusion_execution::config::SessionConfig; use datafusion_expr::{col, lit, Expr, Operator}; use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; -use datafusion_physical_plan::source::DataSourceExec; +use datafusion_datasource::source::DataSourceExec; use async_trait::async_trait; use bytes::Bytes; diff --git a/datafusion/core/tests/user_defined/user_defined_table_functions.rs b/datafusion/core/tests/user_defined/user_defined_table_functions.rs index 0ec9a5fd7620..12e278b4e877 100644 --- a/datafusion/core/tests/user_defined/user_defined_table_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_table_functions.rs @@ -35,7 +35,7 @@ use datafusion_catalog::Session; use datafusion_catalog::TableFunctionImpl; use datafusion_common::{assert_batches_eq, DFSchema, ScalarValue}; use datafusion_expr::{EmptyRelation, Expr, LogicalPlan, Projection, TableType}; -use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_datasource::memory::MemorySourceConfig; use async_trait::async_trait; diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index b84243b1b56b..33ccff6378d2 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -74,7 +74,14 @@ tokio = { workspace = true, features = [ "fs", "parking_lot", ] } +datafusion-datasource = { workspace = true } +datafusion-physical-plan = { workspace = true} [[bench]] harness = false name = "spm" + +[[test]] +harness = false +name = "tests" +path = "tests/mod.rs" diff --git a/datafusion/physical-plan/benches/spm.rs b/datafusion/physical-plan/benches/spm.rs index 3a2ecb57394b..63b06f20cd86 100644 --- a/datafusion/physical-plan/benches/spm.rs +++ b/datafusion/physical-plan/benches/spm.rs @@ -27,7 +27,7 @@ use datafusion_physical_plan::{collect, ExecutionPlan}; use criterion::async_executor::FuturesExecutor; use criterion::{black_box, criterion_group, criterion_main, Criterion}; -use datafusion_physical_plan::memory::MemorySourceConfig; +use datafusion_datasource::memory::MemorySourceConfig; fn generate_spm_for_round_robin_tie_breaker( has_same_value: bool, From 34cd4aeaad4c8b8dbf03c90f9fec7d10cd522985 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Sat, 15 Feb 2025 17:19:39 +0530 Subject: [PATCH 03/18] some tests fixed --- .../core/tests/fuzz_cases/merge_fuzz.rs | 2 +- datafusion/datasource/src/empty.rs | 2 +- datafusion/datasource/src/memory.rs | 4 +- datafusion/datasource/src/mod.rs | 1 + datafusion/datasource/src/placeholder_row.rs | 3 +- datafusion/datasource/src/test.rs | 177 ++++ datafusion/datasource/src/test/exec.rs | 862 ++++++++++++++++++ datafusion/datasource/src/values.rs | 2 +- 8 files changed, 1047 insertions(+), 6 deletions(-) create mode 100644 datafusion/datasource/src/test.rs create mode 100644 datafusion/datasource/src/test/exec.rs diff --git a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs index 35fca789ddcb..a9943ce5a07f 100644 --- a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs @@ -27,9 +27,9 @@ use arrow::{ use datafusion::physical_plan::{ collect, expressions::{col, PhysicalSortExpr}, - memory::MemorySourceConfig, sorts::sort_preserving_merge::SortPreservingMergeExec, }; +use datafusion_datasource::memory::MemorySourceConfig; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_physical_expr_common::sort_expr::LexOrdering; diff --git a/datafusion/datasource/src/empty.rs b/datafusion/datasource/src/empty.rs index f212a3548dd5..9b5854e9cb6c 100644 --- a/datafusion/datasource/src/empty.rs +++ b/datafusion/datasource/src/empty.rs @@ -163,7 +163,7 @@ impl ExecutionPlan for EmptyExec { mod tests { use super::*; use crate::test; - use crate::with_new_children_if_necessary; + use datafusion_physical_plan::with_new_children_if_necessary; #[tokio::test] async fn empty() -> Result<()> { diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 5ebf4de8851b..46fe8853655c 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -968,7 +968,7 @@ mod memory_exec_tests { use crate::memory::MemorySourceConfig; use crate::source::DataSourceExec; - use crate::ExecutionPlan; + use datafusion_physical_plan::ExecutionPlan; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; @@ -1141,7 +1141,7 @@ mod lazy_memory_tests { #[cfg(test)] mod tests { use super::*; - use crate::expressions::lit; + use datafusion_physical_plan::expressions::lit; use crate::test::{self, make_partition}; use arrow::datatypes::{DataType, Field}; diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index b6975049a686..43be5f4d2ac1 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -40,6 +40,7 @@ use futures::Stream; use object_store::{path::Path, ObjectMeta}; use std::pin::Pin; use std::sync::Arc; +pub mod test; pub use self::url::ListingTableUrl; diff --git a/datafusion/datasource/src/placeholder_row.rs b/datafusion/datasource/src/placeholder_row.rs index 4f3cd87bb222..6b16d1665f5a 100644 --- a/datafusion/datasource/src/placeholder_row.rs +++ b/datafusion/datasource/src/placeholder_row.rs @@ -180,7 +180,8 @@ impl ExecutionPlan for PlaceholderRowExec { #[cfg(test)] mod tests { use super::*; - use crate::{test, with_new_children_if_necessary}; + use crate::test; + use datafusion_physical_plan::with_new_children_if_necessary; #[test] fn with_new_children() -> Result<()> { diff --git a/datafusion/datasource/src/test.rs b/datafusion/datasource/src/test.rs new file mode 100644 index 000000000000..c6ca0efa5205 --- /dev/null +++ b/datafusion/datasource/src/test.rs @@ -0,0 +1,177 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Utilities for testing datafusion-physical-plan + +use std::collections::HashMap; +use std::pin::Pin; +use std::sync::Arc; + +use arrow::array::{ArrayRef, Int32Array, RecordBatch}; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use futures::{Future, FutureExt}; + +use crate::memory::MemorySourceConfig; +use crate::source::DataSourceExec; +use datafusion_physical_plan::stream::RecordBatchStreamAdapter; +use datafusion_physical_plan::streaming::PartitionStream; +use datafusion_physical_plan::ExecutionPlan; + +pub mod exec; +// pub use crate::test::exec; + +/// Asserts that given future is pending. +pub fn assert_is_pending<'a, T>(fut: &mut Pin + Send + 'a>>) { + let waker = futures::task::noop_waker(); + let mut cx = futures::task::Context::from_waker(&waker); + let poll = fut.poll_unpin(&mut cx); + + assert!(poll.is_pending()); +} + +/// Get the schema for the aggregate_test_* csv files +pub fn aggr_test_schema() -> SchemaRef { + let mut f1 = Field::new("c1", DataType::Utf8, false); + f1.set_metadata(HashMap::from_iter(vec![("testing".into(), "test".into())])); + let schema = Schema::new(vec![ + f1, + Field::new("c2", DataType::UInt32, false), + Field::new("c3", DataType::Int8, false), + Field::new("c4", DataType::Int16, false), + Field::new("c5", DataType::Int32, false), + Field::new("c6", DataType::Int64, false), + Field::new("c7", DataType::UInt8, false), + Field::new("c8", DataType::UInt16, false), + Field::new("c9", DataType::UInt32, false), + Field::new("c10", DataType::UInt64, false), + Field::new("c11", DataType::Float32, false), + Field::new("c12", DataType::Float64, false), + Field::new("c13", DataType::Utf8, false), + ]); + + Arc::new(schema) +} + +/// Returns record batch with 3 columns of i32 in memory +pub fn build_table_i32( + a: (&str, &Vec), + b: (&str, &Vec), + c: (&str, &Vec), +) -> RecordBatch { + let schema = Schema::new(vec![ + Field::new(a.0, DataType::Int32, false), + Field::new(b.0, DataType::Int32, false), + Field::new(c.0, DataType::Int32, false), + ]); + + RecordBatch::try_new( + Arc::new(schema), + vec![ + Arc::new(Int32Array::from(a.1.clone())), + Arc::new(Int32Array::from(b.1.clone())), + Arc::new(Int32Array::from(c.1.clone())), + ], + ) + .unwrap() +} + +/// Returns record batch with 2 columns of i32 in memory +pub fn build_table_i32_two_cols( + a: (&str, &Vec), + b: (&str, &Vec), +) -> RecordBatch { + let schema = Schema::new(vec![ + Field::new(a.0, DataType::Int32, false), + Field::new(b.0, DataType::Int32, false), + ]); + + RecordBatch::try_new( + Arc::new(schema), + vec![ + Arc::new(Int32Array::from(a.1.clone())), + Arc::new(Int32Array::from(b.1.clone())), + ], + ) + .unwrap() +} + +/// Returns memory table scan wrapped around record batch with 3 columns of i32 +pub fn build_table_scan_i32( + a: (&str, &Vec), + b: (&str, &Vec), + c: (&str, &Vec), +) -> Arc { + let batch = build_table_i32(a, b, c); + let schema = batch.schema(); + MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() +} + +/// Return a RecordBatch with a single Int32 array with values (0..sz) in a field named "i" +pub fn make_partition(sz: i32) -> RecordBatch { + let seq_start = 0; + let seq_end = sz; + let values = (seq_start..seq_end).collect::>(); + let schema = Arc::new(Schema::new(vec![Field::new("i", DataType::Int32, true)])); + let arr = Arc::new(Int32Array::from(values)); + let arr = arr as ArrayRef; + + RecordBatch::try_new(schema, vec![arr]).unwrap() +} + +/// Returns a `DataSourceExec` that scans `partitions` of 100 batches each +pub fn scan_partitioned(partitions: usize) -> Arc { + Arc::new(mem_exec(partitions)) +} + +/// Returns a `DataSourceExec` that scans `partitions` of 100 batches each +pub fn mem_exec(partitions: usize) -> DataSourceExec { + let data: Vec> = (0..partitions).map(|_| vec![make_partition(100)]).collect(); + + let schema = data[0][0].schema(); + let projection = None; + DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(&data, schema, projection).unwrap(), + )) +} + +// Construct a stream partition for test purposes +#[derive(Debug)] +pub struct TestPartitionStream { + pub schema: SchemaRef, + pub batches: Vec, +} + +impl TestPartitionStream { + /// Create a new stream partition with the provided batches + pub fn new_with_batches(batches: Vec) -> Self { + let schema = batches[0].schema(); + Self { schema, batches } + } +} +impl PartitionStream for TestPartitionStream { + fn schema(&self) -> &SchemaRef { + &self.schema + } + fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { + let stream = futures::stream::iter(self.batches.clone().into_iter().map(Ok)); + Box::pin(RecordBatchStreamAdapter::new( + Arc::clone(&self.schema), + stream, + )) + } +} diff --git a/datafusion/datasource/src/test/exec.rs b/datafusion/datasource/src/test/exec.rs new file mode 100644 index 000000000000..de8fc7e9d1fd --- /dev/null +++ b/datafusion/datasource/src/test/exec.rs @@ -0,0 +1,862 @@ +// 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. + +//! Simple iterator over batches for use in testing + +use std::{ + any::Any, + pin::Pin, + sync::{Arc, Weak}, + task::{Context, Poll}, +}; + +use datafusion_physical_plan::{ + common, execution_plan::Boundedness, DisplayAs, DisplayFormatType, ExecutionPlan, + Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, + Statistics, +}; +use datafusion_physical_plan::{ + execution_plan::EmissionType, + stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}, +}; + +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; +use datafusion_common::{internal_err, DataFusionError, Result}; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; + +use futures::Stream; +use tokio::sync::Barrier; + +/// Index into the data that has been returned so far +#[derive(Debug, Default, Clone)] +pub struct BatchIndex { + inner: Arc>, +} + +impl BatchIndex { + /// Return the current index + pub fn value(&self) -> usize { + let inner = self.inner.lock().unwrap(); + *inner + } + + // increment the current index by one + pub fn incr(&self) { + let mut inner = self.inner.lock().unwrap(); + *inner += 1; + } +} + +/// Iterator over batches +#[derive(Debug, Default)] +pub struct TestStream { + /// Vector of record batches + data: Vec, + /// Index into the data that has been returned so far + index: BatchIndex, +} + +impl TestStream { + /// Create an iterator for a vector of record batches. Assumes at + /// least one entry in data (for the schema) + pub fn new(data: Vec) -> Self { + Self { + data, + ..Default::default() + } + } + + /// Return a handle to the index counter for this stream + pub fn index(&self) -> BatchIndex { + self.index.clone() + } +} + +impl Stream for TestStream { + type Item = Result; + + fn poll_next(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { + let next_batch = self.index.value(); + + Poll::Ready(if next_batch < self.data.len() { + let next_batch = self.index.value(); + self.index.incr(); + Some(Ok(self.data[next_batch].clone())) + } else { + None + }) + } + + fn size_hint(&self) -> (usize, Option) { + (self.data.len(), Some(self.data.len())) + } +} + +impl RecordBatchStream for TestStream { + /// Get the schema + fn schema(&self) -> SchemaRef { + self.data[0].schema() + } +} + +/// A Mock ExecutionPlan that can be used for writing tests of other +/// ExecutionPlans +#[derive(Debug)] +pub struct MockExec { + /// the results to send back + data: Vec>, + schema: SchemaRef, + /// if true (the default), sends data using a separate task to ensure the + /// batches are not available without this stream yielding first + use_task: bool, + cache: PlanProperties, +} + +impl MockExec { + /// Create a new `MockExec` with a single partition that returns + /// the specified `Results`s. + /// + /// By default, the batches are not produced immediately (the + /// caller has to actually yield and another task must run) to + /// ensure any poll loops are correct. This behavior can be + /// changed with `with_use_task` + pub fn new(data: Vec>, schema: SchemaRef) -> Self { + let cache = Self::compute_properties(Arc::clone(&schema)); + Self { + data, + schema, + use_task: true, + cache, + } + } + + /// If `use_task` is true (the default) then the batches are sent + /// back using a separate task to ensure the underlying stream is + /// not immediately ready + pub fn with_use_task(mut self, use_task: bool) -> Self { + self.use_task = use_task; + self + } + + /// 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(1), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } +} + +impl DisplayAs for MockExec { + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "MockExec") + } + } + } +} + +impl ExecutionPlan for MockExec { + 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> { + unimplemented!() + } + + /// Returns a stream which yields data + fn execute( + &self, + partition: usize, + _context: Arc, + ) -> Result { + assert_eq!(partition, 0); + + // Result doesn't implement clone, so do it ourself + let data: Vec<_> = self + .data + .iter() + .map(|r| match r { + Ok(batch) => Ok(batch.clone()), + Err(e) => Err(clone_error(e)), + }) + .collect(); + + if self.use_task { + let mut builder = RecordBatchReceiverStream::builder(self.schema(), 2); + // send data in order but in a separate task (to ensure + // the batches are not available without the stream + // yielding). + let tx = builder.tx(); + builder.spawn(async move { + for batch in data { + println!("Sending batch via delayed stream"); + if let Err(e) = tx.send(batch).await { + println!("ERROR batch via delayed stream: {e}"); + } + } + + Ok(()) + }); + // returned stream simply reads off the rx stream + Ok(builder.build()) + } else { + // make an input that will error + let stream = futures::stream::iter(data); + Ok(Box::pin(RecordBatchStreamAdapter::new( + self.schema(), + stream, + ))) + } + } + + // Panics if one of the batches is an error + fn statistics(&self) -> Result { + let data: Result> = self + .data + .iter() + .map(|r| match r { + Ok(batch) => Ok(batch.clone()), + Err(e) => Err(clone_error(e)), + }) + .collect(); + + let data = data?; + + Ok(common::compute_record_batch_statistics( + &[data], + &self.schema, + None, + )) + } +} + +fn clone_error(e: &DataFusionError) -> DataFusionError { + use DataFusionError::*; + match e { + Execution(msg) => Execution(msg.to_string()), + _ => unimplemented!(), + } +} + +/// A Mock ExecutionPlan that does not start producing input until a +/// barrier is called +/// +#[derive(Debug)] +pub struct BarrierExec { + /// partitions to send back + data: Vec>, + schema: SchemaRef, + + /// all streams wait on this barrier to produce + barrier: Arc, + cache: PlanProperties, +} + +impl BarrierExec { + /// Create a new exec with some number of partitions. + pub fn new(data: Vec>, schema: SchemaRef) -> Self { + // wait for all streams and the input + let barrier = Arc::new(Barrier::new(data.len() + 1)); + let cache = Self::compute_properties(Arc::clone(&schema), &data); + Self { + data, + schema, + barrier, + cache, + } + } + + /// wait until all the input streams and this function is ready + pub async fn wait(&self) { + println!("BarrierExec::wait waiting on barrier"); + self.barrier.wait().await; + println!("BarrierExec::wait done waiting"); + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + data: &[Vec], + ) -> PlanProperties { + PlanProperties::new( + EquivalenceProperties::new(schema), + Partitioning::UnknownPartitioning(data.len()), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } +} + +impl DisplayAs for BarrierExec { + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "BarrierExec") + } + } + } +} + +impl ExecutionPlan for BarrierExec { + 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> { + unimplemented!() + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + unimplemented!() + } + + /// Returns a stream which yields data + fn execute( + &self, + partition: usize, + _context: Arc, + ) -> Result { + assert!(partition < self.data.len()); + + let mut builder = RecordBatchReceiverStream::builder(self.schema(), 2); + + // task simply sends data in order after barrier is reached + let data = self.data[partition].clone(); + let b = Arc::clone(&self.barrier); + let tx = builder.tx(); + builder.spawn(async move { + println!("Partition {partition} waiting on barrier"); + b.wait().await; + for batch in data { + println!("Partition {partition} sending batch"); + if let Err(e) = tx.send(Ok(batch)).await { + println!("ERROR batch via barrier stream stream: {e}"); + } + } + + Ok(()) + }); + + // returned stream simply reads off the rx stream + Ok(builder.build()) + } + + fn statistics(&self) -> Result { + Ok(common::compute_record_batch_statistics( + &self.data, + &self.schema, + None, + )) + } +} + +/// A mock execution plan that errors on a call to execute +#[derive(Debug)] +pub struct ErrorExec { + cache: PlanProperties, +} + +impl Default for ErrorExec { + fn default() -> Self { + Self::new() + } +} + +impl ErrorExec { + pub fn new() -> Self { + let schema = Arc::new(Schema::new(vec![Field::new( + "dummy", + DataType::Int64, + true, + )])); + let cache = Self::compute_properties(schema); + Self { 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(1), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } +} + +impl DisplayAs for ErrorExec { + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "ErrorExec") + } + } + } +} + +impl ExecutionPlan for ErrorExec { + 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> { + unimplemented!() + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + unimplemented!() + } + + /// Returns a stream which yields data + fn execute( + &self, + partition: usize, + _context: Arc, + ) -> Result { + internal_err!("ErrorExec, unsurprisingly, errored in partition {partition}") + } +} + +/// 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()) + } +} + +/// Execution plan that emits streams that block forever. +/// +/// This is useful to test shutdown / cancellation behavior of certain execution plans. +#[derive(Debug)] +pub struct BlockingExec { + /// Schema that is mocked by this plan. + schema: SchemaRef, + + /// Ref-counting helper to check if the plan and the produced stream are still in memory. + refs: Arc<()>, + cache: PlanProperties, +} + +impl BlockingExec { + /// Create new [`BlockingExec`] with a give schema and number of partitions. + pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { + let cache = Self::compute_properties(Arc::clone(&schema), n_partitions); + Self { + schema, + refs: Default::default(), + cache, + } + } + + /// Weak pointer that can be used for ref-counting this execution plan and its streams. + /// + /// Use [`Weak::strong_count`] to determine if the plan itself and its streams are dropped (should be 0 in that + /// case). Note that tokio might take some time to cancel spawned tasks, so you need to wrap this check into a retry + /// loop. Use [`assert_strong_count_converges_to_zero`] to archive this. + pub fn refs(&self) -> Weak<()> { + Arc::downgrade(&self.refs) + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(schema: SchemaRef, n_partitions: usize) -> PlanProperties { + PlanProperties::new( + EquivalenceProperties::new(schema), + Partitioning::UnknownPartitioning(n_partitions), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } +} + +impl DisplayAs for BlockingExec { + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "BlockingExec",) + } + } + } +} + +impl ExecutionPlan for BlockingExec { + 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> { + // this is a leaf node and has no children + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + internal_err!("Children cannot be replaced in {self:?}") + } + + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> Result { + Ok(Box::pin(BlockingStream { + schema: Arc::clone(&self.schema), + _refs: Arc::clone(&self.refs), + })) + } +} + +/// A [`RecordBatchStream`] that is pending forever. +#[derive(Debug)] +pub struct BlockingStream { + /// Schema mocked by this stream. + schema: SchemaRef, + + /// Ref-counting helper to check if the stream are still in memory. + _refs: Arc<()>, +} + +impl Stream for BlockingStream { + type Item = Result; + + fn poll_next( + self: Pin<&mut Self>, + _cx: &mut Context<'_>, + ) -> Poll> { + Poll::Pending + } +} + +impl RecordBatchStream for BlockingStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} + +/// Asserts that the strong count of the given [`Weak`] pointer converges to zero. +/// +/// This might take a while but has a timeout. +pub async fn assert_strong_count_converges_to_zero(refs: Weak) { + tokio::time::timeout(std::time::Duration::from_secs(10), async { + loop { + if dbg!(Weak::strong_count(&refs)) == 0 { + break; + } + tokio::time::sleep(std::time::Duration::from_millis(10)).await; + } + }) + .await + .unwrap(); +} + +/// Execution plan that emits streams that panics. +/// +/// This is useful to test panic handling of certain execution plans. +#[derive(Debug)] +pub struct PanicExec { + /// Schema that is mocked by this plan. + schema: SchemaRef, + + /// Number of output partitions. Each partition will produce this + /// many empty output record batches prior to panicking + batches_until_panics: Vec, + cache: PlanProperties, +} + +impl PanicExec { + /// Create new [`PanicExec`] with a give schema and number of + /// partitions, which will each panic immediately. + pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { + let batches_until_panics = vec![0; n_partitions]; + let cache = Self::compute_properties(Arc::clone(&schema), &batches_until_panics); + Self { + schema, + batches_until_panics, + cache, + } + } + + /// Set the number of batches prior to panic for a partition + pub fn with_partition_panic(mut self, partition: usize, count: usize) -> Self { + self.batches_until_panics[partition] = count; + self + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + schema: SchemaRef, + batches_until_panics: &[usize], + ) -> PlanProperties { + let num_partitions = batches_until_panics.len(); + PlanProperties::new( + EquivalenceProperties::new(schema), + Partitioning::UnknownPartitioning(num_partitions), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } +} + +impl DisplayAs for PanicExec { + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!(f, "PanicExec",) + } + } + } +} + +impl ExecutionPlan for PanicExec { + 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> { + // this is a leaf node and has no children + vec![] + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + internal_err!("Children cannot be replaced in {:?}", self) + } + + fn execute( + &self, + partition: usize, + _context: Arc, + ) -> Result { + Ok(Box::pin(PanicStream { + partition, + batches_until_panic: self.batches_until_panics[partition], + schema: Arc::clone(&self.schema), + ready: false, + })) + } +} + +/// A [`RecordBatchStream`] that yields every other batch and panics +/// after `batches_until_panic` batches have been produced. +/// +/// Useful for testing the behavior of streams on panic +#[derive(Debug)] +struct PanicStream { + /// Which partition was this + partition: usize, + /// How may batches will be produced until panic + batches_until_panic: usize, + /// Schema mocked by this stream. + schema: SchemaRef, + /// Should we return ready ? + ready: bool, +} + +impl Stream for PanicStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + if self.batches_until_panic > 0 { + if self.ready { + self.batches_until_panic -= 1; + self.ready = false; + let batch = RecordBatch::new_empty(Arc::clone(&self.schema)); + return Poll::Ready(Some(Ok(batch))); + } else { + self.ready = true; + // get called again + cx.waker().wake_by_ref(); + return Poll::Pending; + } + } + panic!("PanickingStream did panic: {}", self.partition) + } +} + +impl RecordBatchStream for PanicStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} diff --git a/datafusion/datasource/src/values.rs b/datafusion/datasource/src/values.rs index ae677b4a768a..2279f2d894ce 100644 --- a/datafusion/datasource/src/values.rs +++ b/datafusion/datasource/src/values.rs @@ -232,7 +232,7 @@ impl ExecutionPlan for ValuesExec { mod tests { use super::*; use datafusion_physical_plan::expressions::lit; - use datafusion_physical_plan::test::{self, make_partition}; + use crate::test::{self, make_partition}; use arrow::datatypes::{DataType, Field}; use datafusion_common::stats::{ColumnStatistics, Precision}; From 731e65ae327e244140ecd5fd1f95bf65c198ca59 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Sun, 16 Feb 2025 23:05:02 +0530 Subject: [PATCH 04/18] Mock MemorySourceConfig and DataSource --- .../physical-plan/src/aggregates/mod.rs | 14 +- datafusion/physical-plan/src/filter.rs | 66 +- .../physical-plan/src/joins/hash_join.rs | 26 +- .../src/joins/nested_loop_join.rs | 8 +- .../src/joins/sort_merge_join.rs | 14 +- .../physical-plan/src/joins/test_utils.rs | 12 +- .../physical-plan/src/repartition/mod.rs | 17 +- .../physical-plan/src/sorts/partial_sort.rs | 8 +- datafusion/physical-plan/src/sorts/sort.rs | 8 +- .../src/sorts/sort_preserving_merge.rs | 20 +- datafusion/physical-plan/src/test.rs | 689 +++++++++++++++++- datafusion/physical-plan/src/union.rs | 12 +- .../src/windows/bounded_window_agg_exec.rs | 4 +- 13 files changed, 783 insertions(+), 115 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 85b41da85742..e23ade1d63f4 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1347,7 +1347,7 @@ mod tests { use crate::common::collect; use crate::execution_plan::Boundedness; use crate::expressions::col; - use crate::memory::MemorySourceConfig; + use crate::test::MockMemorySourceConfig; use crate::metrics::MetricValue; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; @@ -2207,7 +2207,7 @@ mod tests { vec![test_last_value_agg_expr(&schema, sort_options)?] }; - let memory_exec = MemorySourceConfig::try_new_exec( + let memory_exec = MockMemorySourceConfig::try_new_exec( &[ vec![partition1], vec![partition2], @@ -2442,7 +2442,7 @@ mod tests { }) .collect(); - let input = MemorySourceConfig::try_new_exec( + let input = MockMemorySourceConfig::try_new_exec( &[input_batches], Arc::clone(&schema), None, @@ -2557,7 +2557,7 @@ mod tests { .build() .map(Arc::new)?]; - let input = MemorySourceConfig::try_new_exec( + let input = MockMemorySourceConfig::try_new_exec( &[vec![batch.clone()]], Arc::::clone(&batch.schema()), None, @@ -2627,7 +2627,7 @@ mod tests { ]; let input = - MemorySourceConfig::try_new_exec(&[input_data], Arc::clone(&schema), None)?; + MockMemorySourceConfig::try_new_exec(&[input_data], Arc::clone(&schema), None)?; let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, group_by, @@ -2714,7 +2714,7 @@ mod tests { ]; let input = - MemorySourceConfig::try_new_exec(&[input_data], Arc::clone(&schema), None)?; + MockMemorySourceConfig::try_new_exec(&[input_data], Arc::clone(&schema), None)?; let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, group_by, @@ -2829,7 +2829,7 @@ mod tests { create_record_batch(&schema, (vec![2, 3, 4, 4], vec![1.0, 2.0, 3.0, 4.0]))?, ]; let plan: Arc = - MemorySourceConfig::try_new_exec(&[batches], Arc::clone(&schema), None)?; + MockMemorySourceConfig::try_new_exec(&[batches], Arc::clone(&schema), None)?; let grouping_set = PhysicalGroupBy::new( vec![(col("a", &schema)?, "a".to_string())], diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 5866f0938e41..60e737834dd9 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -620,7 +620,7 @@ pub type EqualAndNonEqual<'a> = #[cfg(test)] mod tests { use super::*; - use crate::empty::EmptyExec; + // use crate::empty::EmptyExec; use crate::expressions::*; use crate::test; use crate::test::exec::StatisticsExec; @@ -1281,36 +1281,36 @@ mod tests { Ok(()) } - #[test] - fn test_equivalence_properties_union_type() -> Result<()> { - let union_type = DataType::Union( - UnionFields::new( - vec![0, 1], - vec![ - Field::new("f1", DataType::Int32, true), - Field::new("f2", DataType::Utf8, true), - ], - ), - UnionMode::Sparse, - ); - - let schema = Arc::new(Schema::new(vec![ - Field::new("c1", DataType::Int32, true), - Field::new("c2", union_type, true), - ])); - - let exec = FilterExec::try_new( - binary( - binary(col("c1", &schema)?, Operator::GtEq, lit(1i32), &schema)?, - Operator::And, - binary(col("c1", &schema)?, Operator::LtEq, lit(4i32), &schema)?, - &schema, - )?, - Arc::new(EmptyExec::new(Arc::clone(&schema))), - )?; - - exec.statistics().unwrap(); - - Ok(()) - } + // #[test] + // fn test_equivalence_properties_union_type() -> Result<()> { + // let union_type = DataType::Union( + // UnionFields::new( + // vec![0, 1], + // vec![ + // Field::new("f1", DataType::Int32, true), + // Field::new("f2", DataType::Utf8, true), + // ], + // ), + // UnionMode::Sparse, + // ); + + // let schema = Arc::new(Schema::new(vec![ + // Field::new("c1", DataType::Int32, true), + // Field::new("c2", union_type, true), + // ])); + + // let exec = FilterExec::try_new( + // binary( + // binary(col("c1", &schema)?, Operator::GtEq, lit(1i32), &schema)?, + // Operator::And, + // binary(col("c1", &schema)?, Operator::LtEq, lit(4i32), &schema)?, + // &schema, + // )?, + // Arc::new(EmptyExec::new(Arc::clone(&schema))), + // )?; + + // exec.statistics().unwrap(); + + // Ok(()) + // } } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 2983478ada74..3fcf4ea49f95 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -1638,7 +1638,7 @@ impl EmbeddedProjection for HashJoinExec { #[cfg(test)] mod tests { use super::*; - use crate::memory::MemorySourceConfig; + use crate::test::MockMemorySourceConfig; use crate::{ common, expressions::Column, repartition::RepartitionExec, test::build_table_i32, test::exec::MockExec, @@ -1680,7 +1680,7 @@ mod tests { ) -> Arc { let batch = build_table_i32(a, b, c); let schema = batch.schema(); - MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } fn join( @@ -2083,7 +2083,7 @@ mod tests { build_table_i32(("a1", &vec![2]), ("b2", &vec![2]), ("c1", &vec![9])); let schema = batch1.schema(); let left = - MemorySourceConfig::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) + MockMemorySourceConfig::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) .unwrap(); let right = build_table( @@ -2155,7 +2155,7 @@ mod tests { let schema = batch1.schema(); let left = - MemorySourceConfig::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) + MockMemorySourceConfig::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) .unwrap(); let right = build_table( ("a2", &vec![20, 30, 10]), @@ -2209,7 +2209,7 @@ mod tests { build_table_i32(("a2", &vec![30]), ("b1", &vec![5]), ("c2", &vec![90])); let schema = batch1.schema(); let right = - MemorySourceConfig::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) + MockMemorySourceConfig::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) .unwrap(); let on = vec![( @@ -2288,7 +2288,7 @@ mod tests { ) -> Arc { let batch = build_table_i32(a, b, c); let schema = batch.schema(); - MemorySourceConfig::try_new_exec(&[vec![batch.clone(), batch]], schema, None) + MockMemorySourceConfig::try_new_exec(&[vec![batch.clone(), batch]], schema, None) .unwrap() } @@ -2395,7 +2395,7 @@ mod tests { )]; let schema = right.schema(); let right = - MemorySourceConfig::try_new_exec(&[vec![right]], schema, None).unwrap(); + MockMemorySourceConfig::try_new_exec(&[vec![right]], schema, None).unwrap(); let join = join(left, right, on, &JoinType::Left, false).unwrap(); let columns = columns(&join.schema()); @@ -2433,7 +2433,7 @@ mod tests { )]; let schema = right.schema(); let right = - MemorySourceConfig::try_new_exec(&[vec![right]], schema, None).unwrap(); + MockMemorySourceConfig::try_new_exec(&[vec![right]], schema, None).unwrap(); let join = join(left, right, on, &JoinType::Full, false).unwrap(); let columns = columns(&join.schema()); @@ -3738,13 +3738,13 @@ mod tests { let n: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![dates, n])?; let left = - MemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) + MockMemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) .unwrap(); let dates: ArrayRef = Arc::new(Date32Array::from(vec![19108, 19108, 19109])); let n: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6])); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![dates, n])?; let right = - MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap(); + MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap(); let on = vec![( Arc::new(Column::new_with_schema("date", &left.schema()).unwrap()) as _, Arc::new(Column::new_with_schema("date", &right.schema()).unwrap()) as _, @@ -4034,7 +4034,7 @@ mod tests { ("b1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]), ("c1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]), ); - let left = MemorySourceConfig::try_new_exec( + let left = MockMemorySourceConfig::try_new_exec( &[vec![left_batch.clone()], vec![left_batch.clone()]], left_batch.schema(), None, @@ -4045,7 +4045,7 @@ mod tests { ("b2", &vec![12, 13]), ("c2", &vec![14, 15]), ); - let right = MemorySourceConfig::try_new_exec( + let right = MockMemorySourceConfig::try_new_exec( &[vec![right_batch.clone()], vec![right_batch.clone()]], right_batch.schema(), None, @@ -4130,7 +4130,7 @@ mod tests { ) .unwrap(); let schema_ref = batch.schema(); - MemorySourceConfig::try_new_exec(&[vec![batch]], schema_ref, None).unwrap() + MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema_ref, None).unwrap() } #[tokio::test] diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 6de6b3b4dff4..053a329adb51 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -1030,8 +1030,8 @@ impl EmbeddedProjection for NestedLoopJoinExec { #[cfg(test)] pub(crate) mod tests { use super::*; - use crate::memory::MemorySourceConfig; - use crate::source::DataSourceExec; + use crate::test::MockMemorySourceConfig; + use crate::test::MockDataSourceExec; use crate::{ common, expressions::Column, repartition::RepartitionExec, test::build_table_i32, }; @@ -1072,7 +1072,7 @@ pub(crate) mod tests { }; let mut source = - MemorySourceConfig::try_new(&[batches], Arc::clone(&schema), None).unwrap(); + MockMemorySourceConfig::try_new(&[batches], Arc::clone(&schema), None).unwrap(); if !sorted_column_names.is_empty() { let mut sort_info = LexOrdering::default(); for name in sorted_column_names { @@ -1089,7 +1089,7 @@ pub(crate) mod tests { source = source.try_with_sort_information(vec![sort_info]).unwrap(); } - Arc::new(DataSourceExec::new(Arc::new(source))) + Arc::new(MockDataSourceExec::new(Arc::new(source))) } fn build_left_table() -> Arc { diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index a3e835c64131..3ef774917d5b 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -2547,7 +2547,7 @@ mod tests { use crate::joins::sort_merge_join::{get_corrected_filter_mask, JoinedRecordBatches}; use crate::joins::utils::{ColumnIndex, JoinFilter, JoinOn}; use crate::joins::SortMergeJoinExec; - use crate::memory::MemorySourceConfig; + use crate::test::MockMemorySourceConfig; use crate::test::{build_table_i32, build_table_i32_two_cols}; use crate::{common, ExecutionPlan}; @@ -2558,12 +2558,12 @@ mod tests { ) -> Arc { let batch = build_table_i32(a, b, c); let schema = batch.schema(); - MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } fn build_table_from_batches(batches: Vec) -> Arc { let schema = batches.first().unwrap().schema(); - MemorySourceConfig::try_new_exec(&[batches], schema, None).unwrap() + MockMemorySourceConfig::try_new_exec(&[batches], schema, None).unwrap() } fn build_date_table( @@ -2588,7 +2588,7 @@ mod tests { .unwrap(); let schema = batch.schema(); - MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } fn build_date64_table( @@ -2613,7 +2613,7 @@ mod tests { .unwrap(); let schema = batch.schema(); - MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } /// returns a table with 3 columns of i32 in memory @@ -2636,7 +2636,7 @@ mod tests { ], ) .unwrap(); - MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } pub fn build_table_two_cols( @@ -2645,7 +2645,7 @@ mod tests { ) -> Arc { let batch = build_table_i32_two_cols(a, b); let schema = batch.schema(); - MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } fn join( diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index 9932c647be0a..f609108f8e28 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -23,9 +23,9 @@ use crate::joins::utils::{JoinFilter, JoinOn}; use crate::joins::{ HashJoinExec, PartitionMode, StreamJoinPartitionMode, SymmetricHashJoinExec, }; -use crate::memory::MemorySourceConfig; +use crate::test::MockMemorySourceConfig; use crate::repartition::RepartitionExec; -use crate::source::DataSourceExec; +use crate::test::MockDataSourceExec; use crate::{common, ExecutionPlan, ExecutionPlanProperties, Partitioning}; use arrow::array::{ @@ -530,14 +530,14 @@ pub fn create_memory_table( right_sorted: Vec, ) -> Result<(Arc, Arc)> { let left_schema = left_partition[0].schema(); - let left = MemorySourceConfig::try_new(&[left_partition], left_schema, None)? + let left = MockMemorySourceConfig::try_new(&[left_partition], left_schema, None)? .try_with_sort_information(left_sorted)?; let right_schema = right_partition[0].schema(); - let right = MemorySourceConfig::try_new(&[right_partition], right_schema, None)? + let right = MockMemorySourceConfig::try_new(&[right_partition], right_schema, None)? .try_with_sort_information(right_sorted)?; Ok(( - Arc::new(DataSourceExec::new(Arc::new(left))), - Arc::new(DataSourceExec::new(Arc::new(right))), + Arc::new(MockDataSourceExec::new(Arc::new(left))), + Arc::new(MockDataSourceExec::new(Arc::new(right))), )) } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 25668fa67d5b..f45529958cbd 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1059,8 +1059,9 @@ mod tests { ErrorExec, MockExec, }, }, - {collect, expressions::col, memory::MemorySourceConfig}, + {collect, expressions::col}, }; + use crate::test::MockMemorySourceConfig; use arrow::array::{ArrayRef, StringArray, UInt32Array}; use arrow::datatypes::{DataType, Field, Schema}; @@ -1164,7 +1165,7 @@ mod tests { ) -> Result>> { let task_ctx = Arc::new(TaskContext::default()); // create physical plan - let exec = MemorySourceConfig::try_new_exec( + let exec = MockMemorySourceConfig::try_new_exec( &input_partitions, Arc::clone(schema), None, @@ -1559,7 +1560,7 @@ mod tests { let task_ctx = Arc::new(task_ctx); // create physical plan - let exec = MemorySourceConfig::try_new_exec( + let exec = MockMemorySourceConfig::try_new_exec( &input_partitions, Arc::clone(&schema), None, @@ -1604,8 +1605,8 @@ mod test { use arrow::datatypes::{DataType, Field, Schema}; use super::*; - use crate::memory::MemorySourceConfig; - use crate::source::DataSourceExec; + use crate::test::MockMemorySourceConfig; + use crate::test::MockDataSourceExec; use crate::union::UnionExec; use datafusion_physical_expr::expressions::col; @@ -1711,15 +1712,15 @@ mod test { } fn memory_exec(schema: &SchemaRef) -> Arc { - MemorySourceConfig::try_new_exec(&[vec![]], Arc::clone(schema), None).unwrap() + MockMemorySourceConfig::try_new_exec(&[vec![]], Arc::clone(schema), None).unwrap() } fn sorted_memory_exec( schema: &SchemaRef, sort_exprs: LexOrdering, ) -> Arc { - Arc::new(DataSourceExec::new(Arc::new( - MemorySourceConfig::try_new(&[vec![]], Arc::clone(schema), None) + Arc::new(MockDataSourceExec::new(Arc::new( + MockMemorySourceConfig::try_new(&[vec![]], Arc::clone(schema), None) .unwrap() .try_with_sort_information(vec![sort_exprs]) .unwrap(), diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index eeef73c45fc4..655275cf4670 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -466,7 +466,7 @@ mod tests { use crate::collect; use crate::expressions::col; use crate::expressions::PhysicalSortExpr; - use crate::memory::MemorySourceConfig; + use crate::test::MockMemorySourceConfig; use crate::sorts::sort::SortExec; use crate::test; use crate::test::assert_is_pending; @@ -696,7 +696,7 @@ mod tests { ); let schema = batch1.schema(); - MemorySourceConfig::try_new_exec( + MockMemorySourceConfig::try_new_exec( &[vec![batch1, batch2, batch3, batch4]], Arc::clone(&schema), None, @@ -881,7 +881,7 @@ mod tests { let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data])?; let input = - MemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None)?; + MockMemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None)?; let partial_sort_exec = Arc::new(PartialSortExec::new( LexOrdering::new(vec![PhysicalSortExpr { @@ -987,7 +987,7 @@ mod tests { options: option_desc, }, ]), - MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None)?, + MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None)?, 2, )); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 9f7e82f026bd..cd58e178edf1 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1067,7 +1067,7 @@ mod tests { use crate::collect; use crate::execution_plan::Boundedness; use crate::expressions::col; - use crate::memory::MemorySourceConfig; + use crate::test::MockMemorySourceConfig; use crate::test; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; @@ -1375,7 +1375,7 @@ mod tests { let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data]).unwrap(); let input = - MemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) + MockMemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) .unwrap(); let sort_exec = Arc::new(SortExec::new( @@ -1446,7 +1446,7 @@ mod tests { }, }, ]), - MemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None)?, + MockMemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None)?, )); assert_eq!(DataType::Int32, *sort_exec.schema().field(0).data_type()); @@ -1532,7 +1532,7 @@ mod tests { }, }, ]), - MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None)?, + MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None)?, )); assert_eq!(DataType::Float32, *sort_exec.schema().field(0).data_type()); diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 2cc55d60292a..eb70376b77df 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -384,7 +384,7 @@ mod tests { use crate::coalesce_partitions::CoalescePartitionsExec; use crate::execution_plan::{Boundedness, EmissionType}; use crate::expressions::col; - use crate::memory::MemorySourceConfig; + use crate::test::MockMemorySourceConfig; use crate::metrics::{MetricValue, Timestamp}; use crate::repartition::RepartitionExec; use crate::sorts::sort::SortExec; @@ -451,7 +451,7 @@ mod tests { ]); let repartition_exec = RepartitionExec::try_new( - MemorySourceConfig::try_new_exec(&[rbs], schema, None).unwrap(), + MockMemorySourceConfig::try_new_exec(&[rbs], schema, None).unwrap(), Partitioning::RoundRobinBatch(2), )?; let coalesce_batches_exec = @@ -543,7 +543,7 @@ mod tests { let schema = batch.schema(); let sort = LexOrdering::default(); // no sort expressions - let exec = MemorySourceConfig::try_new_exec( + let exec = MockMemorySourceConfig::try_new_exec( &[vec![batch.clone()], vec![batch]], schema, None, @@ -736,7 +736,7 @@ mod tests { options: Default::default(), }, ]); - let exec = MemorySourceConfig::try_new_exec(partitions, schema, None).unwrap(); + let exec = MockMemorySourceConfig::try_new_exec(partitions, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, context).await.unwrap(); @@ -844,7 +844,7 @@ mod tests { let sorted = basic_sort(csv, sort, context).await; let split: Vec<_> = sizes.iter().map(|x| split_batch(&sorted, *x)).collect(); - Ok(MemorySourceConfig::try_new_exec(&split, sorted.schema(), None).unwrap()) + Ok(MockMemorySourceConfig::try_new_exec(&split, sorted.schema(), None).unwrap()) } #[tokio::test] @@ -972,7 +972,7 @@ mod tests { }, }, ]); - let exec = MemorySourceConfig::try_new_exec(&[vec![b1], vec![b2]], schema, None) + let exec = MockMemorySourceConfig::try_new_exec(&[vec![b1], vec![b2]], schema, None) .unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); @@ -1016,7 +1016,7 @@ mod tests { }, }]); let exec = - MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap(); + MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec).with_fetch(Some(2))); @@ -1052,7 +1052,7 @@ mod tests { }, }]); let exec = - MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap(); + MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, task_ctx).await.unwrap(); @@ -1161,7 +1161,7 @@ mod tests { expr: col("b", &schema).unwrap(), options: Default::default(), }]); - let exec = MemorySourceConfig::try_new_exec(&[vec![b1], vec![b2]], schema, None) + let exec = MockMemorySourceConfig::try_new_exec(&[vec![b1], vec![b2]], schema, None) .unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); @@ -1273,7 +1273,7 @@ mod tests { }, }]); - let exec = MemorySourceConfig::try_new_exec(&partitions, schema, None).unwrap(); + let exec = MockMemorySourceConfig::try_new_exec(&partitions, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, task_ctx).await.unwrap(); diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index e73d6d97e986..0d7f350c8a1e 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -20,24 +20,691 @@ use std::collections::HashMap; use std::pin::Pin; use std::sync::Arc; - -use arrow::array::{ArrayRef, Int32Array, RecordBatch}; +use std::fmt; +use arrow::array::{ArrayRef, Int32Array, RecordBatch, RecordBatchOptions}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::utils::collect_columns; +use datafusion_physical_expr::LexOrdering; use futures::{Future, FutureExt}; -use crate::memory::MemorySourceConfig; -use crate::source::DataSourceExec; + +use std::task::{Context, Poll}; + +use crate::projection::{ + all_alias_free_columns, new_projections_for_columns, +}; +use crate::{ + common, ColumnarValue, + PhysicalExpr, RecordBatchStream, +}; + +use datafusion_common::{ + plan_err, project_schema, Result, +}; +use datafusion_execution::memory_pool::MemoryReservation; +use datafusion_physical_expr::equivalence::ProjectionMapping; +use datafusion_physical_expr::expressions::Column; + +use futures::Stream; + +use crate::projection::ProjectionExec; +// use crate::memory::MockMemorySourceConfig; +// use crate::source::MockDataSourceExec; use crate::stream::RecordBatchStreamAdapter; use crate::streaming::PartitionStream; use crate::ExecutionPlan; pub mod exec; +use std::any::Any; +use std::fmt::{Debug, Formatter}; + +use crate::execution_plan::{Boundedness, EmissionType}; +use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use crate::{ + DisplayAs, DisplayFormatType, PlanProperties, +}; + +use datafusion_common::config::ConfigOptions; +use datafusion_common::{internal_err, Constraints, ScalarValue, Statistics}; +use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; + +/// Common behaviors in Data Sources for both from Files and MockMemory. +/// See `MockDataSourceExec` for physical plan implementation +pub trait MockDataSource: Send + Sync { + fn open( + &self, + partition: usize, + context: Arc, + ) -> Result; + fn as_any(&self) -> &dyn Any; + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result; + fn repartitioned( + &self, + _target_partitions: usize, + _repartition_file_min_size: usize, + _output_ordering: Option, + ) -> Result>> { + Ok(None) + } + + fn output_partitioning(&self) -> Partitioning; + fn eq_properties(&self) -> EquivalenceProperties; + fn statistics(&self) -> Result; + fn with_fetch(&self, _limit: Option) -> Option>; + fn fetch(&self) -> Option; + fn metrics(&self) -> ExecutionPlanMetricsSet { + ExecutionPlanMetricsSet::new() + } + fn try_swapping_with_projection( + &self, + _projection: &ProjectionExec, + ) -> Result>>; +} + +impl Debug for dyn MockDataSource { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + write!(f, "MockDataSource: ") + } +} + +/// Unified data source for file formats like JSON, CSV, AVRO, ARROW, PARQUET +#[derive(Clone, Debug)] +pub struct MockDataSourceExec { + source: Arc, + cache: PlanProperties, +} + +impl DisplayAs for MockDataSourceExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { + write!(f, "MockDataSourceExec: ")?; + self.source.fmt_as(t, f) + } +} + +impl ExecutionPlan for MockDataSourceExec { + fn name(&self) -> &'static str { + "MockDataSourceExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn children(&self) -> Vec<&Arc> { + Vec::new() + } + + fn with_new_children( + self: Arc, + _: Vec>, + ) -> Result> { + Ok(self) + } + + fn repartitioned( + &self, + target_partitions: usize, + config: &ConfigOptions, + ) -> Result>> { + let source = self.source.repartitioned( + target_partitions, + config.optimizer.repartition_file_min_size, + self.properties().eq_properties.output_ordering(), + )?; + + if let Some(source) = source { + let output_partitioning = source.output_partitioning(); + let plan = self + .clone() + .with_source(source) + // Changing source partitioning may invalidate output partitioning. Update it also + .with_partitioning(output_partitioning); + Ok(Some(Arc::new(plan))) + } else { + Ok(Some(Arc::new(self.clone()))) + } + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + self.source.open(partition, context) + } + + fn metrics(&self) -> Option { + Some(self.source.metrics().clone_inner()) + } + + fn statistics(&self) -> Result { + self.source.statistics() + } + + fn with_fetch(&self, limit: Option) -> Option> { + let mut source = Arc::clone(&self.source); + source = source.with_fetch(limit)?; + let cache = self.cache.clone(); + + Some(Arc::new(Self { source, cache })) + } + + fn fetch(&self) -> Option { + self.source.fetch() + } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + self.source.try_swapping_with_projection(projection) + } + +} + +impl MockDataSourceExec { + pub fn new(source: Arc) -> Self { + let cache = Self::compute_properties(Arc::clone(&source)); + Self { source, cache } + } + + /// Return the source object + pub fn source(&self) -> &Arc { + &self.source + } + + pub fn with_source(mut self, source: Arc) -> Self { + self.cache = Self::compute_properties(Arc::clone(&source)); + self.source = source; + self + } + + /// Assign constraints + pub fn with_constraints(mut self, constraints: Constraints) -> Self { + self.cache = self.cache.with_constraints(constraints); + self + } + + /// Assign output partitioning + pub fn with_partitioning(mut self, partitioning: Partitioning) -> Self { + self.cache = self.cache.with_partitioning(partitioning); + self + } + + fn compute_properties(source: Arc) -> PlanProperties { + PlanProperties::new( + source.eq_properties(), + source.output_partitioning(), + EmissionType::Incremental, + Boundedness::Bounded, + ) + } +} + +/// Data source configuration for reading in-memory batches of data +#[derive(Clone)] +pub struct MockMemorySourceConfig { + /// The partitions to query + partitions: Vec>, + /// Schema representing the data before projection + schema: SchemaRef, + /// Schema representing the data after the optional projection is applied + projected_schema: SchemaRef, + /// Optional projection + projection: Option>, + /// Sort information: one or more equivalent orderings + sort_information: Vec, + /// if partition sizes should be displayed + show_sizes: bool, + /// The maximum number of records to read from this plan. If `None`, + /// all records after filtering are returned. + fetch: Option, +} + +impl MockDataSource for MockMemorySourceConfig { + fn open( + &self, + partition: usize, + _context: Arc, + ) -> Result { + Ok(Box::pin( + MockMemoryStream::try_new( + self.partitions[partition].clone(), + Arc::clone(&self.projected_schema), + self.projection.clone(), + )? + .with_fetch(self.fetch), + )) + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + let partition_sizes: Vec<_> = + self.partitions.iter().map(|b| b.len()).collect(); + + let output_ordering = self + .sort_information + .first() + .map(|output_ordering| { + format!(", output_ordering={}", output_ordering) + }) + .unwrap_or_default(); + + let eq_properties = self.eq_properties(); + let constraints = eq_properties.constraints(); + let constraints = if constraints.is_empty() { + String::new() + } else { + format!(", {}", constraints) + }; + + let limit = self + .fetch + .map_or(String::new(), |limit| format!(", fetch={}", limit)); + if self.show_sizes { + write!( + f, + "partitions={}, partition_sizes={partition_sizes:?}{limit}{output_ordering}{constraints}", + partition_sizes.len(), + ) + } else { + write!( + f, + "partitions={}{limit}{output_ordering}{constraints}", + partition_sizes.len(), + ) + } + } + } + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.partitions.len()) + } + + fn eq_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( + Arc::clone(&self.projected_schema), + self.sort_information.as_slice(), + ) + } + + fn statistics(&self) -> Result { + Ok(common::compute_record_batch_statistics( + &self.partitions, + &self.schema, + self.projection.clone(), + )) + } + + fn with_fetch(&self, limit: Option) -> Option> { + let source = self.clone(); + Some(Arc::new(source.with_limit(limit))) + } + + fn fetch(&self) -> Option { + self.fetch + } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + // If there is any non-column or alias-carrier expression, Projection should not be removed. + // This process can be moved into MockMemoryExec, but it would be an overlap of their responsibility. + all_alias_free_columns(projection.expr()) + .then(|| { + let all_projections = (0..self.schema.fields().len()).collect(); + let new_projections = new_projections_for_columns( + projection, + self.projection().as_ref().unwrap_or(&all_projections), + ); + + MockMemorySourceConfig::try_new_exec( + self.partitions(), + self.original_schema(), + Some(new_projections), + ) + .map(|e| e as _) + }) + .transpose() + } +} + +impl MockMemorySourceConfig { + /// Create a new `MockMemorySourceConfig` for reading in-memory record batches + /// The provided `schema` should not have the projection applied. + pub fn try_new( + partitions: &[Vec], + schema: SchemaRef, + projection: Option>, + ) -> Result { + let projected_schema = project_schema(&schema, projection.as_ref())?; + Ok(Self { + partitions: partitions.to_vec(), + schema, + projected_schema, + projection, + sort_information: vec![], + show_sizes: true, + fetch: None, + }) + } + + /// Create a new `DataSourceExec` plan for reading in-memory record batches + /// The provided `schema` should not have the projection applied. + pub fn try_new_exec( + partitions: &[Vec], + schema: SchemaRef, + projection: Option>, + ) -> Result> { + let source = Self::try_new(partitions, schema, projection)?; + Ok(Arc::new(MockDataSourceExec::new(Arc::new(source)))) + } + + /// Create a new execution plan from a list of constant values (`ValuesExec`) + pub fn try_new_as_values( + schema: SchemaRef, + data: Vec>>, + ) -> Result> { + if data.is_empty() { + return plan_err!("Values list cannot be empty"); + } + + let n_row = data.len(); + let n_col = schema.fields().len(); + + // We have this single row batch as a placeholder to satisfy evaluation argument + // and generate a single output row + let placeholder_schema = Arc::new(Schema::empty()); + let placeholder_batch = RecordBatch::try_new_with_options( + Arc::clone(&placeholder_schema), + vec![], + &RecordBatchOptions::new().with_row_count(Some(1)), + )?; + + // Evaluate each column + let arrays = (0..n_col) + .map(|j| { + (0..n_row) + .map(|i| { + let expr = &data[i][j]; + let result = expr.evaluate(&placeholder_batch)?; + + match result { + ColumnarValue::Scalar(scalar) => Ok(scalar), + ColumnarValue::Array(array) if array.len() == 1 => { + ScalarValue::try_from_array(&array, 0) + } + ColumnarValue::Array(_) => { + plan_err!("Cannot have array values in a values list") + } + } + }) + .collect::>>() + .and_then(ScalarValue::iter_to_array) + }) + .collect::>>()?; + + let batch = RecordBatch::try_new_with_options( + Arc::clone(&schema), + arrays, + &RecordBatchOptions::new().with_row_count(Some(n_row)), + )?; + + let partitions = vec![batch]; + Self::try_new_from_batches(Arc::clone(&schema), partitions) + } + + /// Create a new plan using the provided schema and batches. + /// + /// Errors if any of the batches don't match the provided schema, or if no + /// batches are provided. + pub fn try_new_from_batches( + schema: SchemaRef, + batches: Vec, + ) -> Result> { + if batches.is_empty() { + return plan_err!("Values list cannot be empty"); + } + + for batch in &batches { + let batch_schema = batch.schema(); + if batch_schema != schema { + return plan_err!( + "Batch has invalid schema. Expected: {}, got: {}", + schema, + batch_schema + ); + } + } + + let partitions = vec![batches]; + let source = Self { + partitions, + schema: Arc::clone(&schema), + projected_schema: Arc::clone(&schema), + projection: None, + sort_information: vec![], + show_sizes: true, + fetch: None, + }; + Ok(Arc::new(MockDataSourceExec::new(Arc::new(source)))) + } + + /// Set the limit of the files + pub fn with_limit(mut self, limit: Option) -> Self { + self.fetch = limit; + self + } + + /// Set `show_sizes` to determine whether to display partition sizes + pub fn with_show_sizes(mut self, show_sizes: bool) -> Self { + self.show_sizes = show_sizes; + self + } + + /// Ref to partitions + pub fn partitions(&self) -> &[Vec] { + &self.partitions + } + + /// Ref to projection + pub fn projection(&self) -> &Option> { + &self.projection + } + + /// Show sizes + pub fn show_sizes(&self) -> bool { + self.show_sizes + } + + /// Ref to sort information + pub fn sort_information(&self) -> &[LexOrdering] { + &self.sort_information + } + + /// A memory table can be ordered by multiple expressions simultaneously. + /// [`EquivalenceProperties`] keeps track of expressions that describe the + /// global ordering of the schema. These columns are not necessarily same; e.g. + /// ```text + /// ┌-------┐ + /// | a | b | + /// |---|---| + /// | 1 | 9 | + /// | 2 | 8 | + /// | 3 | 7 | + /// | 5 | 5 | + /// └---┴---┘ + /// ``` + /// where both `a ASC` and `b DESC` can describe the table ordering. With + /// [`EquivalenceProperties`], we can keep track of these equivalences + /// and treat `a ASC` and `b DESC` as the same ordering requirement. + /// + /// Note that if there is an internal projection, that projection will be + /// also applied to the given `sort_information`. + pub fn try_with_sort_information( + mut self, + mut sort_information: Vec, + ) -> Result { + // All sort expressions must refer to the original schema + let fields = self.schema.fields(); + let ambiguous_column = sort_information + .iter() + .flat_map(|ordering| ordering.clone()) + .flat_map(|expr| collect_columns(&expr.expr)) + .find(|col| { + fields + .get(col.index()) + .map(|field| field.name() != col.name()) + .unwrap_or(true) + }); + if let Some(col) = ambiguous_column { + return internal_err!( + "Column {:?} is not found in the original schema of the MockMemorySourceConfig", + col + ); + } + + // If there is a projection on the source, we also need to project orderings + if let Some(projection) = &self.projection { + let base_eqp = EquivalenceProperties::new_with_orderings( + self.original_schema(), + &sort_information, + ); + let proj_exprs = projection + .iter() + .map(|idx| { + let base_schema = self.original_schema(); + let name = base_schema.field(*idx).name(); + (Arc::new(Column::new(name, *idx)) as _, name.to_string()) + }) + .collect::>(); + let projection_mapping = + ProjectionMapping::try_new(&proj_exprs, &self.original_schema())?; + sort_information = base_eqp + .project(&projection_mapping, Arc::clone(&self.projected_schema)) + .into_oeq_class() + .into_inner(); + } + + self.sort_information = sort_information; + Ok(self) + } + + /// Arc clone of ref to original schema + pub fn original_schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} + + +/// Iterator over batches +pub struct MockMemoryStream { + /// Vector of record batches + data: Vec, + /// Optional memory reservation bound to the data, freed on drop + reservation: Option, + /// Schema representing the data + schema: SchemaRef, + /// Optional projection for which columns to load + projection: Option>, + /// Index into the data + index: usize, + /// The remaining number of rows to return. If None, all rows are returned + fetch: Option, +} + +impl MockMemoryStream { + /// Create an iterator for a vector of record batches + pub fn try_new( + data: Vec, + schema: SchemaRef, + projection: Option>, + ) -> Result { + Ok(Self { + data, + reservation: None, + schema, + projection, + index: 0, + fetch: None, + }) + } + + /// Set the memory reservation for the data + pub(super) fn with_reservation(mut self, reservation: MemoryReservation) -> Self { + self.reservation = Some(reservation); + self + } + + /// Set the number of rows to produce + pub(super) fn with_fetch(mut self, fetch: Option) -> Self { + self.fetch = fetch; + self + } +} + +impl Stream for MockMemoryStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + ) -> Poll> { + if self.index >= self.data.len() { + return Poll::Ready(None); + } + self.index += 1; + let batch = &self.data[self.index - 1]; + // return just the columns requested + let batch = match self.projection.as_ref() { + Some(columns) => batch.project(columns)?, + None => batch.clone(), + }; + + let Some(&fetch) = self.fetch.as_ref() else { + return Poll::Ready(Some(Ok(batch))); + }; + if fetch == 0 { + return Poll::Ready(None); + } + + let batch = if batch.num_rows() > fetch { + batch.slice(0, fetch) + } else { + batch + }; + self.fetch = Some(fetch - batch.num_rows()); + Poll::Ready(Some(Ok(batch))) + } + + fn size_hint(&self) -> (usize, Option) { + (self.data.len(), Some(self.data.len())) + } +} + +impl RecordBatchStream for MockMemoryStream { + /// Get the schema + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} /// Asserts that given future is pending. pub fn assert_is_pending<'a, T>(fut: &mut Pin + Send + 'a>>) { let waker = futures::task::noop_waker(); - let mut cx = futures::task::Context::from_waker(&waker); + let mut cx = Context::from_waker(&waker); let poll = fut.poll_unpin(&mut cx); assert!(poll.is_pending()); @@ -117,7 +784,7 @@ pub fn build_table_scan_i32( ) -> Arc { let batch = build_table_i32(a, b, c); let schema = batch.schema(); - MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() } /// Return a RecordBatch with a single Int32 array with values (0..sz) in a field named "i" @@ -132,19 +799,19 @@ pub fn make_partition(sz: i32) -> RecordBatch { RecordBatch::try_new(schema, vec![arr]).unwrap() } -/// Returns a `DataSourceExec` that scans `partitions` of 100 batches each +/// Returns a `MockDataSourceExec` that scans `partitions` of 100 batches each pub fn scan_partitioned(partitions: usize) -> Arc { Arc::new(mem_exec(partitions)) } -/// Returns a `DataSourceExec` that scans `partitions` of 100 batches each -pub fn mem_exec(partitions: usize) -> DataSourceExec { +/// Returns a `MockDataSourceExec` that scans `partitions` of 100 batches each +pub fn mem_exec(partitions: usize) -> MockDataSourceExec { let data: Vec> = (0..partitions).map(|_| vec![make_partition(100)]).collect(); let schema = data[0][0].schema(); let projection = None; - DataSourceExec::new(Arc::new( - MemorySourceConfig::try_new(&data, schema, projection).unwrap(), + MockDataSourceExec::new(Arc::new( + MockMemorySourceConfig::try_new(&data, schema, projection).unwrap(), )) } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index e1972d267b97..652173c2257c 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -640,10 +640,10 @@ fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { mod tests { use super::*; use crate::collect; - use crate::memory::MemorySourceConfig; + use crate::test::MockMemorySourceConfig; use crate::test; - use crate::source::DataSourceExec; + use crate::test::MockDataSourceExec; use arrow::compute::SortOptions; use arrow::datatypes::DataType; use datafusion_common::ScalarValue; @@ -865,12 +865,12 @@ mod tests { .iter() .map(|ordering| convert_to_sort_exprs(ordering)) .collect::>(); - let child1 = Arc::new(DataSourceExec::new(Arc::new( - MemorySourceConfig::try_new(&[], Arc::clone(&schema), None)? + let child1 = Arc::new(MockDataSourceExec::new(Arc::new( + MockMemorySourceConfig::try_new(&[], Arc::clone(&schema), None)? .try_with_sort_information(first_orderings)?, ))); - let child2 = Arc::new(DataSourceExec::new(Arc::new( - MemorySourceConfig::try_new(&[], Arc::clone(&schema), None)? + let child2 = Arc::new(MockDataSourceExec::new(Arc::new( + MockMemorySourceConfig::try_new(&[], Arc::clone(&schema), None)? .try_with_sort_information(second_orderings)?, ))); diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 1e21d0757c41..b973892451d6 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1190,7 +1190,7 @@ mod tests { use crate::common::collect; use crate::expressions::PhysicalSortExpr; - use crate::memory::MemorySourceConfig; + use crate::test::MockMemorySourceConfig; use crate::projection::ProjectionExec; use crate::streaming::{PartitionStream, StreamingTableExec}; use crate::windows::{ @@ -1551,7 +1551,7 @@ mod tests { vec![Arc::new(arrow::array::Int32Array::from(vec![1, 2, 3]))], )?; - let memory_exec = MemorySourceConfig::try_new_exec( + let memory_exec = MockMemorySourceConfig::try_new_exec( &[vec![batch.clone(), batch.clone(), batch.clone()]], Arc::clone(&schema), None, From 33a5408700cdbc3c62bdbabed308bbb7a4ac9915 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Mon, 17 Feb 2025 01:39:37 +0530 Subject: [PATCH 05/18] some cleanup --- .../datasource/physical_plan/parquet/mod.rs | 2 +- .../core/tests/custom_sources_cases/mod.rs | 2 +- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 4 +- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 4 +- .../core/tests/fuzz_cases/merge_fuzz.rs | 2 +- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 2 +- .../sort_preserving_repartition_fuzz.rs | 2 +- .../core/tests/fuzz_cases/window_fuzz.rs | 4 +- datafusion/core/tests/memory_limit/mod.rs | 4 +- .../core/tests/parquet/file_statistics.rs | 2 +- datafusion/core/tests/parquet/page_pruning.rs | 2 +- datafusion/core/tests/parquet/utils.rs | 2 +- .../aggregate_statistics.rs | 4 +- .../enforce_distribution.rs | 2 +- .../physical_optimizer/limit_pushdown.rs | 2 +- .../physical_optimizer/projection_pushdown.rs | 4 +- .../tests/physical_optimizer/test_utils.rs | 4 +- datafusion/core/tests/sql/path_partition.rs | 2 +- .../user_defined_table_functions.rs | 2 +- datafusion/datasource/src/memory.rs | 2 +- datafusion/datasource/src/values.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 16 +- .../physical-plan/src/joins/hash_join.rs | 36 ++- .../src/joins/nested_loop_join.rs | 5 +- .../physical-plan/src/joins/test_utils.rs | 2 +- .../physical-plan/src/repartition/mod.rs | 4 +- .../physical-plan/src/sorts/partial_sort.rs | 9 +- datafusion/physical-plan/src/sorts/sort.rs | 17 +- .../src/sorts/sort_preserving_merge.rs | 18 +- datafusion/physical-plan/src/test.rs | 244 ++---------------- datafusion/physical-plan/src/union.rs | 2 +- .../src/windows/bounded_window_agg_exec.rs | 2 +- 32 files changed, 119 insertions(+), 292 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 47b0bbf49750..cd72faa17a6e 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -579,10 +579,10 @@ mod tests { use arrow::record_batch::RecordBatch; use bytes::{BufMut, BytesMut}; use datafusion_common::{assert_contains, ScalarValue}; + use datafusion_datasource::source::DataSourceExec; use datafusion_expr::{col, lit, when, Expr}; use datafusion_physical_expr::planner::logical2physical; use datafusion_physical_plan::metrics::{ExecutionPlanMetricsSet, MetricsSet}; - use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use crate::datasource::physical_plan::parquet::source::ParquetSource; diff --git a/datafusion/core/tests/custom_sources_cases/mod.rs b/datafusion/core/tests/custom_sources_cases/mod.rs index c4de715a811e..51e459d3dd67 100644 --- a/datafusion/core/tests/custom_sources_cases/mod.rs +++ b/datafusion/core/tests/custom_sources_cases/mod.rs @@ -39,9 +39,9 @@ use datafusion_catalog::Session; use datafusion_common::cast::as_primitive_array; use datafusion_common::project_schema; use datafusion_common::stats::Precision; +use datafusion_datasource::placeholder_row::PlaceholderRowExec; use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_datasource::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::PlanProperties; use async_trait::async_trait; diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index f6286039a7dd..77a2bd440de8 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -39,12 +39,12 @@ use datafusion::physical_plan::{collect, displayable, ExecutionPlan}; use datafusion::prelude::{DataFrame, SessionConfig, SessionContext}; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor}; use datafusion_common::HashMap; +use datafusion_datasource::memory::MemorySourceConfig; +use datafusion_datasource::source::DataSourceExec; use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_datasource::memory::MemorySourceConfig; -use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::InputOrderMode; use test_utils::{add_empty_batches, StringBatchGenerator}; diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index ef9e4a497776..31412d8cbd8f 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -36,10 +36,10 @@ use datafusion::physical_plan::joins::{ }; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::ScalarValue; -use datafusion_physical_expr::expressions::Literal; -use datafusion_physical_expr::PhysicalExprRef; use datafusion_datasource::memory::MemorySourceConfig; use datafusion_datasource::source::DataSourceExec; +use datafusion_physical_expr::expressions::Literal; +use datafusion_physical_expr::PhysicalExprRef; use itertools::Itertools; use rand::Rng; diff --git a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs index a9943ce5a07f..288f081981fe 100644 --- a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs @@ -29,8 +29,8 @@ use datafusion::physical_plan::{ expressions::{col, PhysicalSortExpr}, sorts::sort_preserving_merge::SortPreservingMergeExec, }; -use datafusion_datasource::memory::MemorySourceConfig; use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion_datasource::memory::MemorySourceConfig; use datafusion_physical_expr_common::sort_expr::LexOrdering; use test_utils::{batches_to_vec, partitions_to_sorted_vec, stagger_batch_with_seed}; diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index fefb8f1c188e..4a66274479ce 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -29,10 +29,10 @@ use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::{collect, ExecutionPlan}; use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion_datasource::memory::MemorySourceConfig; use datafusion_execution::memory_pool::GreedyMemoryPool; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_datasource::memory::MemorySourceConfig; use rand::Rng; use test_utils::{batches_to_vec, partitions_to_sorted_vec}; diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 7bf5ceb6ad53..3d52a2b73b73 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -44,9 +44,9 @@ mod sp_repartition_fuzz_tests { }; use test_utils::add_empty_batches; - use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_datasource::memory::MemorySourceConfig; use datafusion_datasource::source::DataSourceExec; + use datafusion_physical_expr_common::sort_expr::LexOrdering; use itertools::izip; use rand::{rngs::StdRng, seq::SliceRandom, Rng, SeedableRng}; diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 1c07e5759db6..c4febbe2b005 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -33,6 +33,8 @@ use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::HashMap; use datafusion_common::{Result, ScalarValue}; use datafusion_common_runtime::SpawnedTask; +use datafusion_datasource::memory::MemorySourceConfig; +use datafusion_datasource::source::DataSourceExec; use datafusion_expr::type_coercion::functions::data_types_with_aggregate_udf; use datafusion_expr::{ WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, @@ -48,8 +50,6 @@ use datafusion_functions_window::rank::{dense_rank_udwf, rank_udwf}; use datafusion_physical_expr::expressions::{cast, col, lit}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_datasource::memory::MemorySourceConfig; -use datafusion_datasource::source::DataSourceExec; use rand::distributions::Alphanumeric; use rand::rngs::StdRng; diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index c3255a25b419..4bf7b57aad14 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -38,6 +38,8 @@ use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_catalog::streaming::StreamingTable; use datafusion_catalog::Session; use datafusion_common::{assert_contains, Result}; +use datafusion_datasource::memory::MemorySourceConfig; +use datafusion_datasource::source::DataSourceExec; use datafusion_execution::memory_pool::{ GreedyMemoryPool, MemoryPool, TrackConsumersPool, }; @@ -46,8 +48,6 @@ 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_datasource::memory::MemorySourceConfig; -use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::spill::get_record_batch_memory_size; use test_utils::AccessLogGenerator; diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index 61d51aeefefe..95298d37b7d8 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -27,6 +27,7 @@ use datafusion::execution::context::SessionState; use datafusion::execution::session_state::SessionStateBuilder; use datafusion::prelude::SessionContext; use datafusion_common::stats::Precision; +use datafusion_datasource::source::DataSourceExec; use datafusion_execution::cache::cache_manager::CacheManagerConfig; use datafusion_execution::cache::cache_unit::{ DefaultFileStatisticsCache, DefaultListFilesCache, @@ -34,7 +35,6 @@ use datafusion_execution::cache::cache_unit::{ use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_expr::{col, lit, Expr}; -use datafusion_datasource::source::DataSourceExec; use datafusion::datasource::physical_plan::FileScanConfig; use tempfile::tempdir; diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 13c5fbee4ccf..242698215275 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -30,10 +30,10 @@ use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; use datafusion_common::{ScalarValue, ToDFSchema}; +use datafusion_datasource::source::DataSourceExec; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::{col, lit, Expr}; use datafusion_physical_expr::create_physical_expr; -use datafusion_datasource::source::DataSourceExec; use futures::StreamExt; use object_store::path::Path; diff --git a/datafusion/core/tests/parquet/utils.rs b/datafusion/core/tests/parquet/utils.rs index 10b93eddfede..1d573286c435 100644 --- a/datafusion/core/tests/parquet/utils.rs +++ b/datafusion/core/tests/parquet/utils.rs @@ -18,8 +18,8 @@ //! Utilities for parquet tests use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; -use datafusion_physical_plan::metrics::MetricsSet; use datafusion_datasource::source::DataSourceExec; +use datafusion_physical_plan::metrics::MetricsSet; use datafusion_physical_plan::{accept, ExecutionPlan, ExecutionPlanVisitor}; /// Find the metrics from the first DataSourceExec encountered in the plan diff --git a/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs b/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs index fd8e9d21e7e4..78f0dd13aebe 100644 --- a/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs @@ -25,6 +25,8 @@ use arrow::record_batch::RecordBatch; use datafusion_common::cast::as_int64_array; use datafusion_common::config::ConfigOptions; use datafusion_common::Result; +use datafusion_datasource::memory::MemorySourceConfig; +use datafusion_datasource::source::DataSourceExec; use datafusion_execution::TaskContext; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{self, cast}; @@ -36,9 +38,7 @@ 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_datasource::memory::MemorySourceConfig; use datafusion_physical_plan::projection::ProjectionExec; -use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::ExecutionPlan; /// Mock data using a MemorySourceConfig which has an exact count statistic diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 112524dd3c95..13df06bce9b0 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -34,6 +34,7 @@ use datafusion::datasource::physical_plan::{CsvSource, FileScanConfig, ParquetSo use datafusion_common::error::Result; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::ScalarValue; +use datafusion_datasource::source::DataSourceExec; use datafusion_expr::{JoinType, Operator}; use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; use datafusion_physical_expr::PhysicalExpr; @@ -57,7 +58,6 @@ use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::ExecutionPlanProperties; use datafusion_physical_plan::PlanProperties; diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index 4df884ec64e0..ce9deac475c9 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -21,6 +21,7 @@ use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::config::ConfigOptions; use datafusion_common::error::Result; +use datafusion_datasource::empty::EmptyExec; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::BinaryExpr; @@ -30,7 +31,6 @@ 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_datasource::empty::EmptyExec; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::projection::ProjectionExec; diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index 3087998b7867..df222d58ba3a 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -25,6 +25,8 @@ use datafusion::datasource::physical_plan::{CsvSource, FileScanConfig}; use datafusion_common::config::ConfigOptions; use datafusion_common::Result; use datafusion_common::{JoinSide, JoinType, ScalarValue}; +use datafusion_datasource::memory::MemorySourceConfig; +use datafusion_datasource::source::DataSourceExec; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{ @@ -48,12 +50,10 @@ use datafusion_physical_plan::joins::{ HashJoinExec, NestedLoopJoinExec, PartitionMode, StreamJoinPartitionMode, SymmetricHashJoinExec, }; -use datafusion_datasource::memory::MemorySourceConfig; 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_datasource::source::DataSourceExec; use datafusion_physical_plan::streaming::PartitionStream; use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index d04485ede760..07d87fcb1fb4 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -31,6 +31,8 @@ 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_datasource::memory::MemorySourceConfig; +use datafusion_datasource::source::DataSourceExec; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; @@ -52,11 +54,9 @@ use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::{JoinFilter, JoinOn}; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode, SortMergeJoinExec}; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use datafusion_datasource::memory::MemorySourceConfig; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use datafusion_datasource::source::DataSourceExec; use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::union::UnionExec; diff --git a/datafusion/core/tests/sql/path_partition.rs b/datafusion/core/tests/sql/path_partition.rs index 6ec757fd2212..35447c20cd34 100644 --- a/datafusion/core/tests/sql/path_partition.rs +++ b/datafusion/core/tests/sql/path_partition.rs @@ -40,10 +40,10 @@ use datafusion::{ use datafusion_catalog::TableProvider; use datafusion_common::stats::Precision; use datafusion_common::ScalarValue; +use datafusion_datasource::source::DataSourceExec; use datafusion_execution::config::SessionConfig; use datafusion_expr::{col, lit, Expr, Operator}; use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; -use datafusion_datasource::source::DataSourceExec; use async_trait::async_trait; use bytes::Bytes; diff --git a/datafusion/core/tests/user_defined/user_defined_table_functions.rs b/datafusion/core/tests/user_defined/user_defined_table_functions.rs index 12e278b4e877..c0fd7ca25676 100644 --- a/datafusion/core/tests/user_defined/user_defined_table_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_table_functions.rs @@ -34,8 +34,8 @@ use datafusion::prelude::SessionContext; use datafusion_catalog::Session; use datafusion_catalog::TableFunctionImpl; use datafusion_common::{assert_batches_eq, DFSchema, ScalarValue}; -use datafusion_expr::{EmptyRelation, Expr, LogicalPlan, Projection, TableType}; use datafusion_datasource::memory::MemorySourceConfig; +use datafusion_expr::{EmptyRelation, Expr, LogicalPlan, Projection, TableType}; use async_trait::async_trait; diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 46fe8853655c..5e9dc27d64ef 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -1141,8 +1141,8 @@ mod lazy_memory_tests { #[cfg(test)] mod tests { use super::*; - use datafusion_physical_plan::expressions::lit; use crate::test::{self, make_partition}; + use datafusion_physical_plan::expressions::lit; use arrow::datatypes::{DataType, Field}; use datafusion_common::assert_batches_eq; diff --git a/datafusion/datasource/src/values.rs b/datafusion/datasource/src/values.rs index 2279f2d894ce..1f6d996b80ac 100644 --- a/datafusion/datasource/src/values.rs +++ b/datafusion/datasource/src/values.rs @@ -231,8 +231,8 @@ impl ExecutionPlan for ValuesExec { #[cfg(test)] mod tests { use super::*; - use datafusion_physical_plan::expressions::lit; use crate::test::{self, make_partition}; + use datafusion_physical_plan::expressions::lit; use arrow::datatypes::{DataType, Field}; use datafusion_common::stats::{ColumnStatistics, Precision}; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index e23ade1d63f4..b46114f8f66e 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1347,10 +1347,10 @@ mod tests { use crate::common::collect; use crate::execution_plan::Boundedness; use crate::expressions::col; - use crate::test::MockMemorySourceConfig; use crate::metrics::MetricValue; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; + use crate::test::MockMemorySourceConfig; use crate::RecordBatchStream; use arrow::array::{ @@ -2626,8 +2626,11 @@ mod tests { .unwrap(), ]; - let input = - MockMemorySourceConfig::try_new_exec(&[input_data], Arc::clone(&schema), None)?; + let input = MockMemorySourceConfig::try_new_exec( + &[input_data], + Arc::clone(&schema), + None, + )?; let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, group_by, @@ -2713,8 +2716,11 @@ mod tests { .unwrap(), ]; - let input = - MockMemorySourceConfig::try_new_exec(&[input_data], Arc::clone(&schema), None)?; + let input = MockMemorySourceConfig::try_new_exec( + &[input_data], + Arc::clone(&schema), + None, + )?; let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, group_by, diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 3fcf4ea49f95..b67beb39aa4f 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -2082,9 +2082,12 @@ mod tests { let batch2 = build_table_i32(("a1", &vec![2]), ("b2", &vec![2]), ("c1", &vec![9])); let schema = batch1.schema(); - let left = - MockMemorySourceConfig::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) - .unwrap(); + let left = MockMemorySourceConfig::try_new_exec( + &[vec![batch1], vec![batch2]], + schema, + None, + ) + .unwrap(); let right = build_table( ("a1", &vec![1, 2, 3]), @@ -2154,9 +2157,12 @@ mod tests { ); let schema = batch1.schema(); - let left = - MockMemorySourceConfig::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) - .unwrap(); + let left = MockMemorySourceConfig::try_new_exec( + &[vec![batch1], vec![batch2]], + schema, + None, + ) + .unwrap(); let right = build_table( ("a2", &vec![20, 30, 10]), ("b2", &vec![5, 6, 4]), @@ -2208,9 +2214,12 @@ mod tests { let batch2 = build_table_i32(("a2", &vec![30]), ("b1", &vec![5]), ("c2", &vec![90])); let schema = batch1.schema(); - let right = - MockMemorySourceConfig::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) - .unwrap(); + let right = MockMemorySourceConfig::try_new_exec( + &[vec![batch1], vec![batch2]], + schema, + None, + ) + .unwrap(); let on = vec![( Arc::new(Column::new_with_schema("b1", &left.schema())?) as _, @@ -3737,9 +3746,12 @@ mod tests { let dates: ArrayRef = Arc::new(Date32Array::from(vec![19107, 19108, 19109])); let n: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![dates, n])?; - let left = - MockMemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) - .unwrap(); + let left = MockMemorySourceConfig::try_new_exec( + &[vec![batch]], + Arc::clone(&schema), + None, + ) + .unwrap(); let dates: ArrayRef = Arc::new(Date32Array::from(vec![19108, 19108, 19109])); let n: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6])); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![dates, n])?; diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 053a329adb51..2362413fe669 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -1030,8 +1030,8 @@ impl EmbeddedProjection for NestedLoopJoinExec { #[cfg(test)] pub(crate) mod tests { use super::*; - use crate::test::MockMemorySourceConfig; use crate::test::MockDataSourceExec; + use crate::test::MockMemorySourceConfig; use crate::{ common, expressions::Column, repartition::RepartitionExec, test::build_table_i32, }; @@ -1072,7 +1072,8 @@ pub(crate) mod tests { }; let mut source = - MockMemorySourceConfig::try_new(&[batches], Arc::clone(&schema), None).unwrap(); + MockMemorySourceConfig::try_new(&[batches], Arc::clone(&schema), None) + .unwrap(); if !sorted_column_names.is_empty() { let mut sort_info = LexOrdering::default(); for name in sorted_column_names { diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index f609108f8e28..da094f714c5d 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -23,9 +23,9 @@ use crate::joins::utils::{JoinFilter, JoinOn}; use crate::joins::{ HashJoinExec, PartitionMode, StreamJoinPartitionMode, SymmetricHashJoinExec, }; -use crate::test::MockMemorySourceConfig; use crate::repartition::RepartitionExec; use crate::test::MockDataSourceExec; +use crate::test::MockMemorySourceConfig; use crate::{common, ExecutionPlan, ExecutionPlanProperties, Partitioning}; use arrow::array::{ diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index f45529958cbd..aaef66005c46 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1051,6 +1051,7 @@ mod tests { use std::collections::HashSet; use super::*; + use crate::test::MockMemorySourceConfig; use crate::{ test::{ assert_is_pending, @@ -1061,7 +1062,6 @@ mod tests { }, {collect, expressions::col}, }; - use crate::test::MockMemorySourceConfig; use arrow::array::{ArrayRef, StringArray, UInt32Array}; use arrow::datatypes::{DataType, Field, Schema}; @@ -1605,8 +1605,8 @@ mod test { use arrow::datatypes::{DataType, Field, Schema}; use super::*; - use crate::test::MockMemorySourceConfig; use crate::test::MockDataSourceExec; + use crate::test::MockMemorySourceConfig; use crate::union::UnionExec; use datafusion_physical_expr::expressions::col; diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 655275cf4670..a544810eb9a4 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -466,11 +466,11 @@ mod tests { use crate::collect; use crate::expressions::col; use crate::expressions::PhysicalSortExpr; - use crate::test::MockMemorySourceConfig; use crate::sorts::sort::SortExec; use crate::test; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; + use crate::test::MockMemorySourceConfig; use super::*; @@ -880,8 +880,11 @@ mod tests { Arc::new(vec![1, 1, 2].into_iter().map(Some).collect::()); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data])?; - let input = - MockMemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None)?; + let input = MockMemorySourceConfig::try_new_exec( + &[vec![batch]], + Arc::clone(&schema), + None, + )?; let partial_sort_exec = Arc::new(PartialSortExec::new( LexOrdering::new(vec![PhysicalSortExpr { diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index cd58e178edf1..d986e610b9a5 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1067,10 +1067,10 @@ mod tests { use crate::collect; use crate::execution_plan::Boundedness; use crate::expressions::col; - use crate::test::MockMemorySourceConfig; use crate::test; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; + use crate::test::MockMemorySourceConfig; use arrow::array::*; use arrow::compute::SortOptions; @@ -1374,9 +1374,12 @@ mod tests { Arc::new(vec![3, 2, 1].into_iter().map(Some).collect::()); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data]).unwrap(); - let input = - MockMemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) - .unwrap(); + let input = MockMemorySourceConfig::try_new_exec( + &[vec![batch]], + Arc::clone(&schema), + None, + ) + .unwrap(); let sort_exec = Arc::new(SortExec::new( LexOrdering::new(vec![PhysicalSortExpr { @@ -1446,7 +1449,11 @@ mod tests { }, }, ]), - MockMemorySourceConfig::try_new_exec(&[vec![batch]], Arc::clone(&schema), None)?, + MockMemorySourceConfig::try_new_exec( + &[vec![batch]], + Arc::clone(&schema), + None, + )?, )); assert_eq!(DataType::Int32, *sort_exec.schema().field(0).data_type()); diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index eb70376b77df..16d57aa1494b 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -384,12 +384,12 @@ mod tests { use crate::coalesce_partitions::CoalescePartitionsExec; use crate::execution_plan::{Boundedness, EmissionType}; use crate::expressions::col; - use crate::test::MockMemorySourceConfig; use crate::metrics::{MetricValue, Timestamp}; use crate::repartition::RepartitionExec; use crate::sorts::sort::SortExec; use crate::stream::RecordBatchReceiverStream; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; + use crate::test::MockMemorySourceConfig; use crate::test::{self, assert_is_pending, make_partition}; use crate::{collect, common}; @@ -736,7 +736,8 @@ mod tests { options: Default::default(), }, ]); - let exec = MockMemorySourceConfig::try_new_exec(partitions, schema, None).unwrap(); + let exec = + MockMemorySourceConfig::try_new_exec(partitions, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, context).await.unwrap(); @@ -972,8 +973,9 @@ mod tests { }, }, ]); - let exec = MockMemorySourceConfig::try_new_exec(&[vec![b1], vec![b2]], schema, None) - .unwrap(); + let exec = + MockMemorySourceConfig::try_new_exec(&[vec![b1], vec![b2]], schema, None) + .unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, task_ctx).await.unwrap(); @@ -1161,8 +1163,9 @@ mod tests { expr: col("b", &schema).unwrap(), options: Default::default(), }]); - let exec = MockMemorySourceConfig::try_new_exec(&[vec![b1], vec![b2]], schema, None) - .unwrap(); + let exec = + MockMemorySourceConfig::try_new_exec(&[vec![b1], vec![b2]], schema, None) + .unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(Arc::clone(&merge) as Arc, task_ctx) @@ -1273,7 +1276,8 @@ mod tests { }, }]); - let exec = MockMemorySourceConfig::try_new_exec(&partitions, schema, None).unwrap(); + let exec = + MockMemorySourceConfig::try_new_exec(&partitions, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, task_ctx).await.unwrap(); diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index 0d7f350c8a1e..ce605831d0bc 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -17,40 +17,28 @@ //! Utilities for testing datafusion-physical-plan -use std::collections::HashMap; -use std::pin::Pin; -use std::sync::Arc; -use std::fmt; -use arrow::array::{ArrayRef, Int32Array, RecordBatch, RecordBatchOptions}; +use arrow::array::{ArrayRef, Int32Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::LexOrdering; use futures::{Future, FutureExt}; - +use std::collections::HashMap; +use std::fmt; +use std::pin::Pin; +use std::sync::Arc; use std::task::{Context, Poll}; -use crate::projection::{ - all_alias_free_columns, new_projections_for_columns, -}; -use crate::{ - common, ColumnarValue, - PhysicalExpr, RecordBatchStream, -}; - -use datafusion_common::{ - plan_err, project_schema, Result, -}; +use crate::{common, RecordBatchStream}; + +use datafusion_common::{project_schema, Result}; use datafusion_execution::memory_pool::MemoryReservation; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::Column; use futures::Stream; -use crate::projection::ProjectionExec; -// use crate::memory::MockMemorySourceConfig; -// use crate::source::MockDataSourceExec; use crate::stream::RecordBatchStreamAdapter; use crate::streaming::PartitionStream; use crate::ExecutionPlan; @@ -62,12 +50,10 @@ use std::fmt::{Debug, Formatter}; use crate::execution_plan::{Boundedness, EmissionType}; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::{ - DisplayAs, DisplayFormatType, PlanProperties, -}; +use crate::{DisplayAs, DisplayFormatType, PlanProperties}; use datafusion_common::config::ConfigOptions; -use datafusion_common::{internal_err, Constraints, ScalarValue, Statistics}; +use datafusion_common::{internal_err, Statistics}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; /// Common behaviors in Data Sources for both from Files and MockMemory. @@ -92,15 +78,10 @@ pub trait MockDataSource: Send + Sync { fn output_partitioning(&self) -> Partitioning; fn eq_properties(&self) -> EquivalenceProperties; fn statistics(&self) -> Result; - fn with_fetch(&self, _limit: Option) -> Option>; fn fetch(&self) -> Option; fn metrics(&self) -> ExecutionPlanMetricsSet { ExecutionPlanMetricsSet::new() } - fn try_swapping_with_projection( - &self, - _projection: &ProjectionExec, - ) -> Result>>; } impl Debug for dyn MockDataSource { @@ -118,14 +99,14 @@ pub struct MockDataSourceExec { impl DisplayAs for MockDataSourceExec { fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { - write!(f, "MockDataSourceExec: ")?; + write!(f, "DataSourceExec: ")?; self.source.fmt_as(t, f) } } impl ExecutionPlan for MockDataSourceExec { fn name(&self) -> &'static str { - "MockDataSourceExec" + "DataSourceExec" } fn as_any(&self) -> &dyn Any { @@ -144,31 +125,15 @@ impl ExecutionPlan for MockDataSourceExec { self: Arc, _: Vec>, ) -> Result> { - Ok(self) + unimplemented!() } fn repartitioned( &self, - target_partitions: usize, - config: &ConfigOptions, + _target_partitions: usize, + _config: &ConfigOptions, ) -> Result>> { - let source = self.source.repartitioned( - target_partitions, - config.optimizer.repartition_file_min_size, - self.properties().eq_properties.output_ordering(), - )?; - - if let Some(source) = source { - let output_partitioning = source.output_partitioning(); - let plan = self - .clone() - .with_source(source) - // Changing source partitioning may invalidate output partitioning. Update it also - .with_partitioning(output_partitioning); - Ok(Some(Arc::new(plan))) - } else { - Ok(Some(Arc::new(self.clone()))) - } + unimplemented!() } fn execute( @@ -187,25 +152,9 @@ impl ExecutionPlan for MockDataSourceExec { self.source.statistics() } - fn with_fetch(&self, limit: Option) -> Option> { - let mut source = Arc::clone(&self.source); - source = source.with_fetch(limit)?; - let cache = self.cache.clone(); - - Some(Arc::new(Self { source, cache })) - } - fn fetch(&self) -> Option { self.source.fetch() } - - fn try_swapping_with_projection( - &self, - projection: &ProjectionExec, - ) -> Result>> { - self.source.try_swapping_with_projection(projection) - } - } impl MockDataSourceExec { @@ -219,24 +168,6 @@ impl MockDataSourceExec { &self.source } - pub fn with_source(mut self, source: Arc) -> Self { - self.cache = Self::compute_properties(Arc::clone(&source)); - self.source = source; - self - } - - /// Assign constraints - pub fn with_constraints(mut self, constraints: Constraints) -> Self { - self.cache = self.cache.with_constraints(constraints); - self - } - - /// Assign output partitioning - pub fn with_partitioning(mut self, partitioning: Partitioning) -> Self { - self.cache = self.cache.with_partitioning(partitioning); - self - } - fn compute_properties(source: Arc) -> PlanProperties { PlanProperties::new( source.eq_properties(), @@ -348,38 +279,9 @@ impl MockDataSource for MockMemorySourceConfig { )) } - fn with_fetch(&self, limit: Option) -> Option> { - let source = self.clone(); - Some(Arc::new(source.with_limit(limit))) - } - fn fetch(&self) -> Option { self.fetch } - - fn try_swapping_with_projection( - &self, - projection: &ProjectionExec, - ) -> Result>> { - // If there is any non-column or alias-carrier expression, Projection should not be removed. - // This process can be moved into MockMemoryExec, but it would be an overlap of their responsibility. - all_alias_free_columns(projection.expr()) - .then(|| { - let all_projections = (0..self.schema.fields().len()).collect(); - let new_projections = new_projections_for_columns( - projection, - self.projection().as_ref().unwrap_or(&all_projections), - ); - - MockMemorySourceConfig::try_new_exec( - self.partitions(), - self.original_schema(), - Some(new_projections), - ) - .map(|e| e as _) - }) - .transpose() - } } impl MockMemorySourceConfig { @@ -413,108 +315,12 @@ impl MockMemorySourceConfig { Ok(Arc::new(MockDataSourceExec::new(Arc::new(source)))) } - /// Create a new execution plan from a list of constant values (`ValuesExec`) - pub fn try_new_as_values( - schema: SchemaRef, - data: Vec>>, - ) -> Result> { - if data.is_empty() { - return plan_err!("Values list cannot be empty"); - } - - let n_row = data.len(); - let n_col = schema.fields().len(); - - // We have this single row batch as a placeholder to satisfy evaluation argument - // and generate a single output row - let placeholder_schema = Arc::new(Schema::empty()); - let placeholder_batch = RecordBatch::try_new_with_options( - Arc::clone(&placeholder_schema), - vec![], - &RecordBatchOptions::new().with_row_count(Some(1)), - )?; - - // Evaluate each column - let arrays = (0..n_col) - .map(|j| { - (0..n_row) - .map(|i| { - let expr = &data[i][j]; - let result = expr.evaluate(&placeholder_batch)?; - - match result { - ColumnarValue::Scalar(scalar) => Ok(scalar), - ColumnarValue::Array(array) if array.len() == 1 => { - ScalarValue::try_from_array(&array, 0) - } - ColumnarValue::Array(_) => { - plan_err!("Cannot have array values in a values list") - } - } - }) - .collect::>>() - .and_then(ScalarValue::iter_to_array) - }) - .collect::>>()?; - - let batch = RecordBatch::try_new_with_options( - Arc::clone(&schema), - arrays, - &RecordBatchOptions::new().with_row_count(Some(n_row)), - )?; - - let partitions = vec![batch]; - Self::try_new_from_batches(Arc::clone(&schema), partitions) - } - - /// Create a new plan using the provided schema and batches. - /// - /// Errors if any of the batches don't match the provided schema, or if no - /// batches are provided. - pub fn try_new_from_batches( - schema: SchemaRef, - batches: Vec, - ) -> Result> { - if batches.is_empty() { - return plan_err!("Values list cannot be empty"); - } - - for batch in &batches { - let batch_schema = batch.schema(); - if batch_schema != schema { - return plan_err!( - "Batch has invalid schema. Expected: {}, got: {}", - schema, - batch_schema - ); - } - } - - let partitions = vec![batches]; - let source = Self { - partitions, - schema: Arc::clone(&schema), - projected_schema: Arc::clone(&schema), - projection: None, - sort_information: vec![], - show_sizes: true, - fetch: None, - }; - Ok(Arc::new(MockDataSourceExec::new(Arc::new(source)))) - } - /// Set the limit of the files pub fn with_limit(mut self, limit: Option) -> Self { self.fetch = limit; self } - /// Set `show_sizes` to determine whether to display partition sizes - pub fn with_show_sizes(mut self, show_sizes: bool) -> Self { - self.show_sizes = show_sizes; - self - } - /// Ref to partitions pub fn partitions(&self) -> &[Vec] { &self.partitions @@ -525,11 +331,6 @@ impl MockMemorySourceConfig { &self.projection } - /// Show sizes - pub fn show_sizes(&self) -> bool { - self.show_sizes - } - /// Ref to sort information pub fn sort_information(&self) -> &[LexOrdering] { &self.sort_information @@ -609,13 +410,12 @@ impl MockMemorySourceConfig { } } - /// Iterator over batches pub struct MockMemoryStream { /// Vector of record batches data: Vec, /// Optional memory reservation bound to the data, freed on drop - reservation: Option, + _reservation: Option, /// Schema representing the data schema: SchemaRef, /// Optional projection for which columns to load @@ -635,7 +435,7 @@ impl MockMemoryStream { ) -> Result { Ok(Self { data, - reservation: None, + _reservation: None, schema, projection, index: 0, @@ -643,13 +443,7 @@ impl MockMemoryStream { }) } - /// Set the memory reservation for the data - pub(super) fn with_reservation(mut self, reservation: MemoryReservation) -> Self { - self.reservation = Some(reservation); - self - } - - /// Set the number of rows to produce + // /// Set the number of rows to produce pub(super) fn with_fetch(mut self, fetch: Option) -> Self { self.fetch = fetch; self diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 652173c2257c..c3f121e5b4f9 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -640,8 +640,8 @@ fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { mod tests { use super::*; use crate::collect; - use crate::test::MockMemorySourceConfig; use crate::test; + use crate::test::MockMemorySourceConfig; use crate::test::MockDataSourceExec; use arrow::compute::SortOptions; diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index b973892451d6..fb97a8b89b72 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1190,9 +1190,9 @@ mod tests { use crate::common::collect; use crate::expressions::PhysicalSortExpr; - use crate::test::MockMemorySourceConfig; use crate::projection::ProjectionExec; use crate::streaming::{PartitionStream, StreamingTableExec}; + use crate::test::MockMemorySourceConfig; use crate::windows::{ create_udwf_window_expr, create_window_expr, BoundedWindowAggExec, InputOrderMode, }; From 781034b358c98c7a7f316d18c231fea01a2a4473 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Tue, 18 Feb 2025 01:28:09 +0530 Subject: [PATCH 06/18] test pass but Mock is not efficient --- datafusion/physical-plan/src/test.rs | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index ce605831d0bc..192524f9bf70 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -595,18 +595,12 @@ pub fn make_partition(sz: i32) -> RecordBatch { /// Returns a `MockDataSourceExec` that scans `partitions` of 100 batches each pub fn scan_partitioned(partitions: usize) -> Arc { - Arc::new(mem_exec(partitions)) -} - -/// Returns a `MockDataSourceExec` that scans `partitions` of 100 batches each -pub fn mem_exec(partitions: usize) -> MockDataSourceExec { let data: Vec> = (0..partitions).map(|_| vec![make_partition(100)]).collect(); let schema = data[0][0].schema(); let projection = None; - MockDataSourceExec::new(Arc::new( - MockMemorySourceConfig::try_new(&data, schema, projection).unwrap(), - )) + + MockMemorySourceConfig::try_new_exec(&data, schema, projection).unwrap() } // Construct a stream partition for test purposes From 00c800855d1f67520138cb6bae112687b8251dae Mon Sep 17 00:00:00 2001 From: logan-keede Date: Tue, 18 Feb 2025 02:13:58 +0530 Subject: [PATCH 07/18] temporary stable --- Cargo.lock | 1 - datafusion/physical-plan/Cargo.toml | 6 ------ datafusion/physical-plan/src/filter.rs | 2 +- 3 files changed, 1 insertion(+), 8 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 9249867c318c..8fd06d244349 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2266,7 +2266,6 @@ dependencies = [ "datafusion-functions-window-common", "datafusion-physical-expr", "datafusion-physical-expr-common", - "datafusion-physical-plan", "futures", "half", "hashbrown 0.14.5", diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 33ccff6378d2..c25f28315344 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -75,13 +75,7 @@ tokio = { workspace = true, features = [ "parking_lot", ] } datafusion-datasource = { workspace = true } -datafusion-physical-plan = { workspace = true} [[bench]] harness = false name = "spm" - -[[test]] -harness = false -name = "tests" -path = "tests/mod.rs" diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 60e737834dd9..ed0a38247e45 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -624,7 +624,7 @@ mod tests { use crate::expressions::*; use crate::test; use crate::test::exec::StatisticsExec; - + #[allow(unused_imports)] use arrow::datatypes::{Field, Schema, UnionFields, UnionMode}; use datafusion_common::ScalarValue; From f0410901b937c8b3f0ef4c0b388985308a0287ef Mon Sep 17 00:00:00 2001 From: logan-keede Date: Tue, 18 Feb 2025 12:28:51 +0530 Subject: [PATCH 08/18] one struct, test pass, cleaning pending --- datafusion/physical-plan/Cargo.toml | 2 +- .../src/joins/nested_loop_join.rs | 4 +- .../physical-plan/src/joins/test_utils.rs | 6 +- .../physical-plan/src/repartition/mod.rs | 4 +- datafusion/physical-plan/src/test.rs | 301 +++++++++++------- datafusion/physical-plan/src/union.rs | 6 +- 6 files changed, 191 insertions(+), 132 deletions(-) diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index c25f28315344..0ede49103076 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -64,6 +64,7 @@ tokio = { workspace = true } [dev-dependencies] criterion = { version = "0.5", features = ["async_futures"] } +datafusion-datasource = { workspace = true } datafusion-functions-aggregate = { workspace = true } datafusion-functions-window = { workspace = true } rand = { workspace = true } @@ -74,7 +75,6 @@ tokio = { workspace = true, features = [ "fs", "parking_lot", ] } -datafusion-datasource = { workspace = true } [[bench]] harness = false diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 2362413fe669..cc0ce7e30653 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -1030,8 +1030,8 @@ impl EmbeddedProjection for NestedLoopJoinExec { #[cfg(test)] pub(crate) mod tests { use super::*; - use crate::test::MockDataSourceExec; use crate::test::MockMemorySourceConfig; + // use crate::test::MockMemorySourceConfig; use crate::{ common, expressions::Column, repartition::RepartitionExec, test::build_table_i32, }; @@ -1090,7 +1090,7 @@ pub(crate) mod tests { source = source.try_with_sort_information(vec![sort_info]).unwrap(); } - Arc::new(MockDataSourceExec::new(Arc::new(source))) + Arc::new(MockMemorySourceConfig::new(Arc::new(source))) } fn build_left_table() -> Arc { diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index da094f714c5d..c50d25dc2ddf 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -24,8 +24,8 @@ use crate::joins::{ HashJoinExec, PartitionMode, StreamJoinPartitionMode, SymmetricHashJoinExec, }; use crate::repartition::RepartitionExec; -use crate::test::MockDataSourceExec; use crate::test::MockMemorySourceConfig; +// use crate::test::MockMemorySourceConfig; use crate::{common, ExecutionPlan, ExecutionPlanProperties, Partitioning}; use arrow::array::{ @@ -536,8 +536,8 @@ pub fn create_memory_table( let right = MockMemorySourceConfig::try_new(&[right_partition], right_schema, None)? .try_with_sort_information(right_sorted)?; Ok(( - Arc::new(MockDataSourceExec::new(Arc::new(left))), - Arc::new(MockDataSourceExec::new(Arc::new(right))), + Arc::new(MockMemorySourceConfig::new(Arc::new(left))), + Arc::new(MockMemorySourceConfig::new(Arc::new(right))), )) } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index aaef66005c46..efd42ea3de0b 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1605,8 +1605,8 @@ mod test { use arrow::datatypes::{DataType, Field, Schema}; use super::*; - use crate::test::MockDataSourceExec; use crate::test::MockMemorySourceConfig; + // use crate::test::MockMemorySourceConfig; use crate::union::UnionExec; use datafusion_physical_expr::expressions::col; @@ -1719,7 +1719,7 @@ mod test { schema: &SchemaRef, sort_exprs: LexOrdering, ) -> Arc { - Arc::new(MockDataSourceExec::new(Arc::new( + Arc::new(MockMemorySourceConfig::new(Arc::new( MockMemorySourceConfig::try_new(&[vec![]], Arc::clone(schema), None) .unwrap() .try_with_sort_information(vec![sort_exprs]) diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index 192524f9bf70..3be4d24d21ba 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -57,64 +57,98 @@ use datafusion_common::{internal_err, Statistics}; use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; /// Common behaviors in Data Sources for both from Files and MockMemory. -/// See `MockDataSourceExec` for physical plan implementation -pub trait MockDataSource: Send + Sync { - fn open( - &self, - partition: usize, - context: Arc, - ) -> Result; - fn as_any(&self) -> &dyn Any; - fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result; - fn repartitioned( - &self, - _target_partitions: usize, - _repartition_file_min_size: usize, - _output_ordering: Option, - ) -> Result>> { - Ok(None) - } - - fn output_partitioning(&self) -> Partitioning; - fn eq_properties(&self) -> EquivalenceProperties; - fn statistics(&self) -> Result; - fn fetch(&self) -> Option; - fn metrics(&self) -> ExecutionPlanMetricsSet { - ExecutionPlanMetricsSet::new() - } -} - -impl Debug for dyn MockDataSource { - fn fmt(&self, f: &mut Formatter) -> fmt::Result { - write!(f, "MockDataSource: ") - } -} +/// See `MockMemorySourceConfig` for physical plan implementation +// pub trait MockDataSource: Send + Sync { +// fn open( +// &self, +// partition: usize, +// context: Arc, +// ) -> Result; +// fn as_any(&self) -> &dyn Any; +// fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result; +// fn repartitioned( +// &self, +// _target_partitions: usize, +// _repartition_file_min_size: usize, +// _output_ordering: Option, +// ) -> Result>> { +// Ok(None) +// } + +// fn output_partitioning(&self) -> Partitioning; +// fn eq_properties(&self) -> EquivalenceProperties; +// fn statistics(&self) -> Result; +// fn fetch(&self) -> Option; +// fn metrics(&self) -> ExecutionPlanMetricsSet { +// ExecutionPlanMetricsSet::new() +// } +// } + +// /// Unified data source for file formats like JSON, CSV, AVRO, ARROW, PARQUET -#[derive(Clone, Debug)] -pub struct MockDataSourceExec { - source: Arc, - cache: PlanProperties, -} +// #[derive(Clone, Debug)] +// pub struct MockMemorySourceConfig { +// source: Arc, +// cache: PlanProperties, +// } -impl DisplayAs for MockDataSourceExec { +impl DisplayAs for MockMemorySourceConfig { fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { write!(f, "DataSourceExec: ")?; - self.source.fmt_as(t, f) + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + let partition_sizes: Vec<_> = + self.partitions.iter().map(|b| b.len()).collect(); + + let output_ordering = self + .sort_information + .first() + .map(|output_ordering| { + format!(", output_ordering={}", output_ordering) + }) + .unwrap_or_default(); + + let eq_properties = self.eq_properties(); + let constraints = eq_properties.constraints(); + let constraints = if constraints.is_empty() { + String::new() + } else { + format!(", {}", constraints) + }; + + let limit = self + .fetch + .map_or(String::new(), |limit| format!(", fetch={}", limit)); + if self.show_sizes { + write!( + f, + "partitions={}, partition_sizes={partition_sizes:?}{limit}{output_ordering}{constraints}", + partition_sizes.len(), + ) + } else { + write!( + f, + "partitions={}{limit}{output_ordering}{constraints}", + partition_sizes.len(), + ) + } + } + } } } -impl ExecutionPlan for MockDataSourceExec { +impl ExecutionPlan for MockMemorySourceConfig { fn name(&self) -> &'static str { "DataSourceExec" } fn as_any(&self) -> &dyn Any { - self + unimplemented!() } fn properties(&self) -> &PlanProperties { - &self.cache + return &self.cache; } fn children(&self) -> Vec<&Arc> { @@ -141,45 +175,45 @@ impl ExecutionPlan for MockDataSourceExec { partition: usize, context: Arc, ) -> Result { - self.source.open(partition, context) + self.open(partition, context) } fn metrics(&self) -> Option { - Some(self.source.metrics().clone_inner()) + unimplemented!() } fn statistics(&self) -> Result { - self.source.statistics() + self.statistics() } fn fetch(&self) -> Option { - self.source.fetch() + self.fetch } } -impl MockDataSourceExec { - pub fn new(source: Arc) -> Self { - let cache = Self::compute_properties(Arc::clone(&source)); - Self { source, cache } - } +// impl MockMemorySourceConfig { + // pub fn new(source: Arc) -> Self { + // let cache = Self::compute_properties(Arc::clone(&source)); + // Self { source, cache } + // } /// Return the source object - pub fn source(&self) -> &Arc { - &self.source - } - - fn compute_properties(source: Arc) -> PlanProperties { - PlanProperties::new( - source.eq_properties(), - source.output_partitioning(), - EmissionType::Incremental, - Boundedness::Bounded, - ) - } -} + // pub fn source(&self) -> &Arc { + // &self.source + // } + + // fn compute_properties(source: Arc) -> PlanProperties { + // PlanProperties::new( + // source.eq_properties(), + // source.output_partitioning(), + // EmissionType::Incremental, + // Boundedness::Bounded, + // ) + // } +// } /// Data source configuration for reading in-memory batches of data -#[derive(Clone)] +#[derive(Clone, Debug)] pub struct MockMemorySourceConfig { /// The partitions to query partitions: Vec>, @@ -196,9 +230,11 @@ pub struct MockMemorySourceConfig { /// The maximum number of records to read from this plan. If `None`, /// all records after filtering are returned. fetch: Option, + cache: PlanProperties, + // source: MockMemorySourceConfig } -impl MockDataSource for MockMemorySourceConfig { +impl MockMemorySourceConfig { fn open( &self, partition: usize, @@ -217,49 +253,57 @@ impl MockDataSource for MockMemorySourceConfig { fn as_any(&self) -> &dyn Any { self } - - fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - let partition_sizes: Vec<_> = - self.partitions.iter().map(|b| b.len()).collect(); - - let output_ordering = self - .sort_information - .first() - .map(|output_ordering| { - format!(", output_ordering={}", output_ordering) - }) - .unwrap_or_default(); - - let eq_properties = self.eq_properties(); - let constraints = eq_properties.constraints(); - let constraints = if constraints.is_empty() { - String::new() - } else { - format!(", {}", constraints) - }; - - let limit = self - .fetch - .map_or(String::new(), |limit| format!(", fetch={}", limit)); - if self.show_sizes { - write!( - f, - "partitions={}, partition_sizes={partition_sizes:?}{limit}{output_ordering}{constraints}", - partition_sizes.len(), - ) - } else { - write!( - f, - "partitions={}{limit}{output_ordering}{constraints}", - partition_sizes.len(), - ) - } - } - } + fn compute_properties(&self) -> PlanProperties { + PlanProperties::new( + self.eq_properties(), + self.output_partitioning(), + EmissionType::Incremental, + Boundedness::Bounded, + ) } + // fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { + // match t { + // DisplayFormatType::Default | DisplayFormatType::Verbose => { + // let partition_sizes: Vec<_> = + // self.partitions.iter().map(|b| b.len()).collect(); + + // let output_ordering = self + // .sort_information + // .first() + // .map(|output_ordering| { + // format!(", output_ordering={}", output_ordering) + // }) + // .unwrap_or_default(); + + // let eq_properties = self.eq_properties(); + // let constraints = eq_properties.constraints(); + // let constraints = if constraints.is_empty() { + // String::new() + // } else { + // format!(", {}", constraints) + // }; + + // let limit = self + // .fetch + // .map_or(String::new(), |limit| format!(", fetch={}", limit)); + // if self.show_sizes { + // write!( + // f, + // "partitions={}, partition_sizes={partition_sizes:?}{limit}{output_ordering}{constraints}", + // partition_sizes.len(), + // ) + // } else { + // write!( + // f, + // "partitions={}{limit}{output_ordering}{constraints}", + // partition_sizes.len(), + // ) + // } + // } + // } + + fn output_partitioning(&self) -> Partitioning { Partitioning::UnknownPartitioning(self.partitions.len()) } @@ -278,13 +322,9 @@ impl MockDataSource for MockMemorySourceConfig { self.projection.clone(), )) } +// } - fn fetch(&self) -> Option { - self.fetch - } -} - -impl MockMemorySourceConfig { +// impl MockMemorySourceConfig { /// Create a new `MockMemorySourceConfig` for reading in-memory record batches /// The provided `schema` should not have the projection applied. pub fn try_new( @@ -295,12 +335,22 @@ impl MockMemorySourceConfig { let projected_schema = project_schema(&schema, projection.as_ref())?; Ok(Self { partitions: partitions.to_vec(), - schema, - projected_schema, - projection, + schema: schema, + cache: PlanProperties::new( + EquivalenceProperties::new_with_orderings( + Arc::clone(&projected_schema), + vec![].as_slice(), + ), + Partitioning::UnknownPartitioning(partitions.len()), + EmissionType::Incremental, + Boundedness::Bounded, + ), + projected_schema: projected_schema, + projection: projection, sort_information: vec![], show_sizes: true, fetch: None, + }) } @@ -310,11 +360,20 @@ impl MockMemorySourceConfig { partitions: &[Vec], schema: SchemaRef, projection: Option>, - ) -> Result> { - let source = Self::try_new(partitions, schema, projection)?; - Ok(Arc::new(MockDataSourceExec::new(Arc::new(source)))) + ) -> Result> { + let mut source = Self::try_new(partitions, schema, projection)?; + let cache = source.compute_properties(); + source.cache = cache; + Ok(Arc::new(source)) + } + pub fn new(source: Arc) -> MockMemorySourceConfig { + // let source = source; + let cache = source.compute_properties(); + let source = &*source; + let mut source = source.clone(); + source.cache = cache; + source } - /// Set the limit of the files pub fn with_limit(mut self, limit: Option) -> Self { self.fetch = limit; @@ -593,7 +652,7 @@ pub fn make_partition(sz: i32) -> RecordBatch { RecordBatch::try_new(schema, vec![arr]).unwrap() } -/// Returns a `MockDataSourceExec` that scans `partitions` of 100 batches each +/// Returns a `MockMemorySourceConfig` that scans `partitions` of 100 batches each pub fn scan_partitioned(partitions: usize) -> Arc { let data: Vec> = (0..partitions).map(|_| vec![make_partition(100)]).collect(); diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index c3f121e5b4f9..a20ae066a6fa 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -643,7 +643,7 @@ mod tests { use crate::test; use crate::test::MockMemorySourceConfig; - use crate::test::MockDataSourceExec; + // use crate::test::MockMemorySourceConfig; use arrow::compute::SortOptions; use arrow::datatypes::DataType; use datafusion_common::ScalarValue; @@ -865,11 +865,11 @@ mod tests { .iter() .map(|ordering| convert_to_sort_exprs(ordering)) .collect::>(); - let child1 = Arc::new(MockDataSourceExec::new(Arc::new( + let child1 = Arc::new(MockMemorySourceConfig::new(Arc::new( MockMemorySourceConfig::try_new(&[], Arc::clone(&schema), None)? .try_with_sort_information(first_orderings)?, ))); - let child2 = Arc::new(MockDataSourceExec::new(Arc::new( + let child2 = Arc::new(MockMemorySourceConfig::new(Arc::new( MockMemorySourceConfig::try_new(&[], Arc::clone(&schema), None)? .try_with_sort_information(second_orderings)?, ))); From 408521cfbd46d2dc32df30b09600d777c9921fa1 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Tue, 18 Feb 2025 13:11:59 +0530 Subject: [PATCH 09/18] cleaning --- Cargo.lock | 1 + datafusion-examples/Cargo.toml | 1 + .../examples/custom_datasource.rs | 2 +- .../examples/remote_catalog.rs | 2 +- datafusion-examples/examples/simple_udtf.rs | 2 +- datafusion/core/benches/physical_plan.rs | 2 +- datafusion/core/benches/sort.rs | 3 +- datafusion/physical-plan/src/lib.rs | 8 - datafusion/physical-plan/src/test.rs | 205 ++++-------------- 9 files changed, 54 insertions(+), 172 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8fd06d244349..df6208ad362a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1951,6 +1951,7 @@ dependencies = [ "bytes", "dashmap", "datafusion", + "datafusion-datasource", "datafusion-proto", "env_logger", "futures", diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index ec6e0ab71d50..90af77900805 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -62,6 +62,7 @@ bytes = { workspace = true } dashmap = { workspace = true } # note only use main datafusion crate for examples datafusion = { workspace = true, default-features = true, features = ["avro"] } +datafusion-datasource = { workspace = true } datafusion-proto = { workspace = true } env_logger = { workspace = true } futures = { workspace = true } diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index bc865fac5a33..337537ae812e 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -31,12 +31,12 @@ use datafusion::execution::context::TaskContext; use datafusion::logical_expr::LogicalPlanBuilder; use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion::physical_plan::memory::MemoryStream; use datafusion::physical_plan::{ project_schema, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, }; use datafusion::prelude::*; +use datafusion_datasource::memory::MemoryStream; use datafusion::catalog::Session; use tokio::time::timeout; diff --git a/datafusion-examples/examples/remote_catalog.rs b/datafusion-examples/examples/remote_catalog.rs index f84c6a0302ce..e2c9c54654b7 100644 --- a/datafusion-examples/examples/remote_catalog.rs +++ b/datafusion-examples/examples/remote_catalog.rs @@ -38,10 +38,10 @@ 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::MemorySourceConfig; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::{DataFrame, SessionContext}; +use datafusion_datasource::memory::MemorySourceConfig; use futures::TryStreamExt; use std::any::Any; use std::sync::Arc; diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs index afba4c390f71..b61303f2435f 100644 --- a/datafusion-examples/examples/simple_udtf.rs +++ b/datafusion-examples/examples/simple_udtf.rs @@ -29,9 +29,9 @@ 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::MemorySourceConfig; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; +use datafusion_datasource::memory::MemorySourceConfig; use std::fs::File; use std::io::Seek; use std::path::Path; diff --git a/datafusion/core/benches/physical_plan.rs b/datafusion/core/benches/physical_plan.rs index 53c245ecc2b5..aae1457ab9e6 100644 --- a/datafusion/core/benches/physical_plan.rs +++ b/datafusion/core/benches/physical_plan.rs @@ -33,9 +33,9 @@ use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMerge use datafusion::physical_plan::{ collect, expressions::{col, PhysicalSortExpr}, - memory::MemorySourceConfig, }; use datafusion::prelude::SessionContext; +use datafusion_datasource::memory::MemorySourceConfig; use datafusion_physical_expr_common::sort_expr::LexOrdering; // Initialize the operator using the provided record batches and the sort key diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 4d71d4c56a6d..8f0b3753f67c 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -79,12 +79,13 @@ use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::{ execution::context::TaskContext, physical_plan::{ - coalesce_partitions::CoalescePartitionsExec, memory::MemorySourceConfig, + coalesce_partitions::CoalescePartitionsExec, sorts::sort_preserving_merge::SortPreservingMergeExec, ExecutionPlan, ExecutionPlanProperties, }, prelude::SessionContext, }; +use datafusion_datasource::memory::MemorySourceConfig; use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 3acecf9d9502..9ddcb0a7cae8 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -56,18 +56,14 @@ pub mod coalesce_batches; pub mod coalesce_partitions; pub mod common; pub mod display; -// pub mod empty; pub mod execution_plan; pub mod explain; pub mod filter; pub mod insert; pub mod joins; pub mod limit; -// pub mod memory; pub mod metrics; -// pub mod placeholder_row; pub mod projection; -// pub mod recursive_query; pub mod repartition; pub mod sorts; pub mod spill; @@ -76,16 +72,12 @@ pub mod streaming; pub mod tree_node; pub mod union; pub mod unnest; -// pub mod values; pub mod windows; -// pub mod work_table; - pub mod udaf { pub use datafusion_expr::StatisticsArgs; pub use datafusion_physical_expr::aggregate::AggregateFunctionExpr; } pub mod coalesce; -// pub mod source; #[cfg(test)] pub mod test; diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index 3be4d24d21ba..cede1041075d 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -17,81 +17,59 @@ //! Utilities for testing datafusion-physical-plan -use arrow::array::{ArrayRef, Int32Array, RecordBatch}; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::utils::collect_columns; -use datafusion_physical_expr::LexOrdering; -use futures::{Future, FutureExt}; +use std::any::Any; use std::collections::HashMap; use std::fmt; +use std::fmt::{Debug, Formatter}; use std::pin::Pin; use std::sync::Arc; - use std::task::{Context, Poll}; -use crate::{common, RecordBatchStream}; - -use datafusion_common::{project_schema, Result}; -use datafusion_execution::memory_pool::MemoryReservation; -use datafusion_physical_expr::equivalence::ProjectionMapping; -use datafusion_physical_expr::expressions::Column; - -use futures::Stream; - +use crate::execution_plan::{Boundedness, EmissionType}; +use crate::metrics::MetricsSet; use crate::stream::RecordBatchStreamAdapter; use crate::streaming::PartitionStream; use crate::ExecutionPlan; +use crate::{common, RecordBatchStream}; +use crate::{DisplayAs, DisplayFormatType, PlanProperties}; -pub mod exec; +use arrow::array::{ArrayRef, Int32Array, RecordBatch}; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion_common::{ + config::ConfigOptions, internal_err, project_schema, Result, Statistics, +}; +use datafusion_execution::{ + memory_pool::MemoryReservation, SendableRecordBatchStream, TaskContext, +}; +use datafusion_physical_expr::{ + equivalence::ProjectionMapping, expressions::Column, utils::collect_columns, + EquivalenceProperties, LexOrdering, Partitioning, +}; -use std::any::Any; -use std::fmt::{Debug, Formatter}; +use futures::Stream; +use futures::{Future, FutureExt}; -use crate::execution_plan::{Boundedness, EmissionType}; -use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::{DisplayAs, DisplayFormatType, PlanProperties}; +pub mod exec; -use datafusion_common::config::ConfigOptions; -use datafusion_common::{internal_err, Statistics}; -use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; - -/// Common behaviors in Data Sources for both from Files and MockMemory. -/// See `MockMemorySourceConfig` for physical plan implementation -// pub trait MockDataSource: Send + Sync { -// fn open( -// &self, -// partition: usize, -// context: Arc, -// ) -> Result; -// fn as_any(&self) -> &dyn Any; -// fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result; -// fn repartitioned( -// &self, -// _target_partitions: usize, -// _repartition_file_min_size: usize, -// _output_ordering: Option, -// ) -> Result>> { -// Ok(None) -// } - -// fn output_partitioning(&self) -> Partitioning; -// fn eq_properties(&self) -> EquivalenceProperties; -// fn statistics(&self) -> Result; -// fn fetch(&self) -> Option; -// fn metrics(&self) -> ExecutionPlanMetricsSet { -// ExecutionPlanMetricsSet::new() -// } -// } - -// - -/// Unified data source for file formats like JSON, CSV, AVRO, ARROW, PARQUET -// #[derive(Clone, Debug)] -// pub struct MockMemorySourceConfig { -// source: Arc, -// cache: PlanProperties, -// } +#[derive(Clone, Debug)] +pub struct MockMemorySourceConfig { + /// The partitions to query + partitions: Vec>, + /// Schema representing the data before projection + schema: SchemaRef, + /// Schema representing the data after the optional projection is applied + projected_schema: SchemaRef, + /// Optional projection + projection: Option>, + /// Sort information: one or more equivalent orderings + sort_information: Vec, + /// if partition sizes should be displayed + show_sizes: bool, + /// The maximum number of records to read from this plan. If `None`, + /// all records after filtering are returned. + fetch: Option, + cache: PlanProperties, +} impl DisplayAs for MockMemorySourceConfig { fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { @@ -191,49 +169,6 @@ impl ExecutionPlan for MockMemorySourceConfig { } } -// impl MockMemorySourceConfig { - // pub fn new(source: Arc) -> Self { - // let cache = Self::compute_properties(Arc::clone(&source)); - // Self { source, cache } - // } - - /// Return the source object - // pub fn source(&self) -> &Arc { - // &self.source - // } - - // fn compute_properties(source: Arc) -> PlanProperties { - // PlanProperties::new( - // source.eq_properties(), - // source.output_partitioning(), - // EmissionType::Incremental, - // Boundedness::Bounded, - // ) - // } -// } - -/// Data source configuration for reading in-memory batches of data -#[derive(Clone, Debug)] -pub struct MockMemorySourceConfig { - /// The partitions to query - partitions: Vec>, - /// Schema representing the data before projection - schema: SchemaRef, - /// Schema representing the data after the optional projection is applied - projected_schema: SchemaRef, - /// Optional projection - projection: Option>, - /// Sort information: one or more equivalent orderings - sort_information: Vec, - /// if partition sizes should be displayed - show_sizes: bool, - /// The maximum number of records to read from this plan. If `None`, - /// all records after filtering are returned. - fetch: Option, - cache: PlanProperties, - // source: MockMemorySourceConfig -} - impl MockMemorySourceConfig { fn open( &self, @@ -250,9 +185,6 @@ impl MockMemorySourceConfig { )) } - fn as_any(&self) -> &dyn Any { - self - } fn compute_properties(&self) -> PlanProperties { PlanProperties::new( self.eq_properties(), @@ -262,48 +194,6 @@ impl MockMemorySourceConfig { ) } - // fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { - // match t { - // DisplayFormatType::Default | DisplayFormatType::Verbose => { - // let partition_sizes: Vec<_> = - // self.partitions.iter().map(|b| b.len()).collect(); - - // let output_ordering = self - // .sort_information - // .first() - // .map(|output_ordering| { - // format!(", output_ordering={}", output_ordering) - // }) - // .unwrap_or_default(); - - // let eq_properties = self.eq_properties(); - // let constraints = eq_properties.constraints(); - // let constraints = if constraints.is_empty() { - // String::new() - // } else { - // format!(", {}", constraints) - // }; - - // let limit = self - // .fetch - // .map_or(String::new(), |limit| format!(", fetch={}", limit)); - // if self.show_sizes { - // write!( - // f, - // "partitions={}, partition_sizes={partition_sizes:?}{limit}{output_ordering}{constraints}", - // partition_sizes.len(), - // ) - // } else { - // write!( - // f, - // "partitions={}{limit}{output_ordering}{constraints}", - // partition_sizes.len(), - // ) - // } - // } - // } - - fn output_partitioning(&self) -> Partitioning { Partitioning::UnknownPartitioning(self.partitions.len()) } @@ -322,11 +212,7 @@ impl MockMemorySourceConfig { self.projection.clone(), )) } -// } -// impl MockMemorySourceConfig { - /// Create a new `MockMemorySourceConfig` for reading in-memory record batches - /// The provided `schema` should not have the projection applied. pub fn try_new( partitions: &[Vec], schema: SchemaRef, @@ -350,11 +236,10 @@ impl MockMemorySourceConfig { sort_information: vec![], show_sizes: true, fetch: None, - }) } - /// Create a new `DataSourceExec` plan for reading in-memory record batches + /// Create a new `DataSourceExec` Equivalent plan for reading in-memory record batches /// The provided `schema` should not have the projection applied. pub fn try_new_exec( partitions: &[Vec], @@ -366,14 +251,16 @@ impl MockMemorySourceConfig { source.cache = cache; Ok(Arc::new(source)) } + + // Equivalent of `DataSourceExec::new` pub fn new(source: Arc) -> MockMemorySourceConfig { - // let source = source; let cache = source.compute_properties(); - let source = &*source; + let source = &*source; let mut source = source.clone(); source.cache = cache; source } + /// Set the limit of the files pub fn with_limit(mut self, limit: Option) -> Self { self.fetch = limit; @@ -658,7 +545,7 @@ pub fn scan_partitioned(partitions: usize) -> Arc { let schema = data[0][0].schema(); let projection = None; - + MockMemorySourceConfig::try_new_exec(&data, schema, projection).unwrap() } From 7804ae5e2413a6e5c69c9b003ae9fb317d799731 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Tue, 18 Feb 2025 23:56:01 +0530 Subject: [PATCH 10/18] more cleaning --- .../examples/custom_datasource.rs | 2 +- .../core/src/datasource/cte_worktable.rs | 2 +- datafusion/core/src/datasource/empty.rs | 2 +- .../core/src/datasource/listing/table.rs | 2 +- .../physical_plan/parquet/source.rs | 6 +- datafusion/core/src/physical_planner.rs | 6 +- .../core/tests/custom_sources_cases/mod.rs | 2 +- .../physical_optimizer/limit_pushdown.rs | 2 +- datafusion/datasource/src/memory.rs | 379 +--------------- datafusion/datasource/src/mod.rs | 5 - .../functions-table/src/generate_series.rs | 2 +- .../src/aggregate_statistics.rs | 2 +- .../src/empty.rs | 8 +- datafusion/physical-plan/src/filter.rs | 67 ++- .../src/joins/nested_loop_join.rs | 2 +- .../physical-plan/src/joins/test_utils.rs | 4 +- datafusion/physical-plan/src/lib.rs | 6 + datafusion/physical-plan/src/memory.rs | 409 ++++++++++++++++++ .../src/placeholder_row.rs | 10 +- .../src/recursive_query.rs | 12 +- .../physical-plan/src/repartition/mod.rs | 2 +- datafusion/physical-plan/src/test.rs | 10 +- datafusion/physical-plan/src/union.rs | 4 +- .../src/values.rs | 14 +- .../src/work_table.rs | 8 +- datafusion/proto/src/physical_plan/mod.rs | 4 +- .../tests/cases/roundtrip_physical_plan.rs | 4 +- 27 files changed, 501 insertions(+), 475 deletions(-) rename datafusion/{datasource => physical-plan}/src/empty.rs (96%) create mode 100644 datafusion/physical-plan/src/memory.rs rename datafusion/{datasource => physical-plan}/src/placeholder_row.rs (95%) rename datafusion/{datasource => physical-plan}/src/recursive_query.rs (97%) rename datafusion/{datasource => physical-plan}/src/values.rs (97%) rename datafusion/{datasource => physical-plan}/src/work_table.rs (97%) diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index 337537ae812e..bc865fac5a33 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -31,12 +31,12 @@ use datafusion::execution::context::TaskContext; use datafusion::logical_expr::LogicalPlanBuilder; use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion::physical_plan::memory::MemoryStream; use datafusion::physical_plan::{ project_schema, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, }; use datafusion::prelude::*; -use datafusion_datasource::memory::MemoryStream; use datafusion::catalog::Session; use tokio::time::timeout; diff --git a/datafusion/core/src/datasource/cte_worktable.rs b/datafusion/core/src/datasource/cte_worktable.rs index 6ed4fb7781ce..b63755f644a8 100644 --- a/datafusion/core/src/datasource/cte_worktable.rs +++ b/datafusion/core/src/datasource/cte_worktable.rs @@ -23,7 +23,7 @@ use std::{any::Any, borrow::Cow}; use arrow::datatypes::SchemaRef; use async_trait::async_trait; use datafusion_catalog::Session; -use datafusion_datasource::work_table::WorkTableExec; +use datafusion_physical_plan::work_table::WorkTableExec; use crate::{ error::Result, diff --git a/datafusion/core/src/datasource/empty.rs b/datafusion/core/src/datasource/empty.rs index e521b0e2e85f..77686c5eb7c2 100644 --- a/datafusion/core/src/datasource/empty.rs +++ b/datafusion/core/src/datasource/empty.rs @@ -28,7 +28,7 @@ use datafusion_common::project_schema; use crate::datasource::{TableProvider, TableType}; use crate::error::Result; use crate::logical_expr::Expr; -use datafusion_datasource::empty::EmptyExec; +use datafusion_physical_plan::empty::EmptyExec; use datafusion_physical_plan::ExecutionPlan; /// An empty plan that is useful for testing and generating plans diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 8a42627aab08..61a0f9e1a193 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -34,10 +34,10 @@ use crate::datasource::{ use crate::execution::context::SessionState; use datafusion_catalog::TableProvider; use datafusion_common::{config_err, DataFusionError, Result}; -use datafusion_datasource::empty::EmptyExec; use datafusion_expr::dml::InsertOp; use datafusion_expr::{utils::conjunction, Expr, TableProviderFilterPushDown}; use datafusion_expr::{SortExpr, TableType}; +use datafusion_physical_plan::empty::EmptyExec; use datafusion_physical_plan::{ExecutionPlan, Statistics}; use arrow::datatypes::{DataType, Field, Schema, SchemaBuilder, SchemaRef}; diff --git a/datafusion/core/src/datasource/physical_plan/parquet/source.rs b/datafusion/core/src/datasource/physical_plan/parquet/source.rs index a98524b0bead..dbd7d17fb34d 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/source.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/source.rs @@ -81,7 +81,7 @@ use object_store::ObjectStore; /// # use datafusion::datasource::listing::PartitionedFile; /// # use datafusion_execution::object_store::ObjectStoreUrl; /// # use datafusion_physical_expr::expressions::lit; -/// # use datafusion_physical_plan::source::DataSourceExec; +/// # use datafusion_datasource::source::DataSourceExec; /// # use datafusion_common::config::TableParquetOptions; /// /// # let file_schema = Arc::new(Schema::empty()); @@ -160,7 +160,7 @@ use object_store::ObjectStore; /// # use arrow::datatypes::Schema; /// # use datafusion::datasource::physical_plan::FileScanConfig; /// # use datafusion::datasource::listing::PartitionedFile; -/// # use datafusion_physical_plan::source::DataSourceExec; +/// # use datafusion_datasource::source::DataSourceExec; /// /// # fn parquet_exec() -> DataSourceExec { unimplemented!() } /// // Split a single DataSourceExec into multiple DataSourceExecs, one for each file @@ -202,7 +202,7 @@ use object_store::ObjectStore; /// # use datafusion::datasource::physical_plan::FileScanConfig; /// # use datafusion::datasource::physical_plan::parquet::source::ParquetSource; /// # use datafusion_execution::object_store::ObjectStoreUrl; -/// # use datafusion_physical_plan::source::DataSourceExec; +/// # use datafusion_datasource::source::DataSourceExec; /// /// # fn schema() -> SchemaRef { /// # Arc::new(Schema::empty()) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 7c56e5531f23..6f327e621fa2 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -56,8 +56,8 @@ use crate::physical_plan::{ displayable, windows, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, Partitioning, PhysicalExpr, WindowExpr, }; -use datafusion_datasource::empty::EmptyExec; -use datafusion_datasource::recursive_query::RecursiveQueryExec; +use datafusion_physical_plan::empty::EmptyExec; +use datafusion_physical_plan::recursive_query::RecursiveQueryExec; use arrow::array::{builder::StringBuilder, RecordBatch}; use arrow::compute::SortOptions; @@ -69,7 +69,6 @@ use datafusion_common::{ ScalarValue, }; use datafusion_datasource::memory::MemorySourceConfig; -use datafusion_datasource::placeholder_row::PlaceholderRowExec; use datafusion_expr::dml::{CopyTo, InsertOp}; use datafusion_expr::expr::{ physical_name, AggregateFunction, Alias, GroupingSet, WindowFunction, @@ -85,6 +84,7 @@ use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::LexOrdering; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::execution_plan::InvariantLevel; +use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::unnest::ListUnnest; use crate::schema_equivalence::schema_satisfied_by; diff --git a/datafusion/core/tests/custom_sources_cases/mod.rs b/datafusion/core/tests/custom_sources_cases/mod.rs index 51e459d3dd67..aafefac04e32 100644 --- a/datafusion/core/tests/custom_sources_cases/mod.rs +++ b/datafusion/core/tests/custom_sources_cases/mod.rs @@ -39,9 +39,9 @@ use datafusion_catalog::Session; use datafusion_common::cast::as_primitive_array; use datafusion_common::project_schema; use datafusion_common::stats::Precision; -use datafusion_datasource::placeholder_row::PlaceholderRowExec; use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::PlanProperties; use async_trait::async_trait; diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index ce9deac475c9..dd2c1960a658 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -21,7 +21,6 @@ use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::config::ConfigOptions; use datafusion_common::error::Result; -use datafusion_datasource::empty::EmptyExec; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::BinaryExpr; @@ -31,6 +30,7 @@ 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; diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 5e9dc27d64ef..5074439fa3f8 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -20,17 +20,16 @@ use std::any::Any; use std::fmt; use std::sync::Arc; -use std::task::{Context, Poll}; use crate::source::{DataSource, DataSourceExec}; use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; +use datafusion_physical_plan::memory::MemoryStream; use datafusion_physical_plan::projection::{ all_alias_free_columns, new_projections_for_columns, ProjectionExec, }; use datafusion_physical_plan::{ common, ColumnarValue, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, - PhysicalExpr, PlanProperties, RecordBatchStream, SendableRecordBatchStream, - Statistics, + PhysicalExpr, PlanProperties, SendableRecordBatchStream, Statistics, }; use arrow::array::{RecordBatch, RecordBatchOptions}; @@ -38,16 +37,12 @@ use arrow::datatypes::{Schema, SchemaRef}; use datafusion_common::{ internal_err, plan_err, project_schema, Constraints, Result, ScalarValue, }; -use datafusion_execution::memory_pool::MemoryReservation; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use futures::Stream; -use parking_lot::RwLock; - /// Execution plan for reading in-memory batches of data #[derive(Clone)] #[deprecated( @@ -712,256 +707,6 @@ impl MemorySourceConfig { } } -/// Iterator over batches -pub struct MemoryStream { - /// Vector of record batches - data: Vec, - /// Optional memory reservation bound to the data, freed on drop - reservation: Option, - /// Schema representing the data - schema: SchemaRef, - /// Optional projection for which columns to load - projection: Option>, - /// Index into the data - index: usize, - /// The remaining number of rows to return. If None, all rows are returned - fetch: Option, -} - -impl MemoryStream { - /// Create an iterator for a vector of record batches - pub fn try_new( - data: Vec, - schema: SchemaRef, - projection: Option>, - ) -> Result { - Ok(Self { - data, - reservation: None, - schema, - projection, - index: 0, - fetch: None, - }) - } - - /// Set the memory reservation for the data - pub(super) fn with_reservation(mut self, reservation: MemoryReservation) -> Self { - self.reservation = Some(reservation); - self - } - - /// Set the number of rows to produce - pub(super) fn with_fetch(mut self, fetch: Option) -> Self { - self.fetch = fetch; - self - } -} - -impl Stream for MemoryStream { - type Item = Result; - - fn poll_next( - mut self: std::pin::Pin<&mut Self>, - _: &mut Context<'_>, - ) -> Poll> { - if self.index >= self.data.len() { - return Poll::Ready(None); - } - self.index += 1; - let batch = &self.data[self.index - 1]; - // return just the columns requested - let batch = match self.projection.as_ref() { - Some(columns) => batch.project(columns)?, - None => batch.clone(), - }; - - let Some(&fetch) = self.fetch.as_ref() else { - return Poll::Ready(Some(Ok(batch))); - }; - if fetch == 0 { - return Poll::Ready(None); - } - - let batch = if batch.num_rows() > fetch { - batch.slice(0, fetch) - } else { - batch - }; - self.fetch = Some(fetch - batch.num_rows()); - Poll::Ready(Some(Ok(batch))) - } - - fn size_hint(&self) -> (usize, Option) { - (self.data.len(), Some(self.data.len())) - } -} - -impl RecordBatchStream for MemoryStream { - /// Get the schema - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } -} - -pub trait LazyBatchGenerator: Send + Sync + fmt::Debug + fmt::Display { - /// Generate the next batch, return `None` when no more batches are available - fn generate_next_batch(&mut self) -> Result>; -} - -/// Execution plan for lazy in-memory batches of data -/// -/// This plan generates output batches lazily, it doesn't have to buffer all batches -/// in memory up front (compared to `MemorySourceConfig`), thus consuming constant memory. -pub struct LazyMemoryExec { - /// Schema representing the data - schema: SchemaRef, - /// Functions to generate batches for each partition - batch_generators: Vec>>, - /// Plan properties cache storing equivalence properties, partitioning, and execution mode - cache: PlanProperties, -} - -impl LazyMemoryExec { - /// Create a new lazy memory execution plan - pub fn try_new( - schema: SchemaRef, - generators: Vec>>, - ) -> Result { - let cache = PlanProperties::new( - EquivalenceProperties::new(Arc::clone(&schema)), - Partitioning::RoundRobinBatch(generators.len()), - EmissionType::Incremental, - Boundedness::Bounded, - ); - Ok(Self { - schema, - batch_generators: generators, - cache, - }) - } -} - -impl fmt::Debug for LazyMemoryExec { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - f.debug_struct("LazyMemoryExec") - .field("schema", &self.schema) - .field("batch_generators", &self.batch_generators) - .finish() - } -} - -impl DisplayAs for LazyMemoryExec { - fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!( - f, - "LazyMemoryExec: partitions={}, batch_generators=[{}]", - self.batch_generators.len(), - self.batch_generators - .iter() - .map(|g| g.read().to_string()) - .collect::>() - .join(", ") - ) - } - } - } -} - -impl ExecutionPlan for LazyMemoryExec { - fn name(&self) -> &'static str { - "LazyMemoryExec" - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } - - fn properties(&self) -> &PlanProperties { - &self.cache - } - - fn children(&self) -> Vec<&Arc> { - vec![] - } - - fn with_new_children( - self: Arc, - children: Vec>, - ) -> Result> { - if children.is_empty() { - Ok(self) - } else { - internal_err!("Children cannot be replaced in LazyMemoryExec") - } - } - - fn execute( - &self, - partition: usize, - _context: Arc, - ) -> Result { - if partition >= self.batch_generators.len() { - return internal_err!( - "Invalid partition {} for LazyMemoryExec with {} partitions", - partition, - self.batch_generators.len() - ); - } - - Ok(Box::pin(LazyMemoryStream { - schema: Arc::clone(&self.schema), - generator: Arc::clone(&self.batch_generators[partition]), - })) - } - - fn statistics(&self) -> Result { - Ok(Statistics::new_unknown(&self.schema)) - } -} - -/// Stream that generates record batches on demand -pub struct LazyMemoryStream { - schema: SchemaRef, - /// Generator to produce batches - /// - /// Note: Idiomatically, DataFusion uses plan-time parallelism - each stream - /// should have a unique `LazyBatchGenerator`. Use RepartitionExec or - /// construct multiple `LazyMemoryStream`s during planning to enable - /// parallel execution. - /// Sharing generators between streams should be used with caution. - generator: Arc>, -} - -impl Stream for LazyMemoryStream { - type Item = Result; - - fn poll_next( - self: std::pin::Pin<&mut Self>, - _: &mut Context<'_>, - ) -> Poll> { - let batch = self.generator.write().generate_next_batch(); - - match batch { - Ok(Some(batch)) => Poll::Ready(Some(Ok(batch))), - Ok(None) => Poll::Ready(None), - Err(e) => Poll::Ready(Some(Err(e))), - } - } -} - -impl RecordBatchStream for LazyMemoryStream { - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } -} - #[cfg(test)] mod memory_exec_tests { use std::sync::Arc; @@ -1018,126 +763,6 @@ mod memory_exec_tests { } } -#[cfg(test)] -mod lazy_memory_tests { - use super::*; - use arrow::array::Int64Array; - use arrow::datatypes::{DataType, Field, Schema}; - use futures::StreamExt; - - #[derive(Debug, Clone)] - struct TestGenerator { - counter: i64, - max_batches: i64, - batch_size: usize, - schema: SchemaRef, - } - - impl fmt::Display for TestGenerator { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!( - f, - "TestGenerator: counter={}, max_batches={}, batch_size={}", - self.counter, self.max_batches, self.batch_size - ) - } - } - - impl LazyBatchGenerator for TestGenerator { - fn generate_next_batch(&mut self) -> Result> { - if self.counter >= self.max_batches { - return Ok(None); - } - - let array = Int64Array::from_iter_values( - (self.counter * self.batch_size as i64) - ..(self.counter * self.batch_size as i64 + self.batch_size as i64), - ); - self.counter += 1; - Ok(Some(RecordBatch::try_new( - Arc::clone(&self.schema), - vec![Arc::new(array)], - )?)) - } - } - - #[tokio::test] - async fn test_lazy_memory_exec() -> Result<()> { - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)])); - let generator = TestGenerator { - counter: 0, - max_batches: 3, - batch_size: 2, - schema: Arc::clone(&schema), - }; - - let exec = - LazyMemoryExec::try_new(schema, vec![Arc::new(RwLock::new(generator))])?; - - // Test schema - assert_eq!(exec.schema().fields().len(), 1); - assert_eq!(exec.schema().field(0).name(), "a"); - - // Test execution - let stream = exec.execute(0, Arc::new(TaskContext::default()))?; - let batches: Vec<_> = stream.collect::>().await; - - assert_eq!(batches.len(), 3); - - // Verify batch contents - let batch0 = batches[0].as_ref().unwrap(); - let array0 = batch0 - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(array0.values(), &[0, 1]); - - let batch1 = batches[1].as_ref().unwrap(); - let array1 = batch1 - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(array1.values(), &[2, 3]); - - let batch2 = batches[2].as_ref().unwrap(); - let array2 = batch2 - .column(0) - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!(array2.values(), &[4, 5]); - - Ok(()) - } - - #[tokio::test] - async fn test_lazy_memory_exec_invalid_partition() -> Result<()> { - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)])); - let generator = TestGenerator { - counter: 0, - max_batches: 1, - batch_size: 1, - schema: Arc::clone(&schema), - }; - - let exec = - LazyMemoryExec::try_new(schema, vec![Arc::new(RwLock::new(generator))])?; - - // Test invalid partition - let result = exec.execute(1, Arc::new(TaskContext::default())); - - // partition is 0-indexed, so there only should be partition 0 - assert!(matches!( - result, - Err(e) if e.to_string().contains("Invalid partition 1 for LazyMemoryExec with 1 partitions") - )); - - Ok(()) - } -} - #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index 43be5f4d2ac1..ee5bc31a99cf 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -18,7 +18,6 @@ //! A table that uses the `ObjectStore` listing capability //! to get the list of files to process. -pub mod empty; pub mod file_compression_type; pub mod file_groups; pub mod file_meta; @@ -26,12 +25,8 @@ pub mod file_scan_config; pub mod file_sink_config; pub mod file_stream; pub mod memory; -pub mod placeholder_row; -pub mod recursive_query; pub mod source; pub mod url; -pub mod values; -pub mod work_table; pub mod write; use chrono::TimeZone; use datafusion_common::Result; diff --git a/datafusion/functions-table/src/generate_series.rs b/datafusion/functions-table/src/generate_series.rs index 8c460b08ef6a..887daa71ec55 100644 --- a/datafusion/functions-table/src/generate_series.rs +++ b/datafusion/functions-table/src/generate_series.rs @@ -23,8 +23,8 @@ use datafusion_catalog::Session; use datafusion_catalog::TableFunctionImpl; use datafusion_catalog::TableProvider; use datafusion_common::{plan_err, Result, ScalarValue}; -use datafusion_datasource::memory::{LazyBatchGenerator, LazyMemoryExec}; use datafusion_expr::{Expr, TableType}; +use datafusion_physical_plan::memory::{LazyBatchGenerator, LazyMemoryExec}; use datafusion_physical_plan::ExecutionPlan; use parking_lot::RwLock; use std::fmt; diff --git a/datafusion/physical-optimizer/src/aggregate_statistics.rs b/datafusion/physical-optimizer/src/aggregate_statistics.rs index a10f7de96460..a9b02188a7a2 100644 --- a/datafusion/physical-optimizer/src/aggregate_statistics.rs +++ b/datafusion/physical-optimizer/src/aggregate_statistics.rs @@ -20,8 +20,8 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::scalar::ScalarValue; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::Result; -use datafusion_datasource::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::aggregates::AggregateExec; +use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::udaf::{AggregateFunctionExpr, StatisticsArgs}; use datafusion_physical_plan::{expressions, ExecutionPlan}; diff --git a/datafusion/datasource/src/empty.rs b/datafusion/physical-plan/src/empty.rs similarity index 96% rename from datafusion/datasource/src/empty.rs rename to datafusion/physical-plan/src/empty.rs index 9b5854e9cb6c..c4e738cb3ad1 100644 --- a/datafusion/datasource/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -21,10 +21,8 @@ use std::any::Any; use std::sync::Arc; use crate::memory::MemoryStream; -use datafusion_physical_plan::{ - common, DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics, -}; -use datafusion_physical_plan::{ +use crate::{common, DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics}; +use crate::{ execution_plan::{Boundedness, EmissionType}, DisplayFormatType, ExecutionPlan, Partitioning, }; @@ -163,7 +161,7 @@ impl ExecutionPlan for EmptyExec { mod tests { use super::*; use crate::test; - use datafusion_physical_plan::with_new_children_if_necessary; + use crate::with_new_children_if_necessary; #[tokio::test] async fn empty() -> Result<()> { diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index ed0a38247e45..a66873bc6576 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -620,11 +620,10 @@ pub type EqualAndNonEqual<'a> = #[cfg(test)] mod tests { use super::*; - // use crate::empty::EmptyExec; + use crate::empty::EmptyExec; use crate::expressions::*; use crate::test; use crate::test::exec::StatisticsExec; - #[allow(unused_imports)] use arrow::datatypes::{Field, Schema, UnionFields, UnionMode}; use datafusion_common::ScalarValue; @@ -1281,36 +1280,36 @@ mod tests { Ok(()) } - // #[test] - // fn test_equivalence_properties_union_type() -> Result<()> { - // let union_type = DataType::Union( - // UnionFields::new( - // vec![0, 1], - // vec![ - // Field::new("f1", DataType::Int32, true), - // Field::new("f2", DataType::Utf8, true), - // ], - // ), - // UnionMode::Sparse, - // ); - - // let schema = Arc::new(Schema::new(vec![ - // Field::new("c1", DataType::Int32, true), - // Field::new("c2", union_type, true), - // ])); - - // let exec = FilterExec::try_new( - // binary( - // binary(col("c1", &schema)?, Operator::GtEq, lit(1i32), &schema)?, - // Operator::And, - // binary(col("c1", &schema)?, Operator::LtEq, lit(4i32), &schema)?, - // &schema, - // )?, - // Arc::new(EmptyExec::new(Arc::clone(&schema))), - // )?; - - // exec.statistics().unwrap(); - - // Ok(()) - // } + #[test] + fn test_equivalence_properties_union_type() -> Result<()> { + let union_type = DataType::Union( + UnionFields::new( + vec![0, 1], + vec![ + Field::new("f1", DataType::Int32, true), + Field::new("f2", DataType::Utf8, true), + ], + ), + UnionMode::Sparse, + ); + + let schema = Arc::new(Schema::new(vec![ + Field::new("c1", DataType::Int32, true), + Field::new("c2", union_type, true), + ])); + + let exec = FilterExec::try_new( + binary( + binary(col("c1", &schema)?, Operator::GtEq, lit(1i32), &schema)?, + Operator::And, + binary(col("c1", &schema)?, Operator::LtEq, lit(4i32), &schema)?, + &schema, + )?, + Arc::new(EmptyExec::new(Arc::clone(&schema))), + )?; + + exec.statistics().unwrap(); + + Ok(()) + } } diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index cc0ce7e30653..19a507996670 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -1090,7 +1090,7 @@ pub(crate) mod tests { source = source.try_with_sort_information(vec![sort_info]).unwrap(); } - Arc::new(MockMemorySourceConfig::new(Arc::new(source))) + Arc::new(MockMemorySourceConfig::update_cache(Arc::new(source))) } fn build_left_table() -> Arc { diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index c50d25dc2ddf..eca7215ce28e 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -536,8 +536,8 @@ pub fn create_memory_table( let right = MockMemorySourceConfig::try_new(&[right_partition], right_schema, None)? .try_with_sort_information(right_sorted)?; Ok(( - Arc::new(MockMemorySourceConfig::new(Arc::new(left))), - Arc::new(MockMemorySourceConfig::new(Arc::new(right))), + Arc::new(MockMemorySourceConfig::update_cache(Arc::new(left))), + Arc::new(MockMemorySourceConfig::update_cache(Arc::new(right))), )) } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 9ddcb0a7cae8..a73cf78ab7f4 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -56,14 +56,18 @@ pub mod coalesce_batches; pub mod coalesce_partitions; pub mod common; pub mod display; +pub mod empty; pub mod execution_plan; pub mod explain; pub mod filter; pub mod insert; pub mod joins; pub mod limit; +pub mod memory; pub mod metrics; +pub mod placeholder_row; pub mod projection; +pub mod recursive_query; pub mod repartition; pub mod sorts; pub mod spill; @@ -72,7 +76,9 @@ pub mod streaming; pub mod tree_node; pub mod union; pub mod unnest; +pub mod values; pub mod windows; +pub mod work_table; pub mod udaf { pub use datafusion_expr::StatisticsArgs; pub use datafusion_physical_expr::aggregate::AggregateFunctionExpr; diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs new file mode 100644 index 000000000000..fd338cc91353 --- /dev/null +++ b/datafusion/physical-plan/src/memory.rs @@ -0,0 +1,409 @@ +// 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. + +//! Execution plan for reading in-memory batches of data + +use std::any::Any; +use std::fmt; +use std::sync::Arc; +use std::task::{Context, Poll}; + +use crate::execution_plan::{Boundedness, EmissionType}; +use crate::{ + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, + RecordBatchStream, SendableRecordBatchStream, Statistics, +}; + +use arrow::array::RecordBatch; +use arrow::datatypes::SchemaRef; +use datafusion_common::{internal_err, Result}; +use datafusion_execution::memory_pool::MemoryReservation; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; + +use futures::Stream; +use parking_lot::RwLock; + +/// Iterator over batches +pub struct MemoryStream { + /// Vector of record batches + data: Vec, + /// Optional memory reservation bound to the data, freed on drop + reservation: Option, + /// Schema representing the data + schema: SchemaRef, + /// Optional projection for which columns to load + projection: Option>, + /// Index into the data + index: usize, + /// The remaining number of rows to return. If None, all rows are returned + fetch: Option, +} + +impl MemoryStream { + /// Create an iterator for a vector of record batches + pub fn try_new( + data: Vec, + schema: SchemaRef, + projection: Option>, + ) -> Result { + Ok(Self { + data, + reservation: None, + schema, + projection, + index: 0, + fetch: None, + }) + } + + /// Set the memory reservation for the data + pub fn with_reservation(mut self, reservation: MemoryReservation) -> Self { + self.reservation = Some(reservation); + self + } + + /// Set the number of rows to produce + pub fn with_fetch(mut self, fetch: Option) -> Self { + self.fetch = fetch; + self + } +} + +impl Stream for MemoryStream { + type Item = Result; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + _: &mut Context<'_>, + ) -> Poll> { + if self.index >= self.data.len() { + return Poll::Ready(None); + } + self.index += 1; + let batch = &self.data[self.index - 1]; + // return just the columns requested + let batch = match self.projection.as_ref() { + Some(columns) => batch.project(columns)?, + None => batch.clone(), + }; + + let Some(&fetch) = self.fetch.as_ref() else { + return Poll::Ready(Some(Ok(batch))); + }; + if fetch == 0 { + return Poll::Ready(None); + } + + let batch = if batch.num_rows() > fetch { + batch.slice(0, fetch) + } else { + batch + }; + self.fetch = Some(fetch - batch.num_rows()); + Poll::Ready(Some(Ok(batch))) + } + + fn size_hint(&self) -> (usize, Option) { + (self.data.len(), Some(self.data.len())) + } +} + +impl RecordBatchStream for MemoryStream { + /// Get the schema + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} + +pub trait LazyBatchGenerator: Send + Sync + fmt::Debug + fmt::Display { + /// Generate the next batch, return `None` when no more batches are available + fn generate_next_batch(&mut self) -> Result>; +} + +/// Execution plan for lazy in-memory batches of data +/// +/// This plan generates output batches lazily, it doesn't have to buffer all batches +/// in memory up front (compared to `MemorySourceConfig`), thus consuming constant memory. +pub struct LazyMemoryExec { + /// Schema representing the data + schema: SchemaRef, + /// Functions to generate batches for each partition + batch_generators: Vec>>, + /// Plan properties cache storing equivalence properties, partitioning, and execution mode + cache: PlanProperties, +} + +impl LazyMemoryExec { + /// Create a new lazy memory execution plan + pub fn try_new( + schema: SchemaRef, + generators: Vec>>, + ) -> Result { + let cache = PlanProperties::new( + EquivalenceProperties::new(Arc::clone(&schema)), + Partitioning::RoundRobinBatch(generators.len()), + EmissionType::Incremental, + Boundedness::Bounded, + ); + Ok(Self { + schema, + batch_generators: generators, + cache, + }) + } +} + +impl fmt::Debug for LazyMemoryExec { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_struct("LazyMemoryExec") + .field("schema", &self.schema) + .field("batch_generators", &self.batch_generators) + .finish() + } +} + +impl DisplayAs for LazyMemoryExec { + fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!( + f, + "LazyMemoryExec: partitions={}, batch_generators=[{}]", + self.batch_generators.len(), + self.batch_generators + .iter() + .map(|g| g.read().to_string()) + .collect::>() + .join(", ") + ) + } + } + } +} + +impl ExecutionPlan for LazyMemoryExec { + fn name(&self) -> &'static str { + "LazyMemoryExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn children(&self) -> Vec<&Arc> { + vec![] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + if children.is_empty() { + Ok(self) + } else { + internal_err!("Children cannot be replaced in LazyMemoryExec") + } + } + + fn execute( + &self, + partition: usize, + _context: Arc, + ) -> Result { + if partition >= self.batch_generators.len() { + return internal_err!( + "Invalid partition {} for LazyMemoryExec with {} partitions", + partition, + self.batch_generators.len() + ); + } + + Ok(Box::pin(LazyMemoryStream { + schema: Arc::clone(&self.schema), + generator: Arc::clone(&self.batch_generators[partition]), + })) + } + + fn statistics(&self) -> Result { + Ok(Statistics::new_unknown(&self.schema)) + } +} + +/// Stream that generates record batches on demand +pub struct LazyMemoryStream { + schema: SchemaRef, + /// Generator to produce batches + /// + /// Note: Idiomatically, DataFusion uses plan-time parallelism - each stream + /// should have a unique `LazyBatchGenerator`. Use RepartitionExec or + /// construct multiple `LazyMemoryStream`s during planning to enable + /// parallel execution. + /// Sharing generators between streams should be used with caution. + generator: Arc>, +} + +impl Stream for LazyMemoryStream { + type Item = Result; + + fn poll_next( + self: std::pin::Pin<&mut Self>, + _: &mut Context<'_>, + ) -> Poll> { + let batch = self.generator.write().generate_next_batch(); + + match batch { + Ok(Some(batch)) => Poll::Ready(Some(Ok(batch))), + Ok(None) => Poll::Ready(None), + Err(e) => Poll::Ready(Some(Err(e))), + } + } +} + +impl RecordBatchStream for LazyMemoryStream { + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} + +#[cfg(test)] +mod lazy_memory_tests { + use super::*; + use arrow::array::Int64Array; + use arrow::datatypes::{DataType, Field, Schema}; + use futures::StreamExt; + + #[derive(Debug, Clone)] + struct TestGenerator { + counter: i64, + max_batches: i64, + batch_size: usize, + schema: SchemaRef, + } + + impl fmt::Display for TestGenerator { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "TestGenerator: counter={}, max_batches={}, batch_size={}", + self.counter, self.max_batches, self.batch_size + ) + } + } + + impl LazyBatchGenerator for TestGenerator { + fn generate_next_batch(&mut self) -> Result> { + if self.counter >= self.max_batches { + return Ok(None); + } + + let array = Int64Array::from_iter_values( + (self.counter * self.batch_size as i64) + ..(self.counter * self.batch_size as i64 + self.batch_size as i64), + ); + self.counter += 1; + Ok(Some(RecordBatch::try_new( + Arc::clone(&self.schema), + vec![Arc::new(array)], + )?)) + } + } + + #[tokio::test] + async fn test_lazy_memory_exec() -> Result<()> { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)])); + let generator = TestGenerator { + counter: 0, + max_batches: 3, + batch_size: 2, + schema: Arc::clone(&schema), + }; + + let exec = + LazyMemoryExec::try_new(schema, vec![Arc::new(RwLock::new(generator))])?; + + // Test schema + assert_eq!(exec.schema().fields().len(), 1); + assert_eq!(exec.schema().field(0).name(), "a"); + + // Test execution + let stream = exec.execute(0, Arc::new(TaskContext::default()))?; + let batches: Vec<_> = stream.collect::>().await; + + assert_eq!(batches.len(), 3); + + // Verify batch contents + let batch0 = batches[0].as_ref().unwrap(); + let array0 = batch0 + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(array0.values(), &[0, 1]); + + let batch1 = batches[1].as_ref().unwrap(); + let array1 = batch1 + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(array1.values(), &[2, 3]); + + let batch2 = batches[2].as_ref().unwrap(); + let array2 = batch2 + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(array2.values(), &[4, 5]); + + Ok(()) + } + + #[tokio::test] + async fn test_lazy_memory_exec_invalid_partition() -> Result<()> { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)])); + let generator = TestGenerator { + counter: 0, + max_batches: 1, + batch_size: 1, + schema: Arc::clone(&schema), + }; + + let exec = + LazyMemoryExec::try_new(schema, vec![Arc::new(RwLock::new(generator))])?; + + // Test invalid partition + let result = exec.execute(1, Arc::new(TaskContext::default())); + + // partition is 0-indexed, so there only should be partition 0 + assert!(matches!( + result, + Err(e) if e.to_string().contains("Invalid partition 1 for LazyMemoryExec with 1 partitions") + )); + + Ok(()) + } +} diff --git a/datafusion/datasource/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs similarity index 95% rename from datafusion/datasource/src/placeholder_row.rs rename to datafusion/physical-plan/src/placeholder_row.rs index 6b16d1665f5a..6e31f601e152 100644 --- a/datafusion/datasource/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -20,18 +20,16 @@ use std::any::Any; use std::sync::Arc; +use crate::execution_plan::{Boundedness, EmissionType}; use crate::memory::MemoryStream; +use crate::{common, DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics}; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::array::{ArrayRef, NullArray}; use arrow::array::{RecordBatch, RecordBatchOptions}; use arrow::datatypes::{DataType, Field, Fields, Schema, SchemaRef}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::{ - common, DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics, -}; -use datafusion_physical_plan::{DisplayFormatType, ExecutionPlan, Partitioning}; use log::trace; @@ -181,7 +179,7 @@ impl ExecutionPlan for PlaceholderRowExec { mod tests { use super::*; use crate::test; - use datafusion_physical_plan::with_new_children_if_necessary; + use crate::with_new_children_if_necessary; #[test] fn with_new_children() -> Result<()> { diff --git a/datafusion/datasource/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs similarity index 97% rename from datafusion/datasource/src/recursive_query.rs rename to datafusion/physical-plan/src/recursive_query.rs index 4b3b39d003ee..05b78e4e1da4 100644 --- a/datafusion/datasource/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -22,12 +22,12 @@ use std::sync::Arc; use std::task::{Context, Poll}; use super::work_table::{ReservedBatches, WorkTable, WorkTableExec}; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::{ +use crate::execution_plan::{Boundedness, EmissionType}; +use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan}; +use crate::{DisplayAs, DisplayFormatType, ExecutionPlan}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; @@ -156,10 +156,10 @@ impl ExecutionPlan for RecursiveQueryExec { vec![false, false] } - fn required_input_distribution(&self) -> Vec { + fn required_input_distribution(&self) -> Vec { vec![ - datafusion_physical_plan::Distribution::SinglePartition, - datafusion_physical_plan::Distribution::SinglePartition, + crate::Distribution::SinglePartition, + crate::Distribution::SinglePartition, ] } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index efd42ea3de0b..f09fe0d40ea3 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1719,7 +1719,7 @@ mod test { schema: &SchemaRef, sort_exprs: LexOrdering, ) -> Arc { - Arc::new(MockMemorySourceConfig::new(Arc::new( + Arc::new(MockMemorySourceConfig::update_cache(Arc::new( MockMemorySourceConfig::try_new(&[vec![]], Arc::clone(schema), None) .unwrap() .try_with_sort_information(vec![sort_exprs]) diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index cede1041075d..a67d091795be 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -126,7 +126,7 @@ impl ExecutionPlan for MockMemorySourceConfig { } fn properties(&self) -> &PlanProperties { - return &self.cache; + &self.cache } fn children(&self) -> Vec<&Arc> { @@ -221,7 +221,7 @@ impl MockMemorySourceConfig { let projected_schema = project_schema(&schema, projection.as_ref())?; Ok(Self { partitions: partitions.to_vec(), - schema: schema, + schema, cache: PlanProperties::new( EquivalenceProperties::new_with_orderings( Arc::clone(&projected_schema), @@ -231,8 +231,8 @@ impl MockMemorySourceConfig { EmissionType::Incremental, Boundedness::Bounded, ), - projected_schema: projected_schema, - projection: projection, + projected_schema, + projection, sort_information: vec![], show_sizes: true, fetch: None, @@ -253,7 +253,7 @@ impl MockMemorySourceConfig { } // Equivalent of `DataSourceExec::new` - pub fn new(source: Arc) -> MockMemorySourceConfig { + pub fn update_cache(source: Arc) -> MockMemorySourceConfig { let cache = source.compute_properties(); let source = &*source; let mut source = source.clone(); diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index a20ae066a6fa..6d0af8b8b9b3 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -865,11 +865,11 @@ mod tests { .iter() .map(|ordering| convert_to_sort_exprs(ordering)) .collect::>(); - let child1 = Arc::new(MockMemorySourceConfig::new(Arc::new( + let child1 = Arc::new(MockMemorySourceConfig::update_cache(Arc::new( MockMemorySourceConfig::try_new(&[], Arc::clone(&schema), None)? .try_with_sort_information(first_orderings)?, ))); - let child2 = Arc::new(MockMemorySourceConfig::new(Arc::new( + let child2 = Arc::new(MockMemorySourceConfig::update_cache(Arc::new( MockMemorySourceConfig::try_new(&[], Arc::clone(&schema), None)? .try_with_sort_information(second_orderings)?, ))); diff --git a/datafusion/datasource/src/values.rs b/datafusion/physical-plan/src/values.rs similarity index 97% rename from datafusion/datasource/src/values.rs rename to datafusion/physical-plan/src/values.rs index 1f6d996b80ac..b90c50510cb0 100644 --- a/datafusion/datasource/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -20,19 +20,17 @@ use std::any::Any; use std::sync::Arc; +use crate::execution_plan::{Boundedness, EmissionType}; use crate::memory::MemoryStream; +use crate::{common, DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics}; +use crate::{ + ColumnarValue, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, +}; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::{internal_err, plan_err, Result, ScalarValue}; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::{ - common, DisplayAs, PlanProperties, SendableRecordBatchStream, Statistics, -}; -use datafusion_physical_plan::{ - ColumnarValue, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, -}; /// Execution plan for values list based relation (produces constant rows) #[deprecated( @@ -231,8 +229,8 @@ impl ExecutionPlan for ValuesExec { #[cfg(test)] mod tests { use super::*; + use crate::expressions::lit; use crate::test::{self, make_partition}; - use datafusion_physical_plan::expressions::lit; use arrow::datatypes::{DataType, Field}; use datafusion_common::stats::{ColumnStatistics, Precision}; diff --git a/datafusion/datasource/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs similarity index 97% rename from datafusion/datasource/src/work_table.rs rename to datafusion/physical-plan/src/work_table.rs index 6614b1d59f59..d3d29bfad7ce 100644 --- a/datafusion/datasource/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -20,15 +20,13 @@ use std::any::Any; use std::sync::{Arc, Mutex}; +use crate::execution_plan::{Boundedness, EmissionType}; use crate::memory::MemoryStream; -use datafusion_physical_plan::execution_plan::{Boundedness, EmissionType}; -use datafusion_physical_plan::{ +use crate::{ metrics::{ExecutionPlanMetricsSet, MetricsSet}, SendableRecordBatchStream, Statistics, }; -use datafusion_physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, -}; +use crate::{DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 356fc18f84e3..7e006adbdc87 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -41,6 +41,7 @@ use datafusion::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; use datafusion::physical_plan::analyze::AnalyzeExec; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::explain::ExplainExec; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::filter::FilterExec; @@ -51,6 +52,7 @@ use datafusion::physical_plan::joins::{ }; use datafusion::physical_plan::joins::{HashJoinExec, PartitionMode}; use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion::physical_plan::placeholder_row::PlaceholderRowExec; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; @@ -63,8 +65,6 @@ use datafusion::physical_plan::{ }; use datafusion_common::config::TableParquetOptions; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; -use datafusion_datasource::empty::EmptyExec; -use datafusion_datasource::placeholder_row::PlaceholderRowExec; use datafusion_datasource::source::DataSourceExec; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index eabfb4f87745..7418184fcac1 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -64,6 +64,7 @@ use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; use datafusion::physical_plan::analyze::AnalyzeExec; +use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::{ binary, cast, col, in_list, like, lit, BinaryExpr, Column, NotExpr, PhysicalSortExpr, }; @@ -73,6 +74,7 @@ use datafusion::physical_plan::joins::{ HashJoinExec, NestedLoopJoinExec, PartitionMode, StreamJoinPartitionMode, }; use datafusion::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion::physical_plan::placeholder_row::PlaceholderRowExec; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; @@ -95,8 +97,6 @@ use datafusion_common::stats::Precision; use datafusion_common::{ internal_err, not_impl_err, Constraints, DataFusionError, Result, UnnestOptions, }; -use datafusion_datasource::empty::EmptyExec; -use datafusion_datasource::placeholder_row::PlaceholderRowExec; use datafusion_expr::{ Accumulator, AccumulatorFactoryFunction, AggregateUDF, ColumnarValue, ScalarUDF, Signature, SimpleAggregateUDF, WindowFrame, WindowFrameBound, WindowUDF, From 7cfd35a90e5b163648edbcf3f0d8627a4607cd85 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Wed, 19 Feb 2025 00:40:41 +0530 Subject: [PATCH 11/18] clippy --- Cargo.lock | 1 - datafusion/datasource/Cargo.toml | 1 - datafusion/physical-plan/src/test.rs | 2 +- 3 files changed, 1 insertion(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index bb01c74685e5..fb40e2358e16 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1927,7 +1927,6 @@ dependencies = [ "itertools 0.14.0", "log", "object_store", - "parking_lot", "rand 0.8.5", "tempfile", "tokio", diff --git a/datafusion/datasource/Cargo.toml b/datafusion/datasource/Cargo.toml index 953b5ba2fd54..521c3f59e525 100644 --- a/datafusion/datasource/Cargo.toml +++ b/datafusion/datasource/Cargo.toml @@ -58,7 +58,6 @@ glob = "0.3.0" itertools = { workspace = true } log = { workspace = true } object_store = { workspace = true } -parking_lot = { workspace = true } rand = { workspace = true } tokio = { workspace = true } tokio-util = { version = "0.7.4", features = ["io"], optional = true } diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index 75ad1ed35d93..33afdb7fde84 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -25,13 +25,13 @@ use std::pin::Pin; use std::sync::Arc; use std::task::Context; +use crate::common; use crate::execution_plan::{Boundedness, EmissionType}; use crate::memory::MemoryStream; use crate::metrics::MetricsSet; use crate::stream::RecordBatchStreamAdapter; use crate::streaming::PartitionStream; use crate::ExecutionPlan; -use crate::{common, RecordBatchStream}; use crate::{DisplayAs, DisplayFormatType, PlanProperties}; use arrow::array::{Array, ArrayRef, Int32Array, RecordBatch}; From c70e1097189ff3399793b61cfc981d37132ab4f0 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Wed, 19 Feb 2025 00:57:37 +0530 Subject: [PATCH 12/18] =?UTF-8?q?=F0=9F=A7=B9=F0=9F=A7=B9*cleaning*?= =?UTF-8?q?=F0=9F=A7=B9=F0=9F=A7=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- datafusion/datasource/src/memory.rs | 40 +- datafusion/datasource/src/mod.rs | 1 - datafusion/datasource/src/test.rs | 177 ----- datafusion/datasource/src/test/exec.rs | 862 ------------------------- 4 files changed, 38 insertions(+), 1042 deletions(-) delete mode 100644 datafusion/datasource/src/test.rs delete mode 100644 datafusion/datasource/src/test/exec.rs diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 5074439fa3f8..47ecd5c29f13 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -766,7 +766,8 @@ mod memory_exec_tests { #[cfg(test)] mod tests { use super::*; - use crate::test::{self, make_partition}; + use arrow::array::{ArrayRef, Int32Array}; + use std::collections::HashMap; use datafusion_physical_plan::expressions::lit; use arrow::datatypes::{DataType, Field}; @@ -774,6 +775,18 @@ mod tests { use datafusion_common::stats::{ColumnStatistics, Precision}; use futures::StreamExt; + // Return a RecordBatch with a single Int32 array with values (0..sz) in a field named "i" + pub fn make_partition(sz: i32) -> RecordBatch { + let seq_start = 0; + let seq_end = sz; + let values = (seq_start..seq_end).collect::>(); + let schema = Arc::new(Schema::new(vec![Field::new("i", DataType::Int32, true)])); + let arr = Arc::new(Int32Array::from(values)); + let arr = arr as ArrayRef; + + RecordBatch::try_new(schema, vec![arr]).unwrap() + } + #[tokio::test] async fn exec_with_limit() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); @@ -800,9 +813,32 @@ mod tests { Ok(()) } + /// Get the schema for the aggregate_test_* csv files + pub fn aggr_test_schema() -> SchemaRef { + let mut f1 = Field::new("c1", DataType::Utf8, false); + f1.set_metadata(HashMap::from_iter(vec![("testing".into(), "test".into())])); + let schema = Schema::new(vec![ + f1, + Field::new("c2", DataType::UInt32, false), + Field::new("c3", DataType::Int8, false), + Field::new("c4", DataType::Int16, false), + Field::new("c5", DataType::Int32, false), + Field::new("c6", DataType::Int64, false), + Field::new("c7", DataType::UInt8, false), + Field::new("c8", DataType::UInt16, false), + Field::new("c9", DataType::UInt32, false), + Field::new("c10", DataType::UInt64, false), + Field::new("c11", DataType::Float32, false), + Field::new("c12", DataType::Float64, false), + Field::new("c13", DataType::Utf8, false), + ]); + + Arc::new(schema) + } + #[tokio::test] async fn values_empty_case() -> Result<()> { - let schema = test::aggr_test_schema(); + let schema = aggr_test_schema(); let empty = MemorySourceConfig::try_new_as_values(schema, vec![]); assert!(empty.is_err()); Ok(()) diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index ee5bc31a99cf..7b3aa8aa78a8 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -35,7 +35,6 @@ use futures::Stream; use object_store::{path::Path, ObjectMeta}; use std::pin::Pin; use std::sync::Arc; -pub mod test; pub use self::url::ListingTableUrl; diff --git a/datafusion/datasource/src/test.rs b/datafusion/datasource/src/test.rs deleted file mode 100644 index c6ca0efa5205..000000000000 --- a/datafusion/datasource/src/test.rs +++ /dev/null @@ -1,177 +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. - -//! Utilities for testing datafusion-physical-plan - -use std::collections::HashMap; -use std::pin::Pin; -use std::sync::Arc; - -use arrow::array::{ArrayRef, Int32Array, RecordBatch}; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use futures::{Future, FutureExt}; - -use crate::memory::MemorySourceConfig; -use crate::source::DataSourceExec; -use datafusion_physical_plan::stream::RecordBatchStreamAdapter; -use datafusion_physical_plan::streaming::PartitionStream; -use datafusion_physical_plan::ExecutionPlan; - -pub mod exec; -// pub use crate::test::exec; - -/// Asserts that given future is pending. -pub fn assert_is_pending<'a, T>(fut: &mut Pin + Send + 'a>>) { - let waker = futures::task::noop_waker(); - let mut cx = futures::task::Context::from_waker(&waker); - let poll = fut.poll_unpin(&mut cx); - - assert!(poll.is_pending()); -} - -/// Get the schema for the aggregate_test_* csv files -pub fn aggr_test_schema() -> SchemaRef { - let mut f1 = Field::new("c1", DataType::Utf8, false); - f1.set_metadata(HashMap::from_iter(vec![("testing".into(), "test".into())])); - let schema = Schema::new(vec![ - f1, - Field::new("c2", DataType::UInt32, false), - Field::new("c3", DataType::Int8, false), - Field::new("c4", DataType::Int16, false), - Field::new("c5", DataType::Int32, false), - Field::new("c6", DataType::Int64, false), - Field::new("c7", DataType::UInt8, false), - Field::new("c8", DataType::UInt16, false), - Field::new("c9", DataType::UInt32, false), - Field::new("c10", DataType::UInt64, false), - Field::new("c11", DataType::Float32, false), - Field::new("c12", DataType::Float64, false), - Field::new("c13", DataType::Utf8, false), - ]); - - Arc::new(schema) -} - -/// Returns record batch with 3 columns of i32 in memory -pub fn build_table_i32( - a: (&str, &Vec), - b: (&str, &Vec), - c: (&str, &Vec), -) -> RecordBatch { - let schema = Schema::new(vec![ - Field::new(a.0, DataType::Int32, false), - Field::new(b.0, DataType::Int32, false), - Field::new(c.0, DataType::Int32, false), - ]); - - RecordBatch::try_new( - Arc::new(schema), - vec![ - Arc::new(Int32Array::from(a.1.clone())), - Arc::new(Int32Array::from(b.1.clone())), - Arc::new(Int32Array::from(c.1.clone())), - ], - ) - .unwrap() -} - -/// Returns record batch with 2 columns of i32 in memory -pub fn build_table_i32_two_cols( - a: (&str, &Vec), - b: (&str, &Vec), -) -> RecordBatch { - let schema = Schema::new(vec![ - Field::new(a.0, DataType::Int32, false), - Field::new(b.0, DataType::Int32, false), - ]); - - RecordBatch::try_new( - Arc::new(schema), - vec![ - Arc::new(Int32Array::from(a.1.clone())), - Arc::new(Int32Array::from(b.1.clone())), - ], - ) - .unwrap() -} - -/// Returns memory table scan wrapped around record batch with 3 columns of i32 -pub fn build_table_scan_i32( - a: (&str, &Vec), - b: (&str, &Vec), - c: (&str, &Vec), -) -> Arc { - let batch = build_table_i32(a, b, c); - let schema = batch.schema(); - MemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() -} - -/// Return a RecordBatch with a single Int32 array with values (0..sz) in a field named "i" -pub fn make_partition(sz: i32) -> RecordBatch { - let seq_start = 0; - let seq_end = sz; - let values = (seq_start..seq_end).collect::>(); - let schema = Arc::new(Schema::new(vec![Field::new("i", DataType::Int32, true)])); - let arr = Arc::new(Int32Array::from(values)); - let arr = arr as ArrayRef; - - RecordBatch::try_new(schema, vec![arr]).unwrap() -} - -/// Returns a `DataSourceExec` that scans `partitions` of 100 batches each -pub fn scan_partitioned(partitions: usize) -> Arc { - Arc::new(mem_exec(partitions)) -} - -/// Returns a `DataSourceExec` that scans `partitions` of 100 batches each -pub fn mem_exec(partitions: usize) -> DataSourceExec { - let data: Vec> = (0..partitions).map(|_| vec![make_partition(100)]).collect(); - - let schema = data[0][0].schema(); - let projection = None; - DataSourceExec::new(Arc::new( - MemorySourceConfig::try_new(&data, schema, projection).unwrap(), - )) -} - -// Construct a stream partition for test purposes -#[derive(Debug)] -pub struct TestPartitionStream { - pub schema: SchemaRef, - pub batches: Vec, -} - -impl TestPartitionStream { - /// Create a new stream partition with the provided batches - pub fn new_with_batches(batches: Vec) -> Self { - let schema = batches[0].schema(); - Self { schema, batches } - } -} -impl PartitionStream for TestPartitionStream { - fn schema(&self) -> &SchemaRef { - &self.schema - } - fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { - let stream = futures::stream::iter(self.batches.clone().into_iter().map(Ok)); - Box::pin(RecordBatchStreamAdapter::new( - Arc::clone(&self.schema), - stream, - )) - } -} diff --git a/datafusion/datasource/src/test/exec.rs b/datafusion/datasource/src/test/exec.rs deleted file mode 100644 index de8fc7e9d1fd..000000000000 --- a/datafusion/datasource/src/test/exec.rs +++ /dev/null @@ -1,862 +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. - -//! Simple iterator over batches for use in testing - -use std::{ - any::Any, - pin::Pin, - sync::{Arc, Weak}, - task::{Context, Poll}, -}; - -use datafusion_physical_plan::{ - common, execution_plan::Boundedness, DisplayAs, DisplayFormatType, ExecutionPlan, - Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, - Statistics, -}; -use datafusion_physical_plan::{ - execution_plan::EmissionType, - stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}, -}; - -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use arrow::record_batch::RecordBatch; -use datafusion_common::{internal_err, DataFusionError, Result}; -use datafusion_execution::TaskContext; -use datafusion_physical_expr::EquivalenceProperties; - -use futures::Stream; -use tokio::sync::Barrier; - -/// Index into the data that has been returned so far -#[derive(Debug, Default, Clone)] -pub struct BatchIndex { - inner: Arc>, -} - -impl BatchIndex { - /// Return the current index - pub fn value(&self) -> usize { - let inner = self.inner.lock().unwrap(); - *inner - } - - // increment the current index by one - pub fn incr(&self) { - let mut inner = self.inner.lock().unwrap(); - *inner += 1; - } -} - -/// Iterator over batches -#[derive(Debug, Default)] -pub struct TestStream { - /// Vector of record batches - data: Vec, - /// Index into the data that has been returned so far - index: BatchIndex, -} - -impl TestStream { - /// Create an iterator for a vector of record batches. Assumes at - /// least one entry in data (for the schema) - pub fn new(data: Vec) -> Self { - Self { - data, - ..Default::default() - } - } - - /// Return a handle to the index counter for this stream - pub fn index(&self) -> BatchIndex { - self.index.clone() - } -} - -impl Stream for TestStream { - type Item = Result; - - fn poll_next(self: Pin<&mut Self>, _: &mut Context<'_>) -> Poll> { - let next_batch = self.index.value(); - - Poll::Ready(if next_batch < self.data.len() { - let next_batch = self.index.value(); - self.index.incr(); - Some(Ok(self.data[next_batch].clone())) - } else { - None - }) - } - - fn size_hint(&self) -> (usize, Option) { - (self.data.len(), Some(self.data.len())) - } -} - -impl RecordBatchStream for TestStream { - /// Get the schema - fn schema(&self) -> SchemaRef { - self.data[0].schema() - } -} - -/// A Mock ExecutionPlan that can be used for writing tests of other -/// ExecutionPlans -#[derive(Debug)] -pub struct MockExec { - /// the results to send back - data: Vec>, - schema: SchemaRef, - /// if true (the default), sends data using a separate task to ensure the - /// batches are not available without this stream yielding first - use_task: bool, - cache: PlanProperties, -} - -impl MockExec { - /// Create a new `MockExec` with a single partition that returns - /// the specified `Results`s. - /// - /// By default, the batches are not produced immediately (the - /// caller has to actually yield and another task must run) to - /// ensure any poll loops are correct. This behavior can be - /// changed with `with_use_task` - pub fn new(data: Vec>, schema: SchemaRef) -> Self { - let cache = Self::compute_properties(Arc::clone(&schema)); - Self { - data, - schema, - use_task: true, - cache, - } - } - - /// If `use_task` is true (the default) then the batches are sent - /// back using a separate task to ensure the underlying stream is - /// not immediately ready - pub fn with_use_task(mut self, use_task: bool) -> Self { - self.use_task = use_task; - self - } - - /// 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(1), - EmissionType::Incremental, - Boundedness::Bounded, - ) - } -} - -impl DisplayAs for MockExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "MockExec") - } - } - } -} - -impl ExecutionPlan for MockExec { - 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> { - unimplemented!() - } - - /// Returns a stream which yields data - fn execute( - &self, - partition: usize, - _context: Arc, - ) -> Result { - assert_eq!(partition, 0); - - // Result doesn't implement clone, so do it ourself - let data: Vec<_> = self - .data - .iter() - .map(|r| match r { - Ok(batch) => Ok(batch.clone()), - Err(e) => Err(clone_error(e)), - }) - .collect(); - - if self.use_task { - let mut builder = RecordBatchReceiverStream::builder(self.schema(), 2); - // send data in order but in a separate task (to ensure - // the batches are not available without the stream - // yielding). - let tx = builder.tx(); - builder.spawn(async move { - for batch in data { - println!("Sending batch via delayed stream"); - if let Err(e) = tx.send(batch).await { - println!("ERROR batch via delayed stream: {e}"); - } - } - - Ok(()) - }); - // returned stream simply reads off the rx stream - Ok(builder.build()) - } else { - // make an input that will error - let stream = futures::stream::iter(data); - Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema(), - stream, - ))) - } - } - - // Panics if one of the batches is an error - fn statistics(&self) -> Result { - let data: Result> = self - .data - .iter() - .map(|r| match r { - Ok(batch) => Ok(batch.clone()), - Err(e) => Err(clone_error(e)), - }) - .collect(); - - let data = data?; - - Ok(common::compute_record_batch_statistics( - &[data], - &self.schema, - None, - )) - } -} - -fn clone_error(e: &DataFusionError) -> DataFusionError { - use DataFusionError::*; - match e { - Execution(msg) => Execution(msg.to_string()), - _ => unimplemented!(), - } -} - -/// A Mock ExecutionPlan that does not start producing input until a -/// barrier is called -/// -#[derive(Debug)] -pub struct BarrierExec { - /// partitions to send back - data: Vec>, - schema: SchemaRef, - - /// all streams wait on this barrier to produce - barrier: Arc, - cache: PlanProperties, -} - -impl BarrierExec { - /// Create a new exec with some number of partitions. - pub fn new(data: Vec>, schema: SchemaRef) -> Self { - // wait for all streams and the input - let barrier = Arc::new(Barrier::new(data.len() + 1)); - let cache = Self::compute_properties(Arc::clone(&schema), &data); - Self { - data, - schema, - barrier, - cache, - } - } - - /// wait until all the input streams and this function is ready - pub async fn wait(&self) { - println!("BarrierExec::wait waiting on barrier"); - self.barrier.wait().await; - println!("BarrierExec::wait done waiting"); - } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties( - schema: SchemaRef, - data: &[Vec], - ) -> PlanProperties { - PlanProperties::new( - EquivalenceProperties::new(schema), - Partitioning::UnknownPartitioning(data.len()), - EmissionType::Incremental, - Boundedness::Bounded, - ) - } -} - -impl DisplayAs for BarrierExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "BarrierExec") - } - } - } -} - -impl ExecutionPlan for BarrierExec { - 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> { - unimplemented!() - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - unimplemented!() - } - - /// Returns a stream which yields data - fn execute( - &self, - partition: usize, - _context: Arc, - ) -> Result { - assert!(partition < self.data.len()); - - let mut builder = RecordBatchReceiverStream::builder(self.schema(), 2); - - // task simply sends data in order after barrier is reached - let data = self.data[partition].clone(); - let b = Arc::clone(&self.barrier); - let tx = builder.tx(); - builder.spawn(async move { - println!("Partition {partition} waiting on barrier"); - b.wait().await; - for batch in data { - println!("Partition {partition} sending batch"); - if let Err(e) = tx.send(Ok(batch)).await { - println!("ERROR batch via barrier stream stream: {e}"); - } - } - - Ok(()) - }); - - // returned stream simply reads off the rx stream - Ok(builder.build()) - } - - fn statistics(&self) -> Result { - Ok(common::compute_record_batch_statistics( - &self.data, - &self.schema, - None, - )) - } -} - -/// A mock execution plan that errors on a call to execute -#[derive(Debug)] -pub struct ErrorExec { - cache: PlanProperties, -} - -impl Default for ErrorExec { - fn default() -> Self { - Self::new() - } -} - -impl ErrorExec { - pub fn new() -> Self { - let schema = Arc::new(Schema::new(vec![Field::new( - "dummy", - DataType::Int64, - true, - )])); - let cache = Self::compute_properties(schema); - Self { 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(1), - EmissionType::Incremental, - Boundedness::Bounded, - ) - } -} - -impl DisplayAs for ErrorExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "ErrorExec") - } - } - } -} - -impl ExecutionPlan for ErrorExec { - 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> { - unimplemented!() - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - unimplemented!() - } - - /// Returns a stream which yields data - fn execute( - &self, - partition: usize, - _context: Arc, - ) -> Result { - internal_err!("ErrorExec, unsurprisingly, errored in partition {partition}") - } -} - -/// 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()) - } -} - -/// Execution plan that emits streams that block forever. -/// -/// This is useful to test shutdown / cancellation behavior of certain execution plans. -#[derive(Debug)] -pub struct BlockingExec { - /// Schema that is mocked by this plan. - schema: SchemaRef, - - /// Ref-counting helper to check if the plan and the produced stream are still in memory. - refs: Arc<()>, - cache: PlanProperties, -} - -impl BlockingExec { - /// Create new [`BlockingExec`] with a give schema and number of partitions. - pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { - let cache = Self::compute_properties(Arc::clone(&schema), n_partitions); - Self { - schema, - refs: Default::default(), - cache, - } - } - - /// Weak pointer that can be used for ref-counting this execution plan and its streams. - /// - /// Use [`Weak::strong_count`] to determine if the plan itself and its streams are dropped (should be 0 in that - /// case). Note that tokio might take some time to cancel spawned tasks, so you need to wrap this check into a retry - /// loop. Use [`assert_strong_count_converges_to_zero`] to archive this. - pub fn refs(&self) -> Weak<()> { - Arc::downgrade(&self.refs) - } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties(schema: SchemaRef, n_partitions: usize) -> PlanProperties { - PlanProperties::new( - EquivalenceProperties::new(schema), - Partitioning::UnknownPartitioning(n_partitions), - EmissionType::Incremental, - Boundedness::Bounded, - ) - } -} - -impl DisplayAs for BlockingExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "BlockingExec",) - } - } - } -} - -impl ExecutionPlan for BlockingExec { - 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> { - // this is a leaf node and has no children - vec![] - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - internal_err!("Children cannot be replaced in {self:?}") - } - - fn execute( - &self, - _partition: usize, - _context: Arc, - ) -> Result { - Ok(Box::pin(BlockingStream { - schema: Arc::clone(&self.schema), - _refs: Arc::clone(&self.refs), - })) - } -} - -/// A [`RecordBatchStream`] that is pending forever. -#[derive(Debug)] -pub struct BlockingStream { - /// Schema mocked by this stream. - schema: SchemaRef, - - /// Ref-counting helper to check if the stream are still in memory. - _refs: Arc<()>, -} - -impl Stream for BlockingStream { - type Item = Result; - - fn poll_next( - self: Pin<&mut Self>, - _cx: &mut Context<'_>, - ) -> Poll> { - Poll::Pending - } -} - -impl RecordBatchStream for BlockingStream { - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } -} - -/// Asserts that the strong count of the given [`Weak`] pointer converges to zero. -/// -/// This might take a while but has a timeout. -pub async fn assert_strong_count_converges_to_zero(refs: Weak) { - tokio::time::timeout(std::time::Duration::from_secs(10), async { - loop { - if dbg!(Weak::strong_count(&refs)) == 0 { - break; - } - tokio::time::sleep(std::time::Duration::from_millis(10)).await; - } - }) - .await - .unwrap(); -} - -/// Execution plan that emits streams that panics. -/// -/// This is useful to test panic handling of certain execution plans. -#[derive(Debug)] -pub struct PanicExec { - /// Schema that is mocked by this plan. - schema: SchemaRef, - - /// Number of output partitions. Each partition will produce this - /// many empty output record batches prior to panicking - batches_until_panics: Vec, - cache: PlanProperties, -} - -impl PanicExec { - /// Create new [`PanicExec`] with a give schema and number of - /// partitions, which will each panic immediately. - pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { - let batches_until_panics = vec![0; n_partitions]; - let cache = Self::compute_properties(Arc::clone(&schema), &batches_until_panics); - Self { - schema, - batches_until_panics, - cache, - } - } - - /// Set the number of batches prior to panic for a partition - pub fn with_partition_panic(mut self, partition: usize, count: usize) -> Self { - self.batches_until_panics[partition] = count; - self - } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties( - schema: SchemaRef, - batches_until_panics: &[usize], - ) -> PlanProperties { - let num_partitions = batches_until_panics.len(); - PlanProperties::new( - EquivalenceProperties::new(schema), - Partitioning::UnknownPartitioning(num_partitions), - EmissionType::Incremental, - Boundedness::Bounded, - ) - } -} - -impl DisplayAs for PanicExec { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!(f, "PanicExec",) - } - } - } -} - -impl ExecutionPlan for PanicExec { - 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> { - // this is a leaf node and has no children - vec![] - } - - fn with_new_children( - self: Arc, - _: Vec>, - ) -> Result> { - internal_err!("Children cannot be replaced in {:?}", self) - } - - fn execute( - &self, - partition: usize, - _context: Arc, - ) -> Result { - Ok(Box::pin(PanicStream { - partition, - batches_until_panic: self.batches_until_panics[partition], - schema: Arc::clone(&self.schema), - ready: false, - })) - } -} - -/// A [`RecordBatchStream`] that yields every other batch and panics -/// after `batches_until_panic` batches have been produced. -/// -/// Useful for testing the behavior of streams on panic -#[derive(Debug)] -struct PanicStream { - /// Which partition was this - partition: usize, - /// How may batches will be produced until panic - batches_until_panic: usize, - /// Schema mocked by this stream. - schema: SchemaRef, - /// Should we return ready ? - ready: bool, -} - -impl Stream for PanicStream { - type Item = Result; - - fn poll_next( - mut self: Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - if self.batches_until_panic > 0 { - if self.ready { - self.batches_until_panic -= 1; - self.ready = false; - let batch = RecordBatch::new_empty(Arc::clone(&self.schema)); - return Poll::Ready(Some(Ok(batch))); - } else { - self.ready = true; - // get called again - cx.waker().wake_by_ref(); - return Poll::Pending; - } - } - panic!("PanickingStream did panic: {}", self.partition) - } -} - -impl RecordBatchStream for PanicStream { - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } -} From e9966f81396ad3f7c0eea8d954b365469bfe2765 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Wed, 19 Feb 2025 02:03:36 +0530 Subject: [PATCH 13/18] adding re-export --- Cargo.lock | 162 +++++++++--------- datafusion-cli/Cargo.toml | 1 - datafusion-cli/src/functions.rs | 2 +- datafusion-examples/Cargo.toml | 1 - .../examples/parquet_exec_visitor.rs | 2 +- .../examples/remote_catalog.rs | 2 +- datafusion-examples/examples/simple_udtf.rs | 2 +- .../src/datasource/avro_to_arrow/reader.rs | 2 +- .../core/src/datasource/listing/table.rs | 14 +- datafusion/core/src/datasource/memory.rs | 4 +- datafusion/core/src/datasource/mod.rs | 2 + .../core/src/datasource/physical_plan/csv.rs | 4 +- .../physical_plan/file_scan_config.rs | 6 +- .../physical_plan/parquet/access_plan.rs | 2 +- .../physical_plan/parquet/source.rs | 18 +- .../core/src/datasource/schema_adapter.rs | 2 +- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 4 +- datafusion/core/tests/fuzz_cases/join_fuzz.rs | 4 +- .../core/tests/fuzz_cases/merge_fuzz.rs | 2 +- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 2 +- .../sort_preserving_repartition_fuzz.rs | 4 +- .../core/tests/fuzz_cases/window_fuzz.rs | 4 +- datafusion/core/tests/memory_limit/mod.rs | 4 +- .../core/tests/parquet/file_statistics.rs | 2 +- datafusion/core/tests/parquet/page_pruning.rs | 2 +- datafusion/core/tests/parquet/utils.rs | 2 +- .../aggregate_statistics.rs | 4 +- .../enforce_distribution.rs | 2 +- .../physical_optimizer/projection_pushdown.rs | 4 +- .../replace_with_order_preserving_variants.rs | 4 +- .../tests/physical_optimizer/test_utils.rs | 4 +- datafusion/core/tests/sql/path_partition.rs | 2 +- .../user_defined_table_functions.rs | 2 +- datafusion/datasource/src/memory.rs | 2 +- datafusion/functions-table/Cargo.toml | 1 - datafusion/proto/Cargo.toml | 1 - datafusion/proto/src/physical_plan/mod.rs | 2 +- datafusion/substrait/Cargo.toml | 1 - .../substrait/src/physical_plan/producer.rs | 2 +- 39 files changed, 141 insertions(+), 146 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index fb40e2358e16..b05e511f843e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -361,9 +361,9 @@ dependencies = [ [[package]] name = "arrow-flight" -version = "54.1.0" +version = "54.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9b3aaba47ed4b6146563c8b79ad0f7aa283f794cde0c057c656291b81196746" +checksum = "cf7806ee3d229ee866013e83446e937ab3c8a9e6a664b259d41dd960b309c5d0" dependencies = [ "arrow-arith", "arrow-array", @@ -674,9 +674,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.58.0" +version = "1.59.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16ff718c9ee45cc1ebd4774a0e086bb80a6ab752b4902edf1c9f56b86ee1f770" +checksum = "00a35fc7e74f5be45839eb753568535c074a592185dd0a2d406685018d581c43" dependencies = [ "aws-credential-types", "aws-runtime", @@ -696,9 +696,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.59.0" +version = "1.60.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5183e088715cc135d8d396fdd3bc02f018f0da4c511f53cb8d795b6a31c55809" +checksum = "f8fa655b4f313124ce272cbc38c5fef13793c832279cec750103e5e6b71a54b8" dependencies = [ "aws-credential-types", "aws-runtime", @@ -718,9 +718,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.59.0" +version = "1.60.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9f944ef032717596639cea4a2118a3a457268ef51bbb5fde9637e54c465da00" +checksum = "dc1cfe5e16b90421ea031f4c6348b534ef442e76f6bf4a1b2b592c12cc2c6af9" dependencies = [ "aws-credential-types", "aws-runtime", @@ -741,9 +741,9 @@ dependencies = [ [[package]] name = "aws-sigv4" -version = "1.2.8" +version = "1.2.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0bc5bbd1e4a2648fd8c5982af03935972c24a2f9846b396de661d351ee3ce837" +checksum = "9bfe75fad52793ce6dec0dc3d4b1f388f038b5eb866c8d4d7f3a8e21b5ea5051" dependencies = [ "aws-credential-types", "aws-smithy-http", @@ -1035,15 +1035,16 @@ dependencies = [ [[package]] name = "blake3" -version = "1.5.5" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8ee0c1824c4dea5b5f81736aff91bae041d2c07ee1192bec91054e10e3e601e" +checksum = "1230237285e3e10cde447185e8975408ae24deaa67205ce684805c25bc0c7937" dependencies = [ "arrayref", "arrayvec", "cc", "cfg-if", "constant_time_eq", + "memmap2", ] [[package]] @@ -1077,7 +1078,7 @@ dependencies = [ "hyperlocal", "log", "pin-project-lite", - "rustls 0.23.22", + "rustls 0.23.23", "rustls-native-certs 0.8.1", "rustls-pemfile 2.2.0", "rustls-pki-types", @@ -1248,9 +1249,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.2.13" +version = "1.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7777341816418c02e033934a09f20dc0ccaf65a5201ef8a450ae0105a573fda" +checksum = "0c3d1b2e905a3a7b00a6141adb0e4c0bb941d11caf55349d863942a1cc44e3c9" dependencies = [ "jobserver", "libc", @@ -1345,9 +1346,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.29" +version = "4.5.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8acebd8ad879283633b343856142139f2da2317c96b05b4dd6181c61e2480184" +checksum = "92b7b18d71fad5313a1e320fa9897994228ce274b60faa4d694fe0ea89cd9e6d" dependencies = [ "clap_builder", "clap_derive", @@ -1355,9 +1356,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.29" +version = "4.5.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6ba32cbda51c7e1dfd49acc1457ba1a7dec5b64fe360e828acb13ca8dc9c2f9" +checksum = "a35db2071778a7344791a4fb4f95308b5673d219dee3ae348b86642574ecc90c" dependencies = [ "anstream", "anstyle", @@ -1549,7 +1550,7 @@ dependencies = [ "anes", "cast", "ciborium", - "clap 4.5.29", + "clap 4.5.30", "criterion-plot", "futures", "is-terminal", @@ -1642,9 +1643,9 @@ dependencies = [ [[package]] name = "csv-core" -version = "0.1.11" +version = "0.1.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5efa2b3d7902f4b634a20cae3c9c4e6209dc4779feb6863329607560143efa70" +checksum = "7d02f3b0da4c6504f86e9cd789d8dbafab48c2321be74e9987593de5a894d93d" dependencies = [ "memchr", ] @@ -1850,10 +1851,9 @@ dependencies = [ "async-trait", "aws-config", "aws-credential-types", - "clap 4.5.29", + "clap 4.5.30", "ctor", "datafusion", - "datafusion-datasource", "dirs", "env_logger", "futures", @@ -1951,7 +1951,6 @@ dependencies = [ "bytes", "dashmap", "datafusion", - "datafusion-datasource", "datafusion-proto", "env_logger", "futures", @@ -2128,7 +2127,6 @@ dependencies = [ "async-trait", "datafusion-catalog", "datafusion-common", - "datafusion-datasource", "datafusion-expr", "datafusion-physical-plan", "parking_lot", @@ -2290,7 +2288,6 @@ dependencies = [ "chrono", "datafusion", "datafusion-common", - "datafusion-datasource", "datafusion-expr", "datafusion-functions", "datafusion-functions-aggregate", @@ -2351,7 +2348,7 @@ dependencies = [ "bigdecimal", "bytes", "chrono", - "clap 4.5.29", + "clap 4.5.30", "datafusion", "env_logger", "futures", @@ -2381,7 +2378,6 @@ dependencies = [ "async-trait", "chrono", "datafusion", - "datafusion-datasource", "datafusion-functions-aggregate", "itertools 0.14.0", "object_store", @@ -2571,9 +2567,9 @@ dependencies = [ [[package]] name = "equivalent" -version = "1.0.1" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" +checksum = "877a4ace8713b0bcf2a4e7eec82529c029f1d0619886d18145fea96c3ffe5c0f" [[package]] name = "errno" @@ -2932,9 +2928,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.4.7" +version = "0.4.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ccae279728d634d083c00f6099cb58f01cc99c145b84b8be2f6c74618d79922e" +checksum = "5017294ff4bb30944501348f6f8e42e6ad28f42c8bbef7a74029aff064a4e3c2" dependencies = [ "atomic-waker", "bytes", @@ -3142,7 +3138,7 @@ dependencies = [ "bytes", "futures-channel", "futures-util", - "h2 0.4.7", + "h2 0.4.8", "http 1.2.0", "http-body 1.0.1", "httparse", @@ -3195,7 +3191,7 @@ dependencies = [ "http 1.2.0", "hyper 1.6.0", "hyper-util", - "rustls 0.23.22", + "rustls 0.23.23", "rustls-native-certs 0.8.1", "rustls-pki-types", "tokio", @@ -3685,7 +3681,7 @@ checksum = "5297962ef19edda4ce33aaa484386e0a5b3d7f2f4e037cbeee00503ef6b29d33" dependencies = [ "anstream", "anstyle", - "clap 4.5.29", + "clap 4.5.30", "escape8259", ] @@ -3759,6 +3755,15 @@ version = "2.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "78ca9ab1a0babb1e7d5695e3530886289c18cf2f87ec19a575a0abdce112e3a3" +[[package]] +name = "memmap2" +version = "0.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fd3f7eed9d3848f8b98834af67102b720745c4ec028fcd0aa0239277e7de374f" +dependencies = [ + "libc", +] + [[package]] name = "memoffset" version = "0.9.1" @@ -3795,9 +3800,9 @@ dependencies = [ [[package]] name = "miniz_oxide" -version = "0.8.3" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8402cab7aefae129c6977bb0ff1b8fd9a04eb5b51efc50a70bea51cda0c7924" +checksum = "b3b1c9bd4fe1f0f8b387f6eb9eb3b4a1aa26185e5750efb9140301703f62cd1b" dependencies = [ "adler2", ] @@ -4052,9 +4057,9 @@ dependencies = [ [[package]] name = "parquet" -version = "54.1.0" +version = "54.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a01a0efa30bbd601ae85b375c728efdb211ade54390281628a7b16708beb235" +checksum = "761c44d824fe83106e0600d2510c07bf4159a4985bf0569b513ea4288dc1b4fb" dependencies = [ "ahash 0.8.11", "arrow-array", @@ -4493,9 +4498,9 @@ dependencies = [ [[package]] name = "psm" -version = "0.1.24" +version = "0.1.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "200b9ff220857e53e184257720a14553b2f4aa02577d2ed9842d45d4b9654810" +checksum = "f58e5423e24c18cc840e1c98370b3993c6649cd1678b4d24318bcf0a083cbe88" dependencies = [ "cc", ] @@ -4610,7 +4615,7 @@ dependencies = [ "quinn-proto", "quinn-udp", "rustc-hash", - "rustls 0.23.22", + "rustls 0.23.23", "socket2", "thiserror 2.0.11", "tokio", @@ -4628,7 +4633,7 @@ dependencies = [ "rand 0.8.5", "ring", "rustc-hash", - "rustls 0.23.22", + "rustls 0.23.23", "rustls-pki-types", "slab", "thiserror 2.0.11", @@ -4639,9 +4644,9 @@ dependencies = [ [[package]] name = "quinn-udp" -version = "0.5.9" +version = "0.5.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1c40286217b4ba3a71d644d752e6a0b71f13f1b6a2c5311acfcbe0c2418ed904" +checksum = "e46f3055866785f6b92bc6164b76be02ca8f2eb4b002c0354b28cf4c119e5944" dependencies = [ "cfg_aliases", "libc", @@ -4694,8 +4699,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3779b94aeb87e8bd4e834cee3650289ee9e0d5677f976ecdb6d219e5f4f6cd94" dependencies = [ "rand_chacha 0.9.0", - "rand_core 0.9.0", - "zerocopy 0.8.17", + "rand_core 0.9.1", + "zerocopy 0.8.18", ] [[package]] @@ -4715,7 +4720,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d3022b5f1df60f26e1ffddd6c66e8aa15de382ae63b3a0c1bfc0e4d3e3f325cb" dependencies = [ "ppv-lite86", - "rand_core 0.9.0", + "rand_core 0.9.1", ] [[package]] @@ -4729,12 +4734,12 @@ dependencies = [ [[package]] name = "rand_core" -version = "0.9.0" +version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b08f3c9802962f7e1b25113931d94f43ed9725bebc59db9d0c3e9a23b67e15ff" +checksum = "a88e0da7a2c97baa202165137c158d0a2e824ac465d13d81046727b34cb247d3" dependencies = [ "getrandom 0.3.1", - "zerocopy 0.8.17", + "zerocopy 0.8.18", ] [[package]] @@ -4895,7 +4900,7 @@ dependencies = [ "bytes", "futures-core", "futures-util", - "h2 0.4.7", + "h2 0.4.8", "http 1.2.0", "http-body 1.0.1", "http-body-util", @@ -4910,7 +4915,7 @@ dependencies = [ "percent-encoding", "pin-project-lite", "quinn", - "rustls 0.23.22", + "rustls 0.23.23", "rustls-native-certs 0.8.1", "rustls-pemfile 2.2.0", "rustls-pki-types", @@ -4933,15 +4938,14 @@ dependencies = [ [[package]] name = "ring" -version = "0.17.8" +version = "0.17.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c17fa4cb658e3583423e915b9f3acc01cceaee1860e33d59ebae66adc3a2dc0d" +checksum = "e75ec5e92c4d8aede845126adc388046234541629e76029599ed35a003c7ed24" dependencies = [ "cc", "cfg-if", "getrandom 0.2.15", "libc", - "spin", "untrusted", "windows-sys 0.52.0", ] @@ -5087,9 +5091,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.22" +version = "0.23.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fb9263ab4eb695e42321db096e3b8fbd715a59b154d5c88d82db2175b681ba7" +checksum = "47796c98c480fce5406ef69d1c76378375492c3b0a0de587be0c1d9feb12f395" dependencies = [ "once_cell", "ring", @@ -5505,9 +5509,9 @@ dependencies = [ [[package]] name = "smallvec" -version = "1.13.2" +version = "1.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c5e1a9a646d36c3599cd173a41282daf47c44583ad367b8e6837255952e5c67" +checksum = "7fcf8323ef1faaee30a44a340193b1ac6814fd9b7b4e88e9d4519a3e4abe1cfd" [[package]] name = "snafu" @@ -5564,17 +5568,11 @@ dependencies = [ "windows-sys 0.52.0", ] -[[package]] -name = "spin" -version = "0.9.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6980e8d7511241f8acf4aebddbb1ff938df5eebe98691418c4468d0b72a96a67" - [[package]] name = "sqllogictest" -version = "0.27.1" +version = "0.27.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07a06aea5e52b0a63b9d8328b46ea2740cdab4cac13def8ef4f2e5288610f9ed" +checksum = "6f1c93848602f92e5925690d4805ccbc1ccdb61bee7d4ae79ad6862b542a539c" dependencies = [ "async-trait", "educe", @@ -5859,9 +5857,9 @@ checksum = "61c41af27dd6d1e27b1b16b489db798443478cef1f06a660c96db617ba5de3b1" [[package]] name = "tempfile" -version = "3.17.0" +version = "3.17.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a40f762a77d2afa88c2d919489e390a12bdd261ed568e60cfa7e48d4e20f0d33" +checksum = "22e5a0acb1f3f55f65cc4a866c361b2fb2a0ff6366785ae6fbb5f85df07ba230" dependencies = [ "cfg-if", "fastrand", @@ -6132,7 +6130,7 @@ version = "0.26.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5f6d0975eaace0cf0fcadee4e4aaa5da15b5c079146f2cffb67c113be122bf37" dependencies = [ - "rustls 0.23.22", + "rustls 0.23.23", "tokio", ] @@ -6203,7 +6201,7 @@ dependencies = [ "axum", "base64 0.22.1", "bytes", - "h2 0.4.7", + "h2 0.4.8", "http 1.2.0", "http-body 1.0.1", "http-body-util", @@ -6359,9 +6357,9 @@ dependencies = [ [[package]] name = "typenum" -version = "1.17.0" +version = "1.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" +checksum = "1dccffe3ce07af9386bfd29e80c0ab1a8205a2fc34e4bcd40364df902cfa8f3f" [[package]] name = "typify" @@ -6511,9 +6509,9 @@ checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" -version = "1.13.1" +version = "1.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ced87ca4be083373936a67f8de945faa23b6b42384bd5b64434850802c6dccd0" +checksum = "8c1f41ffb7cf259f1ecc2876861a17e7142e63ead296f671f81f6ae85903e0d6" dependencies = [ "getrandom 0.3.1", "js-sys", @@ -7093,11 +7091,11 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.8.17" +version = "0.8.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aa91407dacce3a68c56de03abe2760159582b846c6a4acd2f456618087f12713" +checksum = "79386d31a42a4996e3336b0919ddb90f81112af416270cff95b5f5af22b839c2" dependencies = [ - "zerocopy-derive 0.8.17", + "zerocopy-derive 0.8.18", ] [[package]] @@ -7113,9 +7111,9 @@ dependencies = [ [[package]] name = "zerocopy-derive" -version = "0.8.17" +version = "0.8.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06718a168365cad3d5ff0bb133aad346959a2074bd4a85c121255a11304a8626" +checksum = "76331675d372f91bf8d17e13afbd5fe639200b73d01f0fc748bb059f9cca2db7" dependencies = [ "proc-macro2", "quote", diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index 197a12c803c6..cef32279371e 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -45,7 +45,6 @@ datafusion = { workspace = true, features = [ "unicode_expressions", "compression", ] } -datafusion-datasource = { workspace = true } dirs = "6.0.0" env_logger = { workspace = true } futures = { workspace = true } diff --git a/datafusion-cli/src/functions.rs b/datafusion-cli/src/functions.rs index b6f7fcdc51a5..13d2d5fd3547 100644 --- a/datafusion-cli/src/functions.rs +++ b/datafusion-cli/src/functions.rs @@ -28,12 +28,12 @@ use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; use datafusion::catalog::{Session, TableFunctionImpl}; use datafusion::common::{plan_err, Column}; +use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::TableProvider; use datafusion::error::Result; use datafusion::logical_expr::Expr; use datafusion::physical_plan::ExecutionPlan; use datafusion::scalar::ScalarValue; -use datafusion_datasource::memory::MemorySourceConfig; use async_trait::async_trait; use parquet::basic::ConvertedType; diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 90af77900805..ec6e0ab71d50 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -62,7 +62,6 @@ bytes = { workspace = true } dashmap = { workspace = true } # note only use main datafusion crate for examples datafusion = { workspace = true, default-features = true, features = ["avro"] } -datafusion-datasource = { workspace = true } datafusion-proto = { workspace = true } env_logger = { workspace = true } futures = { workspace = true } diff --git a/datafusion-examples/examples/parquet_exec_visitor.rs b/datafusion-examples/examples/parquet_exec_visitor.rs index 3d3067ada01b..6c9f1a354430 100644 --- a/datafusion-examples/examples/parquet_exec_visitor.rs +++ b/datafusion-examples/examples/parquet_exec_visitor.rs @@ -20,13 +20,13 @@ use std::sync::Arc; use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::{ListingOptions, PartitionedFile}; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; +use datafusion::datasource::source::DataSourceExec; use datafusion::error::DataFusionError; use datafusion::execution::context::SessionContext; use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::{ execute_stream, visit_execution_plan, ExecutionPlan, ExecutionPlanVisitor, }; -use datafusion_datasource::source::DataSourceExec; use futures::StreamExt; /// Example of collecting metrics after execution by visiting the `ExecutionPlan` diff --git a/datafusion-examples/examples/remote_catalog.rs b/datafusion-examples/examples/remote_catalog.rs index e2c9c54654b7..70c0963545e0 100644 --- a/datafusion-examples/examples/remote_catalog.rs +++ b/datafusion-examples/examples/remote_catalog.rs @@ -36,12 +36,12 @@ 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::datasource::memory::MemorySourceConfig; use datafusion::execution::SendableRecordBatchStream; use datafusion::logical_expr::{Expr, TableType}; use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::{DataFrame, SessionContext}; -use datafusion_datasource::memory::MemorySourceConfig; use futures::TryStreamExt; use std::any::Any; use std::sync::Arc; diff --git a/datafusion-examples/examples/simple_udtf.rs b/datafusion-examples/examples/simple_udtf.rs index b61303f2435f..d2b2d1bf9655 100644 --- a/datafusion-examples/examples/simple_udtf.rs +++ b/datafusion-examples/examples/simple_udtf.rs @@ -23,6 +23,7 @@ use datafusion::arrow::record_batch::RecordBatch; use datafusion::catalog::Session; use datafusion::catalog::TableFunctionImpl; use datafusion::common::{plan_err, ScalarValue}; +use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::TableProvider; use datafusion::error::Result; use datafusion::execution::context::ExecutionProps; @@ -31,7 +32,6 @@ use datafusion::logical_expr::{Expr, TableType}; use datafusion::optimizer::simplify_expressions::ExprSimplifier; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::*; -use datafusion_datasource::memory::MemorySourceConfig; use std::fs::File; use std::io::Seek; use std::path::Path; diff --git a/datafusion/core/src/datasource/avro_to_arrow/reader.rs b/datafusion/core/src/datasource/avro_to_arrow/reader.rs index dbc24da46366..c9cb90e7bd97 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/reader.rs +++ b/datafusion/core/src/datasource/avro_to_arrow/reader.rs @@ -58,7 +58,7 @@ impl ReaderBuilder { /// ``` /// use std::fs::File; /// - /// use datafusion::datasource::avro_to_arrow::{Reader, ReaderBuilder}; + /// use datafusion_datasource::avro_to_arrow::{Reader, ReaderBuilder}; /// /// fn example() -> Reader<'static, File> { /// let file = File::open("test/data/basic.avro").unwrap(); diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 819da155a1a2..2e0a9d09c814 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -307,7 +307,7 @@ impl ListingOptions { /// ``` /// # use std::sync::Arc; /// # use datafusion::prelude::SessionContext; - /// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; + /// # use datafusion_datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; /// /// let listing_options = ListingOptions::new(Arc::new( /// ParquetFormat::default() @@ -329,7 +329,7 @@ impl ListingOptions { /// ``` /// # use std::sync::Arc; /// # use datafusion::prelude::SessionContext; - /// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; + /// # use datafusion_datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; /// let extension = Some(".parquet"); /// let listing_options = ListingOptions::new(Arc::new( /// ParquetFormat::default() @@ -392,7 +392,7 @@ impl ListingOptions { /// # use std::sync::Arc; /// # use arrow::datatypes::DataType; /// # use datafusion::prelude::col; - /// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; + /// # use datafusion_datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; /// /// // listing options for files with paths such as `/mnt/data/col_a=x/col_b=y/data.parquet` /// // `col_a` and `col_b` will be included in the data read from those files @@ -420,7 +420,7 @@ impl ListingOptions { /// /// ``` /// # use std::sync::Arc; - /// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; + /// # use datafusion_datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; /// /// let listing_options = ListingOptions::new(Arc::new( /// ParquetFormat::default() @@ -438,7 +438,7 @@ impl ListingOptions { /// /// ``` /// # use std::sync::Arc; - /// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; + /// # use datafusion_datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; /// /// let listing_options = ListingOptions::new(Arc::new( /// ParquetFormat::default() @@ -457,7 +457,7 @@ impl ListingOptions { /// ``` /// # use std::sync::Arc; /// # use datafusion::prelude::col; - /// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; + /// # use datafusion_datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; /// /// // Tell datafusion that the files are sorted by column "a" /// let file_sort_order = vec![vec![ @@ -662,7 +662,7 @@ impl ListingOptions { /// # use datafusion::prelude::SessionContext; /// # use datafusion::error::Result; /// # use std::sync::Arc; -/// # use datafusion::datasource::{ +/// # use datafusion_datasource::{ /// # listing::{ /// # ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, /// # }, diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index 09e6f7616ee7..b8bec410070c 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -38,8 +38,8 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_catalog::Session; use datafusion_common::{not_impl_err, plan_err, Constraints, DFSchema, SchemaExt}; -use datafusion_datasource::memory::MemorySourceConfig; -use datafusion_datasource::source::DataSourceExec; +pub use datafusion_datasource::memory::MemorySourceConfig; +pub use datafusion_datasource::source::DataSourceExec; use datafusion_execution::TaskContext; use datafusion_expr::dml::InsertOp; use datafusion_expr::SortExpr; diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 12dd9d7cab38..96687913fb42 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -36,6 +36,8 @@ mod statistics; pub mod stream; pub mod view; +pub use datafusion_datasource::source; + // backwards compatibility pub use self::default_table_source::{ provider_as_source, source_as_provider, DefaultTableSource, diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index e4612de818be..68495fa848c4 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -403,11 +403,11 @@ impl ExecutionPlan for CsvExec { /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::Schema; -/// # use datafusion::datasource::{ +/// # use datafusion_datasource::{ /// # physical_plan::FileScanConfig, /// # listing::PartitionedFile, /// # }; -/// # use datafusion::datasource::physical_plan::CsvSource; +/// # use datafusion_datasource::physical_plan::CsvSource; /// # use datafusion_execution::object_store::ObjectStoreUrl; /// # use datafusion_datasource::source::DataSourceExec; /// diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 4996b6d97b58..28e18a3ac8b6 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -77,10 +77,10 @@ pub fn wrap_partition_value_in_dict(val: ScalarValue) -> ScalarValue { /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::{Field, Fields, DataType, Schema}; -/// # use datafusion::datasource::listing::PartitionedFile; -/// # use datafusion::datasource::physical_plan::FileScanConfig; +/// # use datafusion_datasource::listing::PartitionedFile; +/// # use datafusion_datasource::physical_plan::FileScanConfig; /// # use datafusion_execution::object_store::ObjectStoreUrl; -/// # use datafusion::datasource::physical_plan::ArrowSource; +/// # use datafusion_datasource::physical_plan::ArrowSource; /// # use datafusion_physical_plan::ExecutionPlan; /// # let file_schema = Arc::new(Schema::new(vec![ /// # Field::new("c1", DataType::Int32, false), diff --git a/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs b/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs index d30549708bbd..05425bad94de 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs @@ -35,7 +35,7 @@ use parquet::file::metadata::RowGroupMetaData; /// /// ```rust /// # use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; -/// # use datafusion::datasource::physical_plan::parquet::ParquetAccessPlan; +/// # use datafusion_datasource::physical_plan::parquet::ParquetAccessPlan; /// // Default to scan all row groups /// let mut access_plan = ParquetAccessPlan::new_all(4); /// access_plan.skip(0); // skip row group diff --git a/datafusion/core/src/datasource/physical_plan/parquet/source.rs b/datafusion/core/src/datasource/physical_plan/parquet/source.rs index 00f31a60d527..40494f7263c6 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/source.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/source.rs @@ -76,9 +76,9 @@ use object_store::ObjectStore; /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::Schema; -/// # use datafusion::datasource::physical_plan::FileScanConfig; -/// # use datafusion::datasource::physical_plan::parquet::source::ParquetSource; -/// # use datafusion::datasource::listing::PartitionedFile; +/// # use datafusion_datasource::physical_plan::FileScanConfig; +/// # use datafusion_datasource::physical_plan::parquet::source::ParquetSource; +/// # use datafusion_datasource::listing::PartitionedFile; /// # use datafusion_execution::object_store::ObjectStoreUrl; /// # use datafusion_physical_expr::expressions::lit; /// # use datafusion_datasource::source::DataSourceExec; @@ -158,8 +158,8 @@ use object_store::ObjectStore; /// ```no_run /// # use std::sync::Arc; /// # use arrow::datatypes::Schema; -/// # use datafusion::datasource::physical_plan::FileScanConfig; -/// # use datafusion::datasource::listing::PartitionedFile; +/// # use datafusion_datasource::physical_plan::FileScanConfig; +/// # use datafusion_datasource::listing::PartitionedFile; /// # use datafusion_datasource::source::DataSourceExec; /// /// # fn parquet_exec() -> DataSourceExec { unimplemented!() } @@ -197,10 +197,10 @@ use object_store::ObjectStore; /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::{Schema, SchemaRef}; -/// # use datafusion::datasource::listing::PartitionedFile; -/// # use datafusion::datasource::physical_plan::parquet::ParquetAccessPlan; -/// # use datafusion::datasource::physical_plan::FileScanConfig; -/// # use datafusion::datasource::physical_plan::parquet::source::ParquetSource; +/// # use datafusion_datasource::listing::PartitionedFile; +/// # use datafusion_datasource::physical_plan::parquet::ParquetAccessPlan; +/// # use datafusion_datasource::physical_plan::FileScanConfig; +/// # use datafusion_datasource::physical_plan::parquet::source::ParquetSource; /// # use datafusion_execution::object_store::ObjectStoreUrl; /// # use datafusion_datasource::source::DataSourceExec; /// diff --git a/datafusion/core/src/datasource/schema_adapter.rs b/datafusion/core/src/datasource/schema_adapter.rs index 41e375cf81f8..f52809246487 100644 --- a/datafusion/core/src/datasource/schema_adapter.rs +++ b/datafusion/core/src/datasource/schema_adapter.rs @@ -159,7 +159,7 @@ pub trait SchemaMapper: Debug + Send + Sync { /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::{DataType, Field, Schema}; -/// # use datafusion::datasource::schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapterFactory}; +/// # use datafusion_datasource::schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapterFactory}; /// # use datafusion_common::record_batch; /// // Table has fields "a", "b" and "c" /// let table_schema = Schema::new(vec![ diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 77a2bd440de8..1025a49ea1e3 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -30,6 +30,8 @@ use arrow::datatypes::{ }; use arrow::util::pretty::pretty_format_batches; use datafusion::common::Result; +use datafusion::datasource::memory::MemorySourceConfig; +use datafusion::datasource::source::DataSourceExec; use datafusion::datasource::MemTable; use datafusion::physical_expr::aggregate::AggregateExprBuilder; use datafusion::physical_plan::aggregates::{ @@ -39,8 +41,6 @@ use datafusion::physical_plan::{collect, displayable, ExecutionPlan}; use datafusion::prelude::{DataFrame, SessionConfig, SessionContext}; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor}; use datafusion_common::HashMap; -use datafusion_datasource::memory::MemorySourceConfig; -use datafusion_datasource::source::DataSourceExec; use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; diff --git a/datafusion/core/tests/fuzz_cases/join_fuzz.rs b/datafusion/core/tests/fuzz_cases/join_fuzz.rs index 31412d8cbd8f..da93dd5edf29 100644 --- a/datafusion/core/tests/fuzz_cases/join_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/join_fuzz.rs @@ -26,6 +26,8 @@ use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; use datafusion::common::JoinSide; +use datafusion::datasource::memory::MemorySourceConfig; +use datafusion::datasource::source::DataSourceExec; use datafusion::logical_expr::{JoinType, Operator}; use datafusion::physical_expr::expressions::BinaryExpr; use datafusion::physical_plan::collect; @@ -36,8 +38,6 @@ use datafusion::physical_plan::joins::{ }; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::ScalarValue; -use datafusion_datasource::memory::MemorySourceConfig; -use datafusion_datasource::source::DataSourceExec; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::PhysicalExprRef; diff --git a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs index 288f081981fe..92f375525066 100644 --- a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs @@ -24,13 +24,13 @@ use arrow::{ compute::SortOptions, record_batch::RecordBatch, }; +use datafusion::datasource::memory::MemorySourceConfig; use datafusion::physical_plan::{ collect, expressions::{col, PhysicalSortExpr}, sorts::sort_preserving_merge::SortPreservingMergeExec, }; use datafusion::prelude::{SessionConfig, SessionContext}; -use datafusion_datasource::memory::MemorySourceConfig; use datafusion_physical_expr_common::sort_expr::LexOrdering; use test_utils::{batches_to_vec, partitions_to_sorted_vec, stagger_batch_with_seed}; diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index e797035943ab..0b0f0aa2f105 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -24,13 +24,13 @@ use arrow::{ compute::SortOptions, record_batch::RecordBatch, }; +use datafusion::datasource::memory::MemorySourceConfig; use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::physical_plan::expressions::PhysicalSortExpr; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::{collect, ExecutionPlan}; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::cast::as_int32_array; -use datafusion_datasource::memory::MemorySourceConfig; use datafusion_execution::memory_pool::GreedyMemoryPool; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr_common::sort_expr::LexOrdering; diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 3d52a2b73b73..06b93d41af36 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -44,8 +44,8 @@ mod sp_repartition_fuzz_tests { }; use test_utils::add_empty_batches; - use datafusion_datasource::memory::MemorySourceConfig; - use datafusion_datasource::source::DataSourceExec; + use datafusion::datasource::memory::MemorySourceConfig; + use datafusion::datasource::source::DataSourceExec; use datafusion_physical_expr_common::sort_expr::LexOrdering; use itertools::izip; use rand::{rngs::StdRng, seq::SliceRandom, Rng, SeedableRng}; diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index c4febbe2b005..a7f9e38c9ae3 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -22,6 +22,8 @@ use arrow::compute::{concat_batches, SortOptions}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; +use datafusion::datasource::memory::MemorySourceConfig; +use datafusion::datasource::source::DataSourceExec; use datafusion::functions_window::row_number::row_number_udwf; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::windows::{ @@ -33,8 +35,6 @@ use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::HashMap; use datafusion_common::{Result, ScalarValue}; use datafusion_common_runtime::SpawnedTask; -use datafusion_datasource::memory::MemorySourceConfig; -use datafusion_datasource::source::DataSourceExec; use datafusion_expr::type_coercion::functions::data_types_with_aggregate_udf; use datafusion_expr::{ WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index 3b6c9f2af154..a1985a1aa447 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -27,6 +27,8 @@ use arrow::array::{ArrayRef, DictionaryArray, RecordBatch}; use arrow::compute::SortOptions; use arrow::datatypes::{Int32Type, SchemaRef}; use datafusion::assert_batches_eq; +use datafusion::datasource::memory::MemorySourceConfig; +use datafusion::datasource::source::DataSourceExec; use datafusion::datasource::{MemTable, TableProvider}; use datafusion::execution::disk_manager::DiskManagerConfig; use datafusion::execution::runtime_env::RuntimeEnvBuilder; @@ -38,8 +40,6 @@ use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_catalog::streaming::StreamingTable; use datafusion_catalog::Session; use datafusion_common::{assert_contains, Result}; -use datafusion_datasource::memory::MemorySourceConfig; -use datafusion_datasource::source::DataSourceExec; use datafusion_execution::memory_pool::{ GreedyMemoryPool, MemoryPool, TrackConsumersPool, }; diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index 95298d37b7d8..ad75cf2607c4 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -22,12 +22,12 @@ use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::{ ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, }; +use datafusion::datasource::source::DataSourceExec; use datafusion::datasource::TableProvider; use datafusion::execution::context::SessionState; use datafusion::execution::session_state::SessionStateBuilder; use datafusion::prelude::SessionContext; use datafusion_common::stats::Precision; -use datafusion_datasource::source::DataSourceExec; use datafusion_execution::cache::cache_manager::CacheManagerConfig; use datafusion_execution::cache::cache_unit::{ DefaultFileStatisticsCache, DefaultListFilesCache, diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 242698215275..fe96a2eb5e71 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -25,12 +25,12 @@ use datafusion::datasource::file_format::FileFormat; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; +use datafusion::datasource::source::DataSourceExec; use datafusion::execution::context::SessionState; use datafusion::physical_plan::metrics::MetricValue; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; use datafusion_common::{ScalarValue, ToDFSchema}; -use datafusion_datasource::source::DataSourceExec; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::{col, lit, Expr}; use datafusion_physical_expr::create_physical_expr; diff --git a/datafusion/core/tests/parquet/utils.rs b/datafusion/core/tests/parquet/utils.rs index 1d573286c435..8cb50b22cf63 100644 --- a/datafusion/core/tests/parquet/utils.rs +++ b/datafusion/core/tests/parquet/utils.rs @@ -18,7 +18,7 @@ //! Utilities for parquet tests use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; -use datafusion_datasource::source::DataSourceExec; +use datafusion::datasource::source::DataSourceExec; use datafusion_physical_plan::metrics::MetricsSet; use datafusion_physical_plan::{accept, ExecutionPlan, ExecutionPlanVisitor}; diff --git a/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs b/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs index 78f0dd13aebe..a79d743cb253 100644 --- a/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/tests/physical_optimizer/aggregate_statistics.rs @@ -22,11 +22,11 @@ use crate::physical_optimizer::test_utils::TestAggregate; use arrow::array::Int32Array; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; +use datafusion::datasource::memory::MemorySourceConfig; +use datafusion::datasource::source::DataSourceExec; use datafusion_common::cast::as_int64_array; use datafusion_common::config::ConfigOptions; use datafusion_common::Result; -use datafusion_datasource::memory::MemorySourceConfig; -use datafusion_datasource::source::DataSourceExec; use datafusion_execution::TaskContext; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{self, cast}; diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 725c5310140d..66d1380e09c3 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -31,10 +31,10 @@ use datafusion::datasource::file_format::file_compression_type::FileCompressionT use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{CsvSource, FileScanConfig, ParquetSource}; +use datafusion::datasource::source::DataSourceExec; use datafusion_common::error::Result; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::ScalarValue; -use datafusion_datasource::source::DataSourceExec; use datafusion_expr::{JoinType, Operator}; use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; use datafusion_physical_expr::PhysicalExpr; diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index 8b70d63c21b6..dfba57a584ea 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -21,12 +21,12 @@ use std::sync::Arc; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::physical_plan::{CsvSource, FileScanConfig}; +use datafusion::datasource::source::DataSourceExec; use datafusion_common::config::ConfigOptions; use datafusion_common::Result; use datafusion_common::{JoinSide, JoinType, ScalarValue}; -use datafusion_datasource::memory::MemorySourceConfig; -use datafusion_datasource::source::DataSourceExec; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{Operator, ScalarUDF, ScalarUDFImpl, Signature, Volatility}; 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 30cfda2d319a..58eb866c590c 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 @@ -32,13 +32,13 @@ use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::collect; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; -use datafusion_datasource::memory::MemorySourceConfig; +use datafusion::datasource::memory::MemorySourceConfig; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::{ displayable, get_plan_string, ExecutionPlan, Partitioning, }; -use datafusion_datasource::source::DataSourceExec; +use datafusion::datasource::source::DataSourceExec; use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_common::Result; use datafusion_expr::{JoinType, Operator}; diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 4fe1beef76dd..e4d72c112c38 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -26,13 +26,13 @@ use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; +use datafusion::datasource::source::DataSourceExec; 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_datasource::memory::MemorySourceConfig; -use datafusion_datasource::source::DataSourceExec; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; diff --git a/datafusion/core/tests/sql/path_partition.rs b/datafusion/core/tests/sql/path_partition.rs index 35447c20cd34..1a19bfe9e86f 100644 --- a/datafusion/core/tests/sql/path_partition.rs +++ b/datafusion/core/tests/sql/path_partition.rs @@ -26,6 +26,7 @@ use std::sync::Arc; use arrow::datatypes::DataType; use datafusion::datasource::listing::ListingTableUrl; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; +use datafusion::datasource::source::DataSourceExec; use datafusion::{ assert_batches_sorted_eq, datasource::{ @@ -40,7 +41,6 @@ use datafusion::{ use datafusion_catalog::TableProvider; use datafusion_common::stats::Precision; use datafusion_common::ScalarValue; -use datafusion_datasource::source::DataSourceExec; use datafusion_execution::config::SessionConfig; use datafusion_expr::{col, lit, Expr, Operator}; use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; diff --git a/datafusion/core/tests/user_defined/user_defined_table_functions.rs b/datafusion/core/tests/user_defined/user_defined_table_functions.rs index c0fd7ca25676..618f0590ab3d 100644 --- a/datafusion/core/tests/user_defined/user_defined_table_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_table_functions.rs @@ -26,6 +26,7 @@ use arrow::csv::ReaderBuilder; use datafusion::arrow::datatypes::SchemaRef; use datafusion::arrow::record_batch::RecordBatch; +use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::TableProvider; use datafusion::error::Result; use datafusion::execution::TaskContext; @@ -34,7 +35,6 @@ use datafusion::prelude::SessionContext; use datafusion_catalog::Session; use datafusion_catalog::TableFunctionImpl; use datafusion_common::{assert_batches_eq, DFSchema, ScalarValue}; -use datafusion_datasource::memory::MemorySourceConfig; use datafusion_expr::{EmptyRelation, Expr, LogicalPlan, Projection, TableType}; use async_trait::async_trait; diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 47ecd5c29f13..dcf99ad2e65b 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -767,8 +767,8 @@ mod memory_exec_tests { mod tests { use super::*; use arrow::array::{ArrayRef, Int32Array}; - use std::collections::HashMap; use datafusion_physical_plan::expressions::lit; + use std::collections::HashMap; use arrow::datatypes::{DataType, Field}; use datafusion_common::assert_batches_eq; diff --git a/datafusion/functions-table/Cargo.toml b/datafusion/functions-table/Cargo.toml index 3020d7d131c5..f722d698f3d3 100644 --- a/datafusion/functions-table/Cargo.toml +++ b/datafusion/functions-table/Cargo.toml @@ -42,7 +42,6 @@ arrow = { workspace = true } async-trait = { workspace = true } datafusion-catalog = { workspace = true } datafusion-common = { workspace = true } -datafusion-datasource = { workspace = true } datafusion-expr = { workspace = true } datafusion-physical-plan = { workspace = true } parking_lot = { workspace = true } diff --git a/datafusion/proto/Cargo.toml b/datafusion/proto/Cargo.toml index a819407ac126..fb5d414dcec4 100644 --- a/datafusion/proto/Cargo.toml +++ b/datafusion/proto/Cargo.toml @@ -45,7 +45,6 @@ arrow = { workspace = true } chrono = { workspace = true } datafusion = { workspace = true, default-features = true } datafusion-common = { workspace = true, default-features = true } -datafusion-datasource = { workspace = true } datafusion-expr = { workspace = true } datafusion-proto-common = { workspace = true } object_store = { workspace = true } diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index f5969814b284..2c596255587b 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -32,6 +32,7 @@ use datafusion::datasource::file_format::parquet::ParquetSink; #[cfg(feature = "parquet")] use datafusion::datasource::physical_plan::ParquetSource; use datafusion::datasource::physical_plan::{AvroSource, CsvSource, FileScanConfig}; +use datafusion::datasource::source::DataSourceExec; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::physical_expr::aggregate::AggregateFunctionExpr; @@ -65,7 +66,6 @@ use datafusion::physical_plan::{ }; use datafusion_common::config::TableParquetOptions; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; -use datafusion_datasource::source::DataSourceExec; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; use crate::common::{byte_to_string, str_to_byte}; diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 6271f2992504..f13d2b77a787 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -35,7 +35,6 @@ async-recursion = "1.0" async-trait = { workspace = true } chrono = { workspace = true } datafusion = { workspace = true } -datafusion-datasource = { workspace = true } itertools = { workspace = true } object_store = { workspace = true } pbjson-types = { workspace = true } diff --git a/datafusion/substrait/src/physical_plan/producer.rs b/datafusion/substrait/src/physical_plan/producer.rs index 5bf1adf4fc16..18e9a0ba22f6 100644 --- a/datafusion/substrait/src/physical_plan/producer.rs +++ b/datafusion/substrait/src/physical_plan/producer.rs @@ -25,7 +25,7 @@ use crate::variation_const::{ use datafusion::arrow::datatypes::DataType; use datafusion::error::{DataFusionError, Result}; use datafusion::physical_plan::{displayable, ExecutionPlan}; -use datafusion_datasource::source::DataSourceExec; +use datafusion::datasource::source::DataSourceExec; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use substrait::proto::expression::mask_expression::{StructItem, StructSelect}; From c73f00ddb3f23783250cecbe862f40af15c92926 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Wed, 19 Feb 2025 02:04:29 +0530 Subject: [PATCH 14/18] fix:cargo fmt --- datafusion/substrait/src/physical_plan/producer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/substrait/src/physical_plan/producer.rs b/datafusion/substrait/src/physical_plan/producer.rs index 18e9a0ba22f6..e8c15731228c 100644 --- a/datafusion/substrait/src/physical_plan/producer.rs +++ b/datafusion/substrait/src/physical_plan/producer.rs @@ -23,9 +23,9 @@ use crate::variation_const::{ }; use datafusion::arrow::datatypes::DataType; +use datafusion::datasource::source::DataSourceExec; use datafusion::error::{DataFusionError, Result}; use datafusion::physical_plan::{displayable, ExecutionPlan}; -use datafusion::datasource::source::DataSourceExec; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetSource}; use substrait::proto::expression::mask_expression::{StructItem, StructSelect}; From 7f0189dc3840d72634773951265efa64eba37c54 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Wed, 19 Feb 2025 02:44:40 +0530 Subject: [PATCH 15/18] fix: doctest --- .../core/src/datasource/listing/table.rs | 12 +++++----- .../core/src/datasource/physical_plan/csv.rs | 6 ++--- .../physical_plan/file_scan_config.rs | 6 ++--- .../physical_plan/parquet/access_plan.rs | 2 +- .../physical_plan/parquet/source.rs | 24 +++++++++---------- .../core/src/datasource/schema_adapter.rs | 2 +- 6 files changed, 26 insertions(+), 26 deletions(-) diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 2e0a9d09c814..7a360d496c16 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -307,7 +307,7 @@ impl ListingOptions { /// ``` /// # use std::sync::Arc; /// # use datafusion::prelude::SessionContext; - /// # use datafusion_datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; + /// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; /// /// let listing_options = ListingOptions::new(Arc::new( /// ParquetFormat::default() @@ -329,7 +329,7 @@ impl ListingOptions { /// ``` /// # use std::sync::Arc; /// # use datafusion::prelude::SessionContext; - /// # use datafusion_datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; + /// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; /// let extension = Some(".parquet"); /// let listing_options = ListingOptions::new(Arc::new( /// ParquetFormat::default() @@ -392,7 +392,7 @@ impl ListingOptions { /// # use std::sync::Arc; /// # use arrow::datatypes::DataType; /// # use datafusion::prelude::col; - /// # use datafusion_datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; + /// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; /// /// // listing options for files with paths such as `/mnt/data/col_a=x/col_b=y/data.parquet` /// // `col_a` and `col_b` will be included in the data read from those files @@ -438,7 +438,7 @@ impl ListingOptions { /// /// ``` /// # use std::sync::Arc; - /// # use datafusion_datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; + /// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; /// /// let listing_options = ListingOptions::new(Arc::new( /// ParquetFormat::default() @@ -457,7 +457,7 @@ impl ListingOptions { /// ``` /// # use std::sync::Arc; /// # use datafusion::prelude::col; - /// # use datafusion_datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; + /// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; /// /// // Tell datafusion that the files are sorted by column "a" /// let file_sort_order = vec![vec![ @@ -662,7 +662,7 @@ impl ListingOptions { /// # use datafusion::prelude::SessionContext; /// # use datafusion::error::Result; /// # use std::sync::Arc; -/// # use datafusion_datasource::{ +/// # use datafusion::datasource::{ /// # listing::{ /// # ListingOptions, ListingTable, ListingTableConfig, ListingTableUrl, /// # }, diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 68495fa848c4..412c90726af0 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -403,13 +403,13 @@ impl ExecutionPlan for CsvExec { /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::Schema; -/// # use datafusion_datasource::{ +/// # use datafusion::datasource::{ /// # physical_plan::FileScanConfig, /// # listing::PartitionedFile, /// # }; -/// # use datafusion_datasource::physical_plan::CsvSource; +/// # use datafusion::datasource::physical_plan::CsvSource; /// # use datafusion_execution::object_store::ObjectStoreUrl; -/// # use datafusion_datasource::source::DataSourceExec; +/// # use datafusion::datasource::source::DataSourceExec; /// /// # let object_store_url = ObjectStoreUrl::local_filesystem(); /// # let file_schema = Arc::new(Schema::empty()); diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 28e18a3ac8b6..4996b6d97b58 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -77,10 +77,10 @@ pub fn wrap_partition_value_in_dict(val: ScalarValue) -> ScalarValue { /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::{Field, Fields, DataType, Schema}; -/// # use datafusion_datasource::listing::PartitionedFile; -/// # use datafusion_datasource::physical_plan::FileScanConfig; +/// # use datafusion::datasource::listing::PartitionedFile; +/// # use datafusion::datasource::physical_plan::FileScanConfig; /// # use datafusion_execution::object_store::ObjectStoreUrl; -/// # use datafusion_datasource::physical_plan::ArrowSource; +/// # use datafusion::datasource::physical_plan::ArrowSource; /// # use datafusion_physical_plan::ExecutionPlan; /// # let file_schema = Arc::new(Schema::new(vec![ /// # Field::new("c1", DataType::Int32, false), diff --git a/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs b/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs index 05425bad94de..d30549708bbd 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/access_plan.rs @@ -35,7 +35,7 @@ use parquet::file::metadata::RowGroupMetaData; /// /// ```rust /// # use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; -/// # use datafusion_datasource::physical_plan::parquet::ParquetAccessPlan; +/// # use datafusion::datasource::physical_plan::parquet::ParquetAccessPlan; /// // Default to scan all row groups /// let mut access_plan = ParquetAccessPlan::new_all(4); /// access_plan.skip(0); // skip row group diff --git a/datafusion/core/src/datasource/physical_plan/parquet/source.rs b/datafusion/core/src/datasource/physical_plan/parquet/source.rs index 40494f7263c6..810a16de41af 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/source.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/source.rs @@ -76,12 +76,12 @@ use object_store::ObjectStore; /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::Schema; -/// # use datafusion_datasource::physical_plan::FileScanConfig; -/// # use datafusion_datasource::physical_plan::parquet::source::ParquetSource; -/// # use datafusion_datasource::listing::PartitionedFile; +/// # use datafusion::datasource::physical_plan::FileScanConfig; +/// # use datafusion::datasource::physical_plan::parquet::source::ParquetSource; +/// # use datafusion::datasource::listing::PartitionedFile; /// # use datafusion_execution::object_store::ObjectStoreUrl; /// # use datafusion_physical_expr::expressions::lit; -/// # use datafusion_datasource::source::DataSourceExec; +/// # use datafusion::datasource::source::DataSourceExec; /// # use datafusion_common::config::TableParquetOptions; /// /// # let file_schema = Arc::new(Schema::empty()); @@ -158,9 +158,9 @@ use object_store::ObjectStore; /// ```no_run /// # use std::sync::Arc; /// # use arrow::datatypes::Schema; -/// # use datafusion_datasource::physical_plan::FileScanConfig; -/// # use datafusion_datasource::listing::PartitionedFile; -/// # use datafusion_datasource::source::DataSourceExec; +/// # use datafusion::datasource::physical_plan::FileScanConfig; +/// # use datafusion::datasource::listing::PartitionedFile; +/// # use datafusion::datasource::source::DataSourceExec; /// /// # fn parquet_exec() -> DataSourceExec { unimplemented!() } /// // Split a single DataSourceExec into multiple DataSourceExecs, one for each file @@ -197,12 +197,12 @@ use object_store::ObjectStore; /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::{Schema, SchemaRef}; -/// # use datafusion_datasource::listing::PartitionedFile; -/// # use datafusion_datasource::physical_plan::parquet::ParquetAccessPlan; -/// # use datafusion_datasource::physical_plan::FileScanConfig; -/// # use datafusion_datasource::physical_plan::parquet::source::ParquetSource; +/// # use datafusion::datasource::listing::PartitionedFile; +/// # use datafusion::datasource::physical_plan::parquet::ParquetAccessPlan; +/// # use datafusion::datasource::physical_plan::FileScanConfig; +/// # use datafusion::datasource::physical_plan::parquet::source::ParquetSource; /// # use datafusion_execution::object_store::ObjectStoreUrl; -/// # use datafusion_datasource::source::DataSourceExec; +/// # use datafusion::datasource::source::DataSourceExec; /// /// # fn schema() -> SchemaRef { /// # Arc::new(Schema::empty()) diff --git a/datafusion/core/src/datasource/schema_adapter.rs b/datafusion/core/src/datasource/schema_adapter.rs index f52809246487..41e375cf81f8 100644 --- a/datafusion/core/src/datasource/schema_adapter.rs +++ b/datafusion/core/src/datasource/schema_adapter.rs @@ -159,7 +159,7 @@ pub trait SchemaMapper: Debug + Send + Sync { /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::{DataType, Field, Schema}; -/// # use datafusion_datasource::schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapterFactory}; +/// # use datafusion::datasource::schema_adapter::{DefaultSchemaAdapterFactory, SchemaAdapterFactory}; /// # use datafusion_common::record_batch; /// // Table has fields "a", "b" and "c" /// let table_schema = Schema::new(vec![ From 673cb4a7789a1c8e58c756e50faebd6904907732 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Wed, 19 Feb 2025 14:19:25 +0530 Subject: [PATCH 16/18] fix: leftout doctest --- datafusion/core/src/datasource/avro_to_arrow/reader.rs | 2 +- datafusion/core/src/datasource/listing/table.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/datasource/avro_to_arrow/reader.rs b/datafusion/core/src/datasource/avro_to_arrow/reader.rs index c9cb90e7bd97..dbc24da46366 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/reader.rs +++ b/datafusion/core/src/datasource/avro_to_arrow/reader.rs @@ -58,7 +58,7 @@ impl ReaderBuilder { /// ``` /// use std::fs::File; /// - /// use datafusion_datasource::avro_to_arrow::{Reader, ReaderBuilder}; + /// use datafusion::datasource::avro_to_arrow::{Reader, ReaderBuilder}; /// /// fn example() -> Reader<'static, File> { /// let file = File::open("test/data/basic.avro").unwrap(); diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 7a360d496c16..819da155a1a2 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -420,7 +420,7 @@ impl ListingOptions { /// /// ``` /// # use std::sync::Arc; - /// # use datafusion_datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; + /// # use datafusion::datasource::{listing::ListingOptions, file_format::parquet::ParquetFormat}; /// /// let listing_options = ListingOptions::new(Arc::new( /// ParquetFormat::default() From c0d2e60cb0a46ed8f26638db45ecc6b8a0f5867d Mon Sep 17 00:00:00 2001 From: logan-keede Date: Wed, 19 Feb 2025 19:58:04 +0530 Subject: [PATCH 17/18] fix: circular dependency --- Cargo.lock | 1 - datafusion/core/Cargo.toml | 4 ++++ datafusion/{physical-plan => core}/benches/spm.rs | 0 datafusion/physical-plan/Cargo.toml | 5 ----- 4 files changed, 4 insertions(+), 6 deletions(-) rename datafusion/{physical-plan => core}/benches/spm.rs (100%) diff --git a/Cargo.lock b/Cargo.lock index b05e511f843e..cbb2f381d648 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2258,7 +2258,6 @@ dependencies = [ "criterion", "datafusion-common", "datafusion-common-runtime", - "datafusion-datasource", "datafusion-execution", "datafusion-expr", "datafusion-functions-aggregate", diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 784b2a89aae9..0c3796a95300 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -222,3 +222,7 @@ required-features = ["nested_expressions"] [[bench]] harness = false name = "dataframe" + +[[bench]] +harness = false +name = "spm" diff --git a/datafusion/physical-plan/benches/spm.rs b/datafusion/core/benches/spm.rs similarity index 100% rename from datafusion/physical-plan/benches/spm.rs rename to datafusion/core/benches/spm.rs diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 68f66d4aedd2..4ff46545b8ce 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -64,7 +64,6 @@ tokio = { workspace = true } [dev-dependencies] criterion = { version = "0.5", features = ["async_futures"] } -datafusion-datasource = { workspace = true } datafusion-functions-aggregate = { workspace = true } datafusion-functions-window = { workspace = true } rand = { workspace = true } @@ -76,10 +75,6 @@ tokio = { workspace = true, features = [ "parking_lot", ] } -[[bench]] -harness = false -name = "spm" - [[bench]] harness = false name = "partial_ordering" From 4edec1d8d61b1ae370c88d8d20eb4d7f319c9f55 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Thu, 20 Feb 2025 20:30:34 +0530 Subject: [PATCH 18/18] clean, rename, document, improve --- datafusion/datasource/src/memory.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 29 ++++------ .../physical-plan/src/joins/hash_join.rs | 58 +++++++------------ .../src/joins/nested_loop_join.rs | 8 +-- .../src/joins/sort_merge_join.rs | 14 ++--- .../physical-plan/src/joins/test_utils.rs | 11 ++-- .../physical-plan/src/repartition/mod.rs | 25 +++----- .../physical-plan/src/sorts/partial_sort.rs | 13 ++--- datafusion/physical-plan/src/sorts/sort.rs | 19 ++---- .../src/sorts/sort_preserving_merge.rs | 26 ++++----- datafusion/physical-plan/src/test.rs | 52 +++++++---------- datafusion/physical-plan/src/union.rs | 11 ++-- .../src/windows/bounded_window_agg_exec.rs | 4 +- 13 files changed, 105 insertions(+), 167 deletions(-) diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index dcf99ad2e65b..efb178ad078e 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -708,7 +708,7 @@ impl MemorySourceConfig { } #[cfg(test)] -mod memory_exec_tests { +mod memory_source_tests { use std::sync::Arc; use crate::memory::MemorySourceConfig; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index b46114f8f66e..0947a2ff5539 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1350,7 +1350,7 @@ mod tests { use crate::metrics::MetricValue; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; - use crate::test::MockMemorySourceConfig; + use crate::test::TestMemoryExec; use crate::RecordBatchStream; use arrow::array::{ @@ -2207,7 +2207,7 @@ mod tests { vec![test_last_value_agg_expr(&schema, sort_options)?] }; - let memory_exec = MockMemorySourceConfig::try_new_exec( + let memory_exec = TestMemoryExec::try_new_exec( &[ vec![partition1], vec![partition2], @@ -2442,11 +2442,8 @@ mod tests { }) .collect(); - let input = MockMemorySourceConfig::try_new_exec( - &[input_batches], - Arc::clone(&schema), - None, - )?; + let input = + TestMemoryExec::try_new_exec(&[input_batches], Arc::clone(&schema), None)?; let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Single, @@ -2557,7 +2554,7 @@ mod tests { .build() .map(Arc::new)?]; - let input = MockMemorySourceConfig::try_new_exec( + let input = TestMemoryExec::try_new_exec( &[vec![batch.clone()]], Arc::::clone(&batch.schema()), None, @@ -2626,11 +2623,8 @@ mod tests { .unwrap(), ]; - let input = MockMemorySourceConfig::try_new_exec( - &[input_data], - Arc::clone(&schema), - None, - )?; + let input = + TestMemoryExec::try_new_exec(&[input_data], Arc::clone(&schema), None)?; let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, group_by, @@ -2716,11 +2710,8 @@ mod tests { .unwrap(), ]; - let input = MockMemorySourceConfig::try_new_exec( - &[input_data], - Arc::clone(&schema), - None, - )?; + let input = + TestMemoryExec::try_new_exec(&[input_data], Arc::clone(&schema), None)?; let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, group_by, @@ -2835,7 +2826,7 @@ mod tests { create_record_batch(&schema, (vec![2, 3, 4, 4], vec![1.0, 2.0, 3.0, 4.0]))?, ]; let plan: Arc = - MockMemorySourceConfig::try_new_exec(&[batches], Arc::clone(&schema), None)?; + TestMemoryExec::try_new_exec(&[batches], Arc::clone(&schema), None)?; let grouping_set = PhysicalGroupBy::new( vec![(col("a", &schema)?, "a".to_string())], diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index b67beb39aa4f..23ffd2a28d3c 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -1638,7 +1638,7 @@ impl EmbeddedProjection for HashJoinExec { #[cfg(test)] mod tests { use super::*; - use crate::test::MockMemorySourceConfig; + use crate::test::TestMemoryExec; use crate::{ common, expressions::Column, repartition::RepartitionExec, test::build_table_i32, test::exec::MockExec, @@ -1680,7 +1680,7 @@ mod tests { ) -> Arc { let batch = build_table_i32(a, b, c); let schema = batch.schema(); - MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + TestMemoryExec::try_new_exec(&[vec![batch]], schema, None).unwrap() } fn join( @@ -2082,12 +2082,9 @@ mod tests { let batch2 = build_table_i32(("a1", &vec![2]), ("b2", &vec![2]), ("c1", &vec![9])); let schema = batch1.schema(); - let left = MockMemorySourceConfig::try_new_exec( - &[vec![batch1], vec![batch2]], - schema, - None, - ) - .unwrap(); + let left = + TestMemoryExec::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) + .unwrap(); let right = build_table( ("a1", &vec![1, 2, 3]), @@ -2157,12 +2154,9 @@ mod tests { ); let schema = batch1.schema(); - let left = MockMemorySourceConfig::try_new_exec( - &[vec![batch1], vec![batch2]], - schema, - None, - ) - .unwrap(); + let left = + TestMemoryExec::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) + .unwrap(); let right = build_table( ("a2", &vec![20, 30, 10]), ("b2", &vec![5, 6, 4]), @@ -2214,12 +2208,9 @@ mod tests { let batch2 = build_table_i32(("a2", &vec![30]), ("b1", &vec![5]), ("c2", &vec![90])); let schema = batch1.schema(); - let right = MockMemorySourceConfig::try_new_exec( - &[vec![batch1], vec![batch2]], - schema, - None, - ) - .unwrap(); + let right = + TestMemoryExec::try_new_exec(&[vec![batch1], vec![batch2]], schema, None) + .unwrap(); let on = vec![( Arc::new(Column::new_with_schema("b1", &left.schema())?) as _, @@ -2297,8 +2288,7 @@ mod tests { ) -> Arc { let batch = build_table_i32(a, b, c); let schema = batch.schema(); - MockMemorySourceConfig::try_new_exec(&[vec![batch.clone(), batch]], schema, None) - .unwrap() + TestMemoryExec::try_new_exec(&[vec![batch.clone(), batch]], schema, None).unwrap() } #[apply(batch_sizes)] @@ -2403,8 +2393,7 @@ mod tests { Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, )]; let schema = right.schema(); - let right = - MockMemorySourceConfig::try_new_exec(&[vec![right]], schema, None).unwrap(); + let right = TestMemoryExec::try_new_exec(&[vec![right]], schema, None).unwrap(); let join = join(left, right, on, &JoinType::Left, false).unwrap(); let columns = columns(&join.schema()); @@ -2441,8 +2430,7 @@ mod tests { Arc::new(Column::new_with_schema("b2", &right.schema()).unwrap()) as _, )]; let schema = right.schema(); - let right = - MockMemorySourceConfig::try_new_exec(&[vec![right]], schema, None).unwrap(); + let right = TestMemoryExec::try_new_exec(&[vec![right]], schema, None).unwrap(); let join = join(left, right, on, &JoinType::Full, false).unwrap(); let columns = columns(&join.schema()); @@ -3746,17 +3734,13 @@ mod tests { let dates: ArrayRef = Arc::new(Date32Array::from(vec![19107, 19108, 19109])); let n: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![dates, n])?; - let left = MockMemorySourceConfig::try_new_exec( - &[vec![batch]], - Arc::clone(&schema), - None, - ) - .unwrap(); + let left = + TestMemoryExec::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) + .unwrap(); let dates: ArrayRef = Arc::new(Date32Array::from(vec![19108, 19108, 19109])); let n: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6])); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![dates, n])?; - let right = - MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap(); + let right = TestMemoryExec::try_new_exec(&[vec![batch]], schema, None).unwrap(); let on = vec![( Arc::new(Column::new_with_schema("date", &left.schema()).unwrap()) as _, Arc::new(Column::new_with_schema("date", &right.schema()).unwrap()) as _, @@ -4046,7 +4030,7 @@ mod tests { ("b1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]), ("c1", &vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 0]), ); - let left = MockMemorySourceConfig::try_new_exec( + let left = TestMemoryExec::try_new_exec( &[vec![left_batch.clone()], vec![left_batch.clone()]], left_batch.schema(), None, @@ -4057,7 +4041,7 @@ mod tests { ("b2", &vec![12, 13]), ("c2", &vec![14, 15]), ); - let right = MockMemorySourceConfig::try_new_exec( + let right = TestMemoryExec::try_new_exec( &[vec![right_batch.clone()], vec![right_batch.clone()]], right_batch.schema(), None, @@ -4142,7 +4126,7 @@ mod tests { ) .unwrap(); let schema_ref = batch.schema(); - MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema_ref, None).unwrap() + TestMemoryExec::try_new_exec(&[vec![batch]], schema_ref, None).unwrap() } #[tokio::test] diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 19a507996670..64dfc8219b64 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -1030,8 +1030,7 @@ impl EmbeddedProjection for NestedLoopJoinExec { #[cfg(test)] pub(crate) mod tests { use super::*; - use crate::test::MockMemorySourceConfig; - // use crate::test::MockMemorySourceConfig; + use crate::test::TestMemoryExec; use crate::{ common, expressions::Column, repartition::RepartitionExec, test::build_table_i32, }; @@ -1072,8 +1071,7 @@ pub(crate) mod tests { }; let mut source = - MockMemorySourceConfig::try_new(&[batches], Arc::clone(&schema), None) - .unwrap(); + TestMemoryExec::try_new(&[batches], Arc::clone(&schema), None).unwrap(); if !sorted_column_names.is_empty() { let mut sort_info = LexOrdering::default(); for name in sorted_column_names { @@ -1090,7 +1088,7 @@ pub(crate) mod tests { source = source.try_with_sort_information(vec![sort_info]).unwrap(); } - Arc::new(MockMemorySourceConfig::update_cache(Arc::new(source))) + Arc::new(TestMemoryExec::update_cache(Arc::new(source))) } fn build_left_table() -> Arc { diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 3ef774917d5b..6c933ca21807 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -2547,7 +2547,7 @@ mod tests { use crate::joins::sort_merge_join::{get_corrected_filter_mask, JoinedRecordBatches}; use crate::joins::utils::{ColumnIndex, JoinFilter, JoinOn}; use crate::joins::SortMergeJoinExec; - use crate::test::MockMemorySourceConfig; + use crate::test::TestMemoryExec; use crate::test::{build_table_i32, build_table_i32_two_cols}; use crate::{common, ExecutionPlan}; @@ -2558,12 +2558,12 @@ mod tests { ) -> Arc { let batch = build_table_i32(a, b, c); let schema = batch.schema(); - MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + TestMemoryExec::try_new_exec(&[vec![batch]], schema, None).unwrap() } fn build_table_from_batches(batches: Vec) -> Arc { let schema = batches.first().unwrap().schema(); - MockMemorySourceConfig::try_new_exec(&[batches], schema, None).unwrap() + TestMemoryExec::try_new_exec(&[batches], schema, None).unwrap() } fn build_date_table( @@ -2588,7 +2588,7 @@ mod tests { .unwrap(); let schema = batch.schema(); - MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + TestMemoryExec::try_new_exec(&[vec![batch]], schema, None).unwrap() } fn build_date64_table( @@ -2613,7 +2613,7 @@ mod tests { .unwrap(); let schema = batch.schema(); - MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + TestMemoryExec::try_new_exec(&[vec![batch]], schema, None).unwrap() } /// returns a table with 3 columns of i32 in memory @@ -2636,7 +2636,7 @@ mod tests { ], ) .unwrap(); - MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + TestMemoryExec::try_new_exec(&[vec![batch]], schema, None).unwrap() } pub fn build_table_two_cols( @@ -2645,7 +2645,7 @@ mod tests { ) -> Arc { let batch = build_table_i32_two_cols(a, b); let schema = batch.schema(); - MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + TestMemoryExec::try_new_exec(&[vec![batch]], schema, None).unwrap() } fn join( diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index eca7215ce28e..e70007aa651f 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -24,8 +24,7 @@ use crate::joins::{ HashJoinExec, PartitionMode, StreamJoinPartitionMode, SymmetricHashJoinExec, }; use crate::repartition::RepartitionExec; -use crate::test::MockMemorySourceConfig; -// use crate::test::MockMemorySourceConfig; +use crate::test::TestMemoryExec; use crate::{common, ExecutionPlan, ExecutionPlanProperties, Partitioning}; use arrow::array::{ @@ -530,14 +529,14 @@ pub fn create_memory_table( right_sorted: Vec, ) -> Result<(Arc, Arc)> { let left_schema = left_partition[0].schema(); - let left = MockMemorySourceConfig::try_new(&[left_partition], left_schema, None)? + let left = TestMemoryExec::try_new(&[left_partition], left_schema, None)? .try_with_sort_information(left_sorted)?; let right_schema = right_partition[0].schema(); - let right = MockMemorySourceConfig::try_new(&[right_partition], right_schema, None)? + let right = TestMemoryExec::try_new(&[right_partition], right_schema, None)? .try_with_sort_information(right_sorted)?; Ok(( - Arc::new(MockMemorySourceConfig::update_cache(Arc::new(left))), - Arc::new(MockMemorySourceConfig::update_cache(Arc::new(right))), + Arc::new(TestMemoryExec::update_cache(Arc::new(left))), + Arc::new(TestMemoryExec::update_cache(Arc::new(right))), )) } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index f09fe0d40ea3..40e68cfcae83 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1051,7 +1051,7 @@ mod tests { use std::collections::HashSet; use super::*; - use crate::test::MockMemorySourceConfig; + use crate::test::TestMemoryExec; use crate::{ test::{ assert_is_pending, @@ -1165,11 +1165,8 @@ mod tests { ) -> Result>> { let task_ctx = Arc::new(TaskContext::default()); // create physical plan - let exec = MockMemorySourceConfig::try_new_exec( - &input_partitions, - Arc::clone(schema), - None, - )?; + let exec = + TestMemoryExec::try_new_exec(&input_partitions, Arc::clone(schema), None)?; let exec = RepartitionExec::try_new(exec, partitioning)?; // execute and collect results @@ -1560,11 +1557,8 @@ mod tests { let task_ctx = Arc::new(task_ctx); // create physical plan - let exec = MockMemorySourceConfig::try_new_exec( - &input_partitions, - Arc::clone(&schema), - None, - )?; + let exec = + TestMemoryExec::try_new_exec(&input_partitions, Arc::clone(&schema), None)?; let exec = RepartitionExec::try_new(exec, partitioning)?; // pull partitions @@ -1605,8 +1599,7 @@ mod test { use arrow::datatypes::{DataType, Field, Schema}; use super::*; - use crate::test::MockMemorySourceConfig; - // use crate::test::MockMemorySourceConfig; + use crate::test::TestMemoryExec; use crate::union::UnionExec; use datafusion_physical_expr::expressions::col; @@ -1712,15 +1705,15 @@ mod test { } fn memory_exec(schema: &SchemaRef) -> Arc { - MockMemorySourceConfig::try_new_exec(&[vec![]], Arc::clone(schema), None).unwrap() + TestMemoryExec::try_new_exec(&[vec![]], Arc::clone(schema), None).unwrap() } fn sorted_memory_exec( schema: &SchemaRef, sort_exprs: LexOrdering, ) -> Arc { - Arc::new(MockMemorySourceConfig::update_cache(Arc::new( - MockMemorySourceConfig::try_new(&[vec![]], Arc::clone(schema), None) + Arc::new(TestMemoryExec::update_cache(Arc::new( + TestMemoryExec::try_new(&[vec![]], Arc::clone(schema), None) .unwrap() .try_with_sort_information(vec![sort_exprs]) .unwrap(), diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index a544810eb9a4..dc03c012d9be 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -470,7 +470,7 @@ mod tests { use crate::test; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; - use crate::test::MockMemorySourceConfig; + use crate::test::TestMemoryExec; use super::*; @@ -696,7 +696,7 @@ mod tests { ); let schema = batch1.schema(); - MockMemorySourceConfig::try_new_exec( + TestMemoryExec::try_new_exec( &[vec![batch1, batch2, batch3, batch4]], Arc::clone(&schema), None, @@ -880,11 +880,8 @@ mod tests { Arc::new(vec![1, 1, 2].into_iter().map(Some).collect::()); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data])?; - let input = MockMemorySourceConfig::try_new_exec( - &[vec![batch]], - Arc::clone(&schema), - None, - )?; + let input = + TestMemoryExec::try_new_exec(&[vec![batch]], Arc::clone(&schema), None)?; let partial_sort_exec = Arc::new(PartialSortExec::new( LexOrdering::new(vec![PhysicalSortExpr { @@ -990,7 +987,7 @@ mod tests { options: option_desc, }, ]), - MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None)?, + TestMemoryExec::try_new_exec(&[vec![batch]], schema, None)?, 2, )); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 5ba2ae6651d3..30b5abcf8897 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -1145,7 +1145,7 @@ mod tests { use crate::test; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; - use crate::test::MockMemorySourceConfig; + use crate::test::TestMemoryExec; use arrow::array::*; use arrow::compute::SortOptions; @@ -1530,12 +1530,9 @@ mod tests { Arc::new(vec![3, 2, 1].into_iter().map(Some).collect::()); let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data]).unwrap(); - let input = MockMemorySourceConfig::try_new_exec( - &[vec![batch]], - Arc::clone(&schema), - None, - ) - .unwrap(); + let input = + TestMemoryExec::try_new_exec(&[vec![batch]], Arc::clone(&schema), None) + .unwrap(); let sort_exec = Arc::new(SortExec::new( LexOrdering::new(vec![PhysicalSortExpr { @@ -1605,11 +1602,7 @@ mod tests { }, }, ]), - MockMemorySourceConfig::try_new_exec( - &[vec![batch]], - Arc::clone(&schema), - None, - )?, + TestMemoryExec::try_new_exec(&[vec![batch]], Arc::clone(&schema), None)?, )); assert_eq!(DataType::Int32, *sort_exec.schema().field(0).data_type()); @@ -1695,7 +1688,7 @@ mod tests { }, }, ]), - MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None)?, + TestMemoryExec::try_new_exec(&[vec![batch]], schema, None)?, )); assert_eq!(DataType::Float32, *sort_exec.schema().field(0).data_type()); diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 16d57aa1494b..454a06855175 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -389,7 +389,7 @@ mod tests { use crate::sorts::sort::SortExec; use crate::stream::RecordBatchReceiverStream; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; - use crate::test::MockMemorySourceConfig; + use crate::test::TestMemoryExec; use crate::test::{self, assert_is_pending, make_partition}; use crate::{collect, common}; @@ -451,7 +451,7 @@ mod tests { ]); let repartition_exec = RepartitionExec::try_new( - MockMemorySourceConfig::try_new_exec(&[rbs], schema, None).unwrap(), + TestMemoryExec::try_new_exec(&[rbs], schema, None).unwrap(), Partitioning::RoundRobinBatch(2), )?; let coalesce_batches_exec = @@ -543,7 +543,7 @@ mod tests { let schema = batch.schema(); let sort = LexOrdering::default(); // no sort expressions - let exec = MockMemorySourceConfig::try_new_exec( + let exec = TestMemoryExec::try_new_exec( &[vec![batch.clone()], vec![batch]], schema, None, @@ -736,8 +736,7 @@ mod tests { options: Default::default(), }, ]); - let exec = - MockMemorySourceConfig::try_new_exec(partitions, schema, None).unwrap(); + let exec = TestMemoryExec::try_new_exec(partitions, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, context).await.unwrap(); @@ -845,7 +844,7 @@ mod tests { let sorted = basic_sort(csv, sort, context).await; let split: Vec<_> = sizes.iter().map(|x| split_batch(&sorted, *x)).collect(); - Ok(MockMemorySourceConfig::try_new_exec(&split, sorted.schema(), None).unwrap()) + Ok(TestMemoryExec::try_new_exec(&split, sorted.schema(), None).unwrap()) } #[tokio::test] @@ -974,8 +973,7 @@ mod tests { }, ]); let exec = - MockMemorySourceConfig::try_new_exec(&[vec![b1], vec![b2]], schema, None) - .unwrap(); + TestMemoryExec::try_new_exec(&[vec![b1], vec![b2]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, task_ctx).await.unwrap(); @@ -1017,8 +1015,7 @@ mod tests { nulls_first: true, }, }]); - let exec = - MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap(); + let exec = TestMemoryExec::try_new_exec(&[vec![batch]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec).with_fetch(Some(2))); @@ -1053,8 +1050,7 @@ mod tests { nulls_first: true, }, }]); - let exec = - MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap(); + let exec = TestMemoryExec::try_new_exec(&[vec![batch]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, task_ctx).await.unwrap(); @@ -1164,8 +1160,7 @@ mod tests { options: Default::default(), }]); let exec = - MockMemorySourceConfig::try_new_exec(&[vec![b1], vec![b2]], schema, None) - .unwrap(); + TestMemoryExec::try_new_exec(&[vec![b1], vec![b2]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(Arc::clone(&merge) as Arc, task_ctx) @@ -1276,8 +1271,7 @@ mod tests { }, }]); - let exec = - MockMemorySourceConfig::try_new_exec(&partitions, schema, None).unwrap(); + let exec = TestMemoryExec::try_new_exec(&partitions, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, exec)); let collected = collect(merge, task_ctx).await.unwrap(); diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index 33afdb7fde84..7d0e3778452f 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -49,8 +49,15 @@ use futures::{Future, FutureExt}; pub mod exec; +/// `TestMemoryExec` is a mock equivalent to [`MemorySourceConfig`] with [`ExecutionPlan`] implemented for testing. +/// i.e. It has some but not all the functionality of [`MemorySourceConfig`]. +/// This implements an in-memory DataSource rather than explicitly implementing a trait. +/// It is implemented in this manner to keep relevant unit tests in place +/// while avoiding circular dependencies between `datafusion-physical-plan` and `datafusion-datasource`. +/// +/// [`MemorySourceConfig`]: https://github.com/apache/datafusion/tree/main/datafusion/datasource/src/memory.rs #[derive(Clone, Debug)] -pub struct MockMemorySourceConfig { +pub struct TestMemoryExec { /// The partitions to query partitions: Vec>, /// Schema representing the data before projection @@ -69,7 +76,7 @@ pub struct MockMemorySourceConfig { cache: PlanProperties, } -impl DisplayAs for MockMemorySourceConfig { +impl DisplayAs for TestMemoryExec { fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { write!(f, "DataSourceExec: ")?; match t { @@ -114,7 +121,7 @@ impl DisplayAs for MockMemorySourceConfig { } } -impl ExecutionPlan for MockMemorySourceConfig { +impl ExecutionPlan for TestMemoryExec { fn name(&self) -> &'static str { "DataSourceExec" } @@ -167,7 +174,7 @@ impl ExecutionPlan for MockMemorySourceConfig { } } -impl MockMemorySourceConfig { +impl TestMemoryExec { fn open( &self, partition: usize, @@ -243,7 +250,7 @@ impl MockMemorySourceConfig { partitions: &[Vec], schema: SchemaRef, projection: Option>, - ) -> Result> { + ) -> Result> { let mut source = Self::try_new(partitions, schema, projection)?; let cache = source.compute_properties(); source.cache = cache; @@ -251,7 +258,7 @@ impl MockMemorySourceConfig { } // Equivalent of `DataSourceExec::new` - pub fn update_cache(source: Arc) -> MockMemorySourceConfig { + pub fn update_cache(source: Arc) -> TestMemoryExec { let cache = source.compute_properties(); let source = &*source; let mut source = source.clone(); @@ -280,25 +287,8 @@ impl MockMemorySourceConfig { &self.sort_information } - /// A memory table can be ordered by multiple expressions simultaneously. - /// [`EquivalenceProperties`] keeps track of expressions that describe the - /// global ordering of the schema. These columns are not necessarily same; e.g. - /// ```text - /// ┌-------┐ - /// | a | b | - /// |---|---| - /// | 1 | 9 | - /// | 2 | 8 | - /// | 3 | 7 | - /// | 5 | 5 | - /// └---┴---┘ - /// ``` - /// where both `a ASC` and `b DESC` can describe the table ordering. With - /// [`EquivalenceProperties`], we can keep track of these equivalences - /// and treat `a ASC` and `b DESC` as the same ordering requirement. - /// - /// Note that if there is an internal projection, that projection will be - /// also applied to the given `sort_information`. + /// refer to `try_with_sort_information` at MemorySourceConfig for more information. + /// https://github.com/apache/datafusion/tree/main/datafusion/datasource/src/memory.rs pub fn try_with_sort_information( mut self, mut sort_information: Vec, @@ -317,7 +307,7 @@ impl MockMemorySourceConfig { }); if let Some(col) = ambiguous_column { return internal_err!( - "Column {:?} is not found in the original schema of the MockMemorySourceConfig", + "Column {:?} is not found in the original schema of the TestMemoryExec", col ); } @@ -437,7 +427,7 @@ pub fn build_table_scan_i32( ) -> Arc { let batch = build_table_i32(a, b, c); let schema = batch.schema(); - MockMemorySourceConfig::try_new_exec(&[vec![batch]], schema, None).unwrap() + TestMemoryExec::try_new_exec(&[vec![batch]], schema, None).unwrap() } /// Return a RecordBatch with a single Int32 array with values (0..sz) in a field named "i" @@ -477,16 +467,16 @@ pub fn scan_partitioned_utf8(partitions: usize) -> Arc { } /// Returns a `DataSourceExec` that scans `partitions` of 100 batches each -pub fn mem_exec(partitions: usize) -> MockMemorySourceConfig { +pub fn mem_exec(partitions: usize) -> TestMemoryExec { let data: Vec> = (0..partitions).map(|_| vec![make_partition(100)]).collect(); let schema = data[0][0].schema(); let projection = None; - MockMemorySourceConfig::try_new(&data, schema, projection).unwrap() + TestMemoryExec::try_new(&data, schema, projection).unwrap() } -pub fn mem_exec_utf8(partitions: usize) -> MockMemorySourceConfig { +pub fn mem_exec_utf8(partitions: usize) -> TestMemoryExec { let data: Vec> = (0..partitions) .map(|_| vec![make_partition_utf8(100)]) .collect(); @@ -494,7 +484,7 @@ pub fn mem_exec_utf8(partitions: usize) -> MockMemorySourceConfig { let schema = data[0][0].schema(); let projection = None; - MockMemorySourceConfig::try_new(&data, schema, projection).unwrap() + TestMemoryExec::try_new(&data, schema, projection).unwrap() } // Construct a stream partition for test purposes diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 6d0af8b8b9b3..68d1803b7133 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -641,9 +641,8 @@ mod tests { use super::*; use crate::collect; use crate::test; - use crate::test::MockMemorySourceConfig; + use crate::test::TestMemoryExec; - // use crate::test::MockMemorySourceConfig; use arrow::compute::SortOptions; use arrow::datatypes::DataType; use datafusion_common::ScalarValue; @@ -865,12 +864,12 @@ mod tests { .iter() .map(|ordering| convert_to_sort_exprs(ordering)) .collect::>(); - let child1 = Arc::new(MockMemorySourceConfig::update_cache(Arc::new( - MockMemorySourceConfig::try_new(&[], Arc::clone(&schema), None)? + let child1 = Arc::new(TestMemoryExec::update_cache(Arc::new( + TestMemoryExec::try_new(&[], Arc::clone(&schema), None)? .try_with_sort_information(first_orderings)?, ))); - let child2 = Arc::new(MockMemorySourceConfig::update_cache(Arc::new( - MockMemorySourceConfig::try_new(&[], Arc::clone(&schema), None)? + let child2 = Arc::new(TestMemoryExec::update_cache(Arc::new( + TestMemoryExec::try_new(&[], Arc::clone(&schema), None)? .try_with_sort_information(second_orderings)?, ))); diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index fb97a8b89b72..c78c870ff383 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1192,7 +1192,7 @@ mod tests { use crate::expressions::PhysicalSortExpr; use crate::projection::ProjectionExec; use crate::streaming::{PartitionStream, StreamingTableExec}; - use crate::test::MockMemorySourceConfig; + use crate::test::TestMemoryExec; use crate::windows::{ create_udwf_window_expr, create_window_expr, BoundedWindowAggExec, InputOrderMode, }; @@ -1551,7 +1551,7 @@ mod tests { vec![Arc::new(arrow::array::Int32Array::from(vec![1, 2, 3]))], )?; - let memory_exec = MockMemorySourceConfig::try_new_exec( + let memory_exec = TestMemoryExec::try_new_exec( &[vec![batch.clone(), batch.clone(), batch.clone()]], Arc::clone(&schema), None,