diff --git a/datafusion-examples/examples/advanced_parquet_index.rs b/datafusion-examples/examples/advanced_parquet_index.rs index efaee23366a1..d16dc3894450 100644 --- a/datafusion-examples/examples/advanced_parquet_index.rs +++ b/datafusion-examples/examples/advanced_parquet_index.rs @@ -55,6 +55,7 @@ use arrow::array::{ArrayRef, Int32Array, RecordBatch, StringArray}; use arrow::datatypes::SchemaRef; use async_trait::async_trait; use bytes::Bytes; +use datafusion::config::TableParquetOptions; use datafusion::datasource::memory::DataSourceExec; use futures::future::BoxFuture; use futures::FutureExt; @@ -491,23 +492,22 @@ impl TableProvider for IndexTableProvider { CachedParquetFileReaderFactory::new(Arc::clone(&self.object_store)) .with_file(indexed_file); - let file_source = Arc::new( - ParquetSource::default() + let file_scan_config = FileScanConfigBuilder::new(object_store_url, schema) + .with_limit(limit) + .with_projection(projection.cloned()) + .with_file(partitioned_file) + .build(); + + let file_source = + ParquetSource::new(TableParquetOptions::default(), file_scan_config.clone()) // provide the predicate so the DataSourceExec can try and prune // row groups internally .with_predicate(predicate) // provide the factory to create parquet reader without re-reading metadata - .with_parquet_file_reader_factory(Arc::new(reader_factory)), - ); - let file_scan_config = - FileScanConfigBuilder::new(object_store_url, schema, file_source) - .with_limit(limit) - .with_projection(projection.cloned()) - .with_file(partitioned_file) - .build(); + .with_parquet_file_reader_factory(Arc::new(reader_factory)); // Finally, put it all together into a DataSourceExec - Ok(DataSourceExec::from_data_source(file_scan_config)) + Ok(DataSourceExec::from_data_source(file_source)) } /// Tell DataFusion to push filters down to the scan method diff --git a/datafusion-examples/examples/csv_json_opener.rs b/datafusion-examples/examples/csv_json_opener.rs index 1a2c2cbff418..7f34defea1aa 100644 --- a/datafusion-examples/examples/csv_json_opener.rs +++ b/datafusion-examples/examples/csv_json_opener.rs @@ -24,7 +24,7 @@ use datafusion::{ file_format::file_compression_type::FileCompressionType, listing::PartitionedFile, object_store::ObjectStoreUrl, - physical_plan::{CsvSource, FileSource, FileStream, JsonOpener, JsonSource}, + physical_plan::{CsvSource, FileSource, FileStream, JsonOpener}, }, error::Result, physical_plan::metrics::ExecutionPlanMetricsSet, @@ -58,20 +58,17 @@ async fn csv_opener() -> Result<()> { let scan_config = FileScanConfigBuilder::new( ObjectStoreUrl::local_filesystem(), Arc::clone(&schema), - Arc::new(CsvSource::default()), ) .with_projection(Some(vec![12, 0])) + .with_batch_size(Some(8192)) .with_limit(Some(5)) .with_file(PartitionedFile::new(path.display().to_string(), 10)) .build(); - let config = CsvSource::new(true, b',', b'"') - .with_comment(Some(b'#')) - .with_schema(schema) - .with_batch_size(8192) - .with_projection(&scan_config); + let source = + CsvSource::new(true, b',', b'"', scan_config.clone()).with_comment(Some(b'#')); - let opener = config.create_file_opener(object_store, &scan_config, 0); + let opener = source.create_file_opener(object_store, 0); let mut result = vec![]; let mut stream = @@ -121,15 +118,12 @@ async fn json_opener() -> Result<()> { Arc::new(object_store), ); - let scan_config = FileScanConfigBuilder::new( - ObjectStoreUrl::local_filesystem(), - schema, - Arc::new(JsonSource::default()), - ) - .with_projection(Some(vec![1, 0])) - .with_limit(Some(5)) - .with_file(PartitionedFile::new(path.to_string(), 10)) - .build(); + let scan_config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), schema) + .with_projection(Some(vec![1, 0])) + .with_limit(Some(5)) + .with_file(PartitionedFile::new(path.to_string(), 10)) + .build(); let mut stream = FileStream::new( &scan_config, diff --git a/datafusion-examples/examples/custom_file_format.rs b/datafusion-examples/examples/custom_file_format.rs index 67fe642fd46e..05e26c5d2fbe 100644 --- a/datafusion-examples/examples/custom_file_format.rs +++ b/datafusion-examples/examples/custom_file_format.rs @@ -111,9 +111,11 @@ impl FileFormat for TSVFileFormat { async fn create_physical_plan( &self, state: &dyn Session, - conf: FileScanConfig, + source: Arc, ) -> Result> { - self.csv_file_format.create_physical_plan(state, conf).await + self.csv_file_format + .create_physical_plan(state, source) + .await } async fn create_writer_physical_plan( @@ -128,8 +130,8 @@ impl FileFormat for TSVFileFormat { .await } - fn file_source(&self) -> Arc { - self.csv_file_format.file_source() + fn file_source(&self, config: FileScanConfig) -> Arc { + self.csv_file_format.file_source(config) } } diff --git a/datafusion-examples/examples/default_column_values.rs b/datafusion-examples/examples/default_column_values.rs index 43e2d4ca0988..bcaa3387b911 100644 --- a/datafusion-examples/examples/default_column_values.rs +++ b/datafusion-examples/examples/default_column_values.rs @@ -29,6 +29,7 @@ use datafusion::catalog::{Session, TableProvider}; use datafusion::common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion::common::DFSchema; use datafusion::common::{Result, ScalarValue}; +use datafusion::config::TableParquetOptions; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::physical_plan::{FileScanConfigBuilder, ParquetSource}; use datafusion::execution::context::SessionContext; @@ -235,10 +236,6 @@ impl TableProvider for DefaultValueTableProvider { &df_schema, )?; - let parquet_source = ParquetSource::default() - .with_predicate(filter) - .with_pushdown_filters(true); - let object_store_url = ObjectStoreUrl::parse("memory://")?; let store = state.runtime_env().object_store(object_store_url)?; @@ -255,19 +252,21 @@ impl TableProvider for DefaultValueTableProvider { .map(|file| PartitionedFile::new(file.location.clone(), file.size)) .collect(); - let file_scan_config = FileScanConfigBuilder::new( + let config = FileScanConfigBuilder::new( ObjectStoreUrl::parse("memory://")?, self.schema.clone(), - Arc::new(parquet_source), ) .with_projection(projection.cloned()) .with_limit(limit) .with_file_group(file_group) - .with_expr_adapter(Some(Arc::new(DefaultValuePhysicalExprAdapterFactory) as _)); - - Ok(Arc::new(DataSourceExec::new(Arc::new( - file_scan_config.build(), - )))) + .with_expr_adapter(Some(Arc::new(DefaultValuePhysicalExprAdapterFactory) as _)) + .build(); + + Ok(DataSourceExec::from_data_source( + ParquetSource::new(TableParquetOptions::default(), config.clone()) + .with_predicate(filter) + .with_pushdown_filters(true), + )) } } diff --git a/datafusion-examples/examples/json_shredding.rs b/datafusion-examples/examples/json_shredding.rs index b7acb5c7b74c..6563c63a3e39 100644 --- a/datafusion-examples/examples/json_shredding.rs +++ b/datafusion-examples/examples/json_shredding.rs @@ -29,6 +29,7 @@ use datafusion::common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; use datafusion::common::{assert_contains, DFSchema, Result}; +use datafusion::config::TableParquetOptions; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::physical_plan::{FileScanConfigBuilder, ParquetSource}; use datafusion::execution::context::SessionContext; @@ -243,10 +244,6 @@ impl TableProvider for ExampleTableProvider { &df_schema, )?; - let parquet_source = ParquetSource::default() - .with_predicate(filter) - .with_pushdown_filters(true); - let object_store_url = ObjectStoreUrl::parse("memory://")?; let store = state.runtime_env().object_store(object_store_url)?; @@ -264,20 +261,20 @@ impl TableProvider for ExampleTableProvider { .map(|file| PartitionedFile::new(file.location.clone(), file.size)) .collect(); - let file_scan_config = FileScanConfigBuilder::new( - ObjectStoreUrl::parse("memory://")?, - schema, - Arc::new(parquet_source), - ) - .with_projection(projection.cloned()) - .with_limit(limit) - .with_file_group(file_group) - // if the rewriter needs a reference to the table schema you can bind self.schema() here - .with_expr_adapter(Some(Arc::new(ShreddedJsonRewriterFactory) as _)); - - Ok(Arc::new(DataSourceExec::new(Arc::new( - file_scan_config.build(), - )))) + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::parse("memory://")?, schema) + .with_projection(projection.cloned()) + .with_limit(limit) + .with_file_group(file_group) + // if the rewriter needs a reference to the table schema you can bind self.schema() here + .with_expr_adapter(Some(Arc::new(ShreddedJsonRewriterFactory) as _)) + .build(); + + Ok(DataSourceExec::from_data_source( + ParquetSource::new(TableParquetOptions::default(), config.clone()) + .with_predicate(filter) + .with_pushdown_filters(true), + )) } } diff --git a/datafusion-examples/examples/parquet_embedded_index.rs b/datafusion-examples/examples/parquet_embedded_index.rs index 5191ae48b3af..f8c162c004ad 100644 --- a/datafusion-examples/examples/parquet_embedded_index.rs +++ b/datafusion-examples/examples/parquet_embedded_index.rs @@ -117,6 +117,7 @@ use arrow_schema::{DataType, Field, Schema, SchemaRef}; use async_trait::async_trait; use datafusion::catalog::{Session, TableProvider}; use datafusion::common::{exec_err, HashMap, HashSet, Result}; +use datafusion::config::TableParquetOptions; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::memory::DataSourceExec; use datafusion::datasource::physical_plan::{FileScanConfigBuilder, ParquetSource}; @@ -426,8 +427,8 @@ impl TableProvider for DistinctIndexTable { // Build ParquetSource to actually read the files let url = ObjectStoreUrl::parse("file://")?; - let source = Arc::new(ParquetSource::default().with_enable_page_index(true)); - let mut builder = FileScanConfigBuilder::new(url, self.schema.clone(), source); + + let mut builder = FileScanConfigBuilder::new(url, self.schema.clone()); for file in files_to_scan { let path = self.dir.join(file); let len = std::fs::metadata(&path)?.len(); @@ -438,7 +439,11 @@ impl TableProvider for DistinctIndexTable { PartitionedFile::new(path.to_str().unwrap().to_string(), len); builder = builder.with_file(partitioned_file); } - Ok(DataSourceExec::from_data_source(builder.build())) + + Ok(DataSourceExec::from_data_source( + ParquetSource::new(TableParquetOptions::default(), builder.build()) + .with_enable_page_index(true), + )) } /// Tell DataFusion that we can handle filters on the "category" column diff --git a/datafusion-examples/examples/parquet_exec_visitor.rs b/datafusion-examples/examples/parquet_exec_visitor.rs index 84f92d4f450e..cd97e0727761 100644 --- a/datafusion-examples/examples/parquet_exec_visitor.rs +++ b/datafusion-examples/examples/parquet_exec_visitor.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use datafusion::datasource::file_format::parquet::ParquetFormat; use datafusion::datasource::listing::ListingOptions; -use datafusion::datasource::physical_plan::{FileGroup, ParquetSource}; +use datafusion::datasource::physical_plan::{FileGroup, FileSource, ParquetSource}; use datafusion::datasource::source::DataSourceExec; use datafusion::error::DataFusionError; use datafusion::execution::context::SessionContext; @@ -98,9 +98,11 @@ impl ExecutionPlanVisitor for ParquetExecVisitor { fn pre_visit(&mut self, plan: &dyn ExecutionPlan) -> Result { // If needed match on a specific `ExecutionPlan` node type if let Some(data_source_exec) = plan.as_any().downcast_ref::() { - if let Some((file_config, _)) = + if let Some(parquet_source) = data_source_exec.downcast_to_file_source::() { + let file_config = parquet_source.config(); + self.file_groups = Some(file_config.file_groups.clone()); let metrics = match data_source_exec.metrics() { diff --git a/datafusion-examples/examples/parquet_index.rs b/datafusion-examples/examples/parquet_index.rs index afc3b279f4a9..5ab3529d2165 100644 --- a/datafusion-examples/examples/parquet_index.rs +++ b/datafusion-examples/examples/parquet_index.rs @@ -27,6 +27,7 @@ use datafusion::common::pruning::PruningStatistics; use datafusion::common::{ internal_datafusion_err, DFSchema, DataFusionError, Result, ScalarValue, }; +use datafusion::config::TableParquetOptions; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::memory::DataSourceExec; use datafusion::datasource::physical_plan::{FileScanConfigBuilder, ParquetSource}; @@ -243,9 +244,9 @@ impl TableProvider for IndexTableProvider { let files = self.index.get_files(predicate.clone())?; let object_store_url = ObjectStoreUrl::parse("file://")?; - let source = Arc::new(ParquetSource::default().with_predicate(predicate)); + let mut file_scan_config_builder = - FileScanConfigBuilder::new(object_store_url, self.schema(), source) + FileScanConfigBuilder::new(object_store_url, self.schema()) .with_projection(projection.cloned()) .with_limit(limit); @@ -258,8 +259,13 @@ impl TableProvider for IndexTableProvider { PartitionedFile::new(canonical_path.display().to_string(), file_size), ); } + Ok(DataSourceExec::from_data_source( - file_scan_config_builder.build(), + ParquetSource::new( + TableParquetOptions::default(), + file_scan_config_builder.build(), + ) + .with_predicate(predicate), )) } diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 5ce70e32843d..3ec02650d66b 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -49,7 +49,7 @@ use datafusion_common::{ use datafusion_common_runtime::{JoinSet, SpawnedTask}; use datafusion_datasource::display::FileGroupDisplay; use datafusion_datasource::file::FileSource; -use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::sink::{DataSink, DataSinkExec}; use datafusion_datasource::write::ObjectWriterBuilder; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; @@ -58,7 +58,6 @@ use datafusion_physical_expr_common::sort_expr::LexRequirement; use async_trait::async_trait; use bytes::Bytes; -use datafusion_datasource::source::DataSourceExec; use futures::stream::BoxStream; use futures::StreamExt; use object_store::{GetResultPayload, ObjectMeta, ObjectStore}; @@ -173,19 +172,6 @@ impl FileFormat for ArrowFormat { Ok(Statistics::new_unknown(&table_schema)) } - async fn create_physical_plan( - &self, - _state: &dyn Session, - conf: FileScanConfig, - ) -> Result> { - let source = Arc::new(ArrowSource::default()); - let config = FileScanConfigBuilder::from(conf) - .with_source(source) - .build(); - - Ok(DataSourceExec::from_data_source(config)) - } - async fn create_writer_physical_plan( &self, input: Arc, @@ -202,8 +188,8 @@ impl FileFormat for ArrowFormat { Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) } - fn file_source(&self) -> Arc { - Arc::new(ArrowSource::default()) + fn file_source(&self, config: FileScanConfig) -> Arc { + Arc::new(ArrowSource::new(config)) } } diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index e165707c2eb0..12e926324d92 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -80,22 +80,16 @@ pub(crate) mod test_util { }] .into()]; - let exec = format - .create_physical_plan( - state, - FileScanConfigBuilder::new( - ObjectStoreUrl::local_filesystem(), - file_schema, - format.file_source(), - ) + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_schema) .with_file_groups(file_groups) - .with_statistics(statistics) + .with_file_source_projected_statistics(statistics) .with_projection(projection) .with_limit(limit) - .build(), - ) - .await?; - Ok(exec) + .build(); + + let source = format.file_source(config); + format.create_physical_plan(state, source).await } } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index d289a1d07129..a2d59a7a956c 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1113,8 +1113,11 @@ impl ListingTable { } /// Creates a file source and applies schema adapter factory if available - fn create_file_source_with_schema_adapter(&self) -> Result> { - let mut source = self.options.format.file_source(); + fn create_file_source_with_schema_adapter( + &self, + config: FileScanConfig, + ) -> Result> { + let mut source = self.options.format.file_source(config); // Apply schema adapter to source if available // // The source will use this SchemaAdapter to adapt data batches as they flow up the plan. @@ -1233,27 +1236,24 @@ impl TableProvider for ListingTable { return Ok(Arc::new(EmptyExec::new(Arc::new(Schema::empty())))); }; - let file_source = self.create_file_source_with_schema_adapter()?; - - // create the execution plan - self.options - .format - .create_physical_plan( - state, - FileScanConfigBuilder::new( - object_store_url, - Arc::clone(&self.file_schema), - file_source, - ) + let conf = + FileScanConfigBuilder::new(object_store_url, Arc::clone(&self.file_schema)) .with_file_groups(partitioned_file_lists) .with_constraints(self.constraints.clone()) - .with_statistics(statistics) + .with_file_source_projected_statistics(statistics) .with_projection(projection.cloned()) .with_limit(limit) .with_output_ordering(output_ordering) .with_table_partition_cols(table_partition_cols) .with_expr_adapter(self.expr_adapter_factory.clone()) - .build(), + .build(); + + // create the execution plan + self.options + .format + .create_physical_plan( + state, + self.create_file_source_with_schema_adapter(conf)?, ) .await } diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 94d651ddadd5..1cb3317d10b2 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -50,7 +50,6 @@ pub use datafusion_physical_expr::create_ordering; #[cfg(all(test, feature = "parquet"))] mod tests { - use crate::prelude::SessionContext; use ::object_store::{path::Path, ObjectMeta}; use arrow::{ @@ -58,6 +57,7 @@ mod tests { datatypes::{DataType, Field, Schema, SchemaRef}, record_batch::RecordBatch, }; + use datafusion_common::config::TableParquetOptions; use datafusion_common::{record_batch, test_util::batches_to_sort_string}; use datafusion_datasource::{ file::FileSource, @@ -123,18 +123,18 @@ mod tests { let f2 = Field::new("extra_column", DataType::Utf8, true); let schema = Arc::new(Schema::new(vec![f1.clone(), f2.clone()])); - let source = ParquetSource::default() - .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})) - .unwrap(); - let base_conf = FileScanConfigBuilder::new( - ObjectStoreUrl::local_filesystem(), - schema, - source, - ) - .with_file(partitioned_file) - .build(); - let parquet_exec = DataSourceExec::from_data_source(base_conf); + let base_conf = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), schema) + .with_file(partitioned_file) + .build(); + + let source = + ParquetSource::new(TableParquetOptions::default(), base_conf.clone()) + .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})) + .unwrap(); + + let parquet_exec = Arc::new(DataSourceExec::new(source.as_data_source())); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index d0af96329b5f..bc1167375c73 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -24,25 +24,35 @@ use datafusion_datasource::as_file_source; use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use arrow::buffer::Buffer; -use arrow::datatypes::SchemaRef; use arrow_ipc::reader::FileDecoder; -use datafusion_common::Statistics; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::PartitionedFile; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion_datasource::source::DataSource; use futures::StreamExt; use itertools::Itertools; use object_store::{GetOptions, GetRange, GetResultPayload, ObjectStore}; /// Arrow configuration struct that is given to DataSourceExec /// Does not hold anything special, since [`FileScanConfig`] is sufficient for arrow -#[derive(Clone, Default)] +#[derive(Clone)] pub struct ArrowSource { metrics: ExecutionPlanMetricsSet, - projected_statistics: Option, schema_adapter_factory: Option>, + config: FileScanConfig, +} + +impl ArrowSource { + /// Returns a [`ArrowSource`] + pub fn new(config: FileScanConfig) -> Self { + Self { + metrics: Default::default(), + schema_adapter_factory: None, + config, + } + } } impl From for Arc { @@ -52,15 +62,29 @@ impl From for Arc { } impl FileSource for ArrowSource { + fn config(&self) -> &FileScanConfig { + &self.config + } + + fn with_config(&self, config: FileScanConfig) -> Arc { + let mut this = self.clone(); + this.config = config; + + Arc::new(this) + } + + fn as_data_source(&self) -> Arc { + Arc::new(self.clone()) + } + fn create_file_opener( &self, object_store: Arc, - base_config: &FileScanConfig, _partition: usize, ) -> Arc { Arc::new(ArrowOpener { object_store, - projection: base_config.file_column_projection_indices(), + projection: self.config().file_column_projection_indices(), }) } @@ -68,34 +92,10 @@ impl FileSource for ArrowSource { self } - fn with_batch_size(&self, _batch_size: usize) -> Arc { - Arc::new(Self { ..self.clone() }) - } - - fn with_schema(&self, _schema: SchemaRef) -> Arc { - Arc::new(Self { ..self.clone() }) - } - fn with_statistics(&self, statistics: Statistics) -> Arc { - let mut conf = self.clone(); - conf.projected_statistics = Some(statistics); - Arc::new(conf) - } - - fn with_projection(&self, _config: &FileScanConfig) -> Arc { - Arc::new(Self { ..self.clone() }) - } - - fn metrics(&self) -> &ExecutionPlanMetricsSet { + fn metrics_inner(&self) -> &ExecutionPlanMetricsSet { &self.metrics } - fn statistics(&self) -> Result { - let statistics = &self.projected_statistics; - Ok(statistics - .clone() - .expect("projected_statistics must be set")) - } - fn file_type(&self) -> &str { "arrow" } @@ -115,6 +115,15 @@ impl FileSource for ArrowSource { } } +impl std::fmt::Debug for ArrowSource { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{} {{ ", self.file_type())?; + write!(f, "statistics={:?}, ", self.file_source_statistics())?; + write!(f, "config={:?}, ", self.config())?; + write!(f, " }}") + } +} + /// The struct arrow that implements `[FileOpener]` trait pub struct ArrowOpener { pub object_store: Arc, diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 8a00af959ccc..cd652c51877b 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -24,7 +24,6 @@ pub use datafusion_datasource_avro::source::*; #[cfg(test)] mod tests { - use std::sync::Arc; use crate::prelude::SessionContext; @@ -81,17 +80,14 @@ mod tests { .infer_schema(&state, &store, std::slice::from_ref(&meta)) .await?; - let source = Arc::new(AvroSource::new()); - let conf = FileScanConfigBuilder::new( - ObjectStoreUrl::local_filesystem(), - file_schema, - source, - ) - .with_file(meta.into()) - .with_projection(Some(vec![0, 1, 2])) - .build(); - - let source_exec = DataSourceExec::from_data_source(conf); + let conf = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_schema) + .with_file(meta.into()) + .with_projection(Some(vec![0, 1, 2])) + .build(); + + let source = AvroSource::new(conf); + let source_exec = DataSourceExec::from_data_source(source); assert_eq!( source_exec .properties() @@ -157,13 +153,13 @@ mod tests { // Include the missing column in the projection let projection = Some(vec![0, 1, 2, actual_schema.fields().len()]); - let source = Arc::new(AvroSource::new()); - let conf = FileScanConfigBuilder::new(object_store_url, file_schema, source) + let conf = FileScanConfigBuilder::new(object_store_url, file_schema) .with_file(meta.into()) .with_projection(projection) .build(); - let source_exec = DataSourceExec::from_data_source(conf); + let source = AvroSource::new(conf.clone()); + let source_exec = DataSourceExec::from_data_source(source); assert_eq!( source_exec .properties() @@ -227,8 +223,7 @@ mod tests { partitioned_file.partition_values = vec![ScalarValue::from("2021-10-26")]; let projection = Some(vec![0, 1, file_schema.fields().len(), 2]); - let source = Arc::new(AvroSource::new()); - let conf = FileScanConfigBuilder::new(object_store_url, file_schema, source) + let conf = FileScanConfigBuilder::new(object_store_url, file_schema) // select specific columns of the files as well as the partitioning // column which is supposed to be the last column in the table schema. .with_projection(projection) @@ -236,7 +231,7 @@ mod tests { .with_table_partition_cols(vec![Field::new("date", DataType::Utf8, false)]) .build(); - let source_exec = DataSourceExec::from_data_source(conf); + let source_exec = DataSourceExec::from_data_source(AvroSource::new(conf)); assert_eq!( source_exec diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index e33761a0abb3..1d43059561d1 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -23,7 +23,6 @@ pub use datafusion_datasource_csv::source::*; #[cfg(test)] mod tests { - use std::collections::HashMap; use std::fs::{self, File}; use std::io::Write; @@ -110,19 +109,16 @@ mod tests { tmp_dir.path(), )?; - let source = Arc::new(CsvSource::new(true, b',', b'"')); - let config = FileScanConfigBuilder::from(partitioned_csv_config( - file_schema, - file_groups, - source, - )) - .with_file_compression_type(file_compression_type) - .with_newlines_in_values(false) - .with_projection(Some(vec![0, 2, 4])) - .build(); + let config = + FileScanConfigBuilder::from(partitioned_csv_config(file_schema, file_groups)) + .with_file_compression_type(file_compression_type) + .with_newlines_in_values(false) + .with_projection(Some(vec![0, 2, 4])) + .build(); + let source = CsvSource::new(true, b',', b'"', config.clone()); assert_eq!(13, config.file_schema.fields().len()); - let csv = DataSourceExec::from_data_source(config); + let csv = DataSourceExec::from_data_source(source); assert_eq!(3, csv.schema().fields().len()); @@ -175,18 +171,16 @@ mod tests { tmp_dir.path(), )?; - let source = Arc::new(CsvSource::new(true, b',', b'"')); - let config = FileScanConfigBuilder::from(partitioned_csv_config( - file_schema, - file_groups, - source, - )) - .with_newlines_in_values(false) - .with_file_compression_type(file_compression_type.to_owned()) - .with_projection(Some(vec![4, 0, 2])) - .build(); + let config = + FileScanConfigBuilder::from(partitioned_csv_config(file_schema, file_groups)) + .with_newlines_in_values(false) + .with_file_compression_type(file_compression_type.to_owned()) + .with_projection(Some(vec![4, 0, 2])) + .build(); + + let source = CsvSource::new(true, b',', b'"', config.clone()); assert_eq!(13, config.file_schema.fields().len()); - let csv = DataSourceExec::from_data_source(config); + let csv = DataSourceExec::from_data_source(source); assert_eq!(3, csv.schema().fields().len()); let mut stream = csv.execute(0, task_ctx)?; @@ -240,18 +234,15 @@ mod tests { tmp_dir.path(), )?; - let source = Arc::new(CsvSource::new(true, b',', b'"')); - let config = FileScanConfigBuilder::from(partitioned_csv_config( - file_schema, - file_groups, - source, - )) - .with_newlines_in_values(false) - .with_file_compression_type(file_compression_type.to_owned()) - .with_limit(Some(5)) - .build(); + let config = + FileScanConfigBuilder::from(partitioned_csv_config(file_schema, file_groups)) + .with_newlines_in_values(false) + .with_file_compression_type(file_compression_type.to_owned()) + .with_limit(Some(5)) + .build(); assert_eq!(13, config.file_schema.fields().len()); - let csv = DataSourceExec::from_data_source(config); + let source = CsvSource::new(true, b',', b'"', config.clone()); + let csv = DataSourceExec::from_data_source(source); assert_eq!(13, csv.schema().fields().len()); let mut it = csv.execute(0, task_ctx)?; @@ -303,18 +294,15 @@ mod tests { tmp_dir.path(), )?; - let source = Arc::new(CsvSource::new(true, b',', b'"')); - let config = FileScanConfigBuilder::from(partitioned_csv_config( - file_schema, - file_groups, - source, - )) - .with_newlines_in_values(false) - .with_file_compression_type(file_compression_type.to_owned()) - .with_limit(Some(5)) - .build(); + let config = + FileScanConfigBuilder::from(partitioned_csv_config(file_schema, file_groups)) + .with_newlines_in_values(false) + .with_file_compression_type(file_compression_type.to_owned()) + .with_limit(Some(5)) + .build(); assert_eq!(14, config.file_schema.fields().len()); - let csv = DataSourceExec::from_data_source(config); + let source = CsvSource::new(true, b',', b'"', config.clone()); + let csv = DataSourceExec::from_data_source(source); assert_eq!(14, csv.schema().fields().len()); // errors due to https://github.com/apache/datafusion/issues/4918 @@ -358,15 +346,11 @@ mod tests { tmp_dir.path(), )?; - let source = Arc::new(CsvSource::new(true, b',', b'"')); - let mut config = FileScanConfigBuilder::from(partitioned_csv_config( - file_schema, - file_groups, - source, - )) - .with_newlines_in_values(false) - .with_file_compression_type(file_compression_type.to_owned()) - .build(); + let mut config = + FileScanConfigBuilder::from(partitioned_csv_config(file_schema, file_groups)) + .with_newlines_in_values(false) + .with_file_compression_type(file_compression_type.to_owned()) + .build(); // Add partition columns config.table_partition_cols = @@ -381,7 +365,9 @@ mod tests { // partitions are resolved during scan anyway assert_eq!(13, config.file_schema.fields().len()); - let csv = DataSourceExec::from_data_source(config); + + let source = CsvSource::new(true, b',', b'"', config.clone()); + let csv = DataSourceExec::from_data_source(source); assert_eq!(2, csv.schema().fields().len()); let mut it = csv.execute(0, task_ctx)?; @@ -464,16 +450,13 @@ mod tests { ) .unwrap(); - let source = Arc::new(CsvSource::new(true, b',', b'"')); - let config = FileScanConfigBuilder::from(partitioned_csv_config( - file_schema, - file_groups, - source, - )) - .with_newlines_in_values(false) - .with_file_compression_type(file_compression_type.to_owned()) - .build(); - let csv = DataSourceExec::from_data_source(config); + let config = + FileScanConfigBuilder::from(partitioned_csv_config(file_schema, file_groups)) + .with_newlines_in_values(false) + .with_file_compression_type(file_compression_type.to_owned()) + .build(); + let source = CsvSource::new(true, b',', b'"', config.clone()); + let csv = DataSourceExec::from_data_source(source); let it = csv.execute(0, task_ctx).unwrap(); let batches: Vec<_> = it.try_collect().await.unwrap(); diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 0d45711c76fb..feee42608b6a 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -23,7 +23,6 @@ pub use datafusion_datasource_json::source::*; #[cfg(test)] mod tests { - use super::*; use std::fs; @@ -176,13 +175,14 @@ mod tests { let (object_store_url, file_groups, file_schema) = prepare_store(&state, file_compression_type.to_owned(), tmp_dir.path()).await; - let source = Arc::new(JsonSource::new()); - let conf = FileScanConfigBuilder::new(object_store_url, file_schema, source) + let conf = FileScanConfigBuilder::new(object_store_url, file_schema) .with_file_groups(file_groups) .with_limit(Some(3)) .with_file_compression_type(file_compression_type.to_owned()) .build(); - let exec = DataSourceExec::from_data_source(conf); + + let source = JsonSource::new(conf.clone()); + let exec = DataSourceExec::from_data_source(source); // TODO: this is not where schema inference should be tested @@ -251,13 +251,13 @@ mod tests { let file_schema = Arc::new(builder.finish()); let missing_field_idx = file_schema.fields.len() - 1; - let source = Arc::new(JsonSource::new()); - let conf = FileScanConfigBuilder::new(object_store_url, file_schema, source) + let conf = FileScanConfigBuilder::new(object_store_url, file_schema) .with_file_groups(file_groups) .with_limit(Some(3)) .with_file_compression_type(file_compression_type.to_owned()) .build(); - let exec = DataSourceExec::from_data_source(conf); + let source = JsonSource::new(conf.clone()); + let exec = DataSourceExec::from_data_source(source); let mut it = exec.execute(0, task_ctx)?; let batch = it.next().await.unwrap()?; @@ -294,13 +294,13 @@ mod tests { let (object_store_url, file_groups, file_schema) = prepare_store(&state, file_compression_type.to_owned(), tmp_dir.path()).await; - let source = Arc::new(JsonSource::new()); - let conf = FileScanConfigBuilder::new(object_store_url, file_schema, source) + let conf = FileScanConfigBuilder::new(object_store_url, file_schema) .with_file_groups(file_groups) .with_projection(Some(vec![0, 2])) .with_file_compression_type(file_compression_type.to_owned()) .build(); - let exec = DataSourceExec::from_data_source(conf); + let source = JsonSource::new(conf.clone()); + let exec = DataSourceExec::from_data_source(source); let inferred_schema = exec.schema(); assert_eq!(inferred_schema.fields().len(), 2); @@ -342,13 +342,13 @@ mod tests { let (object_store_url, file_groups, file_schema) = prepare_store(&state, file_compression_type.to_owned(), tmp_dir.path()).await; - let source = Arc::new(JsonSource::new()); - let conf = FileScanConfigBuilder::new(object_store_url, file_schema, source) + let conf = FileScanConfigBuilder::new(object_store_url, file_schema) .with_file_groups(file_groups) .with_projection(Some(vec![3, 0, 2])) .with_file_compression_type(file_compression_type.to_owned()) .build(); - let exec = DataSourceExec::from_data_source(conf); + let source = JsonSource::new(conf.clone()); + let exec = DataSourceExec::from_data_source(source); let inferred_schema = exec.schema(); assert_eq!(inferred_schema.fields().len(), 3); diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 55db0d854204..918beec1b826 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -160,7 +160,14 @@ mod tests { .as_ref() .map(|p| logical2physical(p, &table_schema)); - let mut source = ParquetSource::default(); + let base_config = FileScanConfigBuilder::new( + ObjectStoreUrl::local_filesystem(), + table_schema, + ) + .build(); + + let mut source = + ParquetSource::new(TableParquetOptions::default(), base_config); if let Some(predicate) = predicate { source = source.with_predicate(predicate); } @@ -185,7 +192,7 @@ mod tests { source = source.with_bloom_filter_on_read(false); } - source.with_schema(Arc::clone(&table_schema)) + Arc::new(source) as Arc } fn build_parquet_exec( @@ -197,12 +204,14 @@ mod tests { let base_config = FileScanConfigBuilder::new( ObjectStoreUrl::local_filesystem(), file_schema, - source, ) .with_file_group(file_group) .with_projection(self.projection.clone()) .build(); - DataSourceExec::from_data_source(base_config) + + Arc::new(DataSourceExec::new( + source.with_config(base_config).as_data_source(), + )) } /// run the test, returning the `RoundTripResult` @@ -1547,12 +1556,14 @@ mod tests { let config = FileScanConfigBuilder::new( ObjectStoreUrl::local_filesystem(), file_schema, - Arc::new(ParquetSource::default()), ) .with_file_groups(file_groups) .build(); - let parquet_exec = DataSourceExec::from_data_source(config); + let source = + ParquetSource::new(TableParquetOptions::default(), config.clone()); + + let parquet_exec = DataSourceExec::from_data_source(source); assert_eq!( parquet_exec .properties() @@ -1649,8 +1660,7 @@ mod tests { ), ]); - let source = Arc::new(ParquetSource::default()); - let config = FileScanConfigBuilder::new(object_store_url, schema.clone(), source) + let config = FileScanConfigBuilder::new(object_store_url, schema.clone()) .with_file(partitioned_file) // file has 10 cols so index 12 should be month and 13 should be day .with_projection(Some(vec![0, 1, 2, 12, 13])) @@ -1668,7 +1678,9 @@ mod tests { ]) .build(); - let parquet_exec = DataSourceExec::from_data_source(config); + let source = ParquetSource::new(TableParquetOptions::default(), config.clone()); + + let parquet_exec = DataSourceExec::from_data_source(source); let partition_count = parquet_exec .data_source() .output_partitioning() @@ -1725,15 +1737,14 @@ mod tests { }; let file_schema = Arc::new(Schema::empty()); - let config = FileScanConfigBuilder::new( - ObjectStoreUrl::local_filesystem(), - file_schema, - Arc::new(ParquetSource::default()), - ) - .with_file(partitioned_file) - .build(); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_schema) + .with_file(partitioned_file) + .build(); - let parquet_exec = DataSourceExec::from_data_source(config); + let source = ParquetSource::new(TableParquetOptions::default(), config.clone()); + + let parquet_exec = DataSourceExec::from_data_source(source); let mut results = parquet_exec.execute(0, state.task_ctx())?; let batch = results.next().await.unwrap(); @@ -2249,12 +2260,7 @@ mod tests { let size_hint_calls = reader_factory.metadata_size_hint_calls.clone(); - let source = Arc::new( - ParquetSource::default() - .with_parquet_file_reader_factory(reader_factory) - .with_metadata_size_hint(456), - ); - let config = FileScanConfigBuilder::new(store_url, schema, source) + let config = FileScanConfigBuilder::new(store_url, schema) .with_file( PartitionedFile { object_meta: ObjectMeta { @@ -2288,7 +2294,11 @@ mod tests { }) .build(); - let exec = DataSourceExec::from_data_source(config); + let source = ParquetSource::new(TableParquetOptions::default(), config.clone()) + .with_parquet_file_reader_factory(reader_factory) + .with_metadata_size_hint(456); + + let exec = DataSourceExec::from_data_source(source); let res = collect(exec, ctx.task_ctx()).await.unwrap(); assert_eq!(res.len(), 2); diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 68f83e7f1f11..f1232379c658 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -92,12 +92,14 @@ pub fn scan_partitioned_csv( FileCompressionType::UNCOMPRESSED, work_dir, )?; - let source = Arc::new(CsvSource::new(true, b'"', b'"')); - let config = - FileScanConfigBuilder::from(partitioned_csv_config(schema, file_groups, source)) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(); - Ok(DataSourceExec::from_data_source(config)) + + let config = FileScanConfigBuilder::from(partitioned_csv_config(schema, file_groups)) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(); + + let source = CsvSource::new(true, b'"', b'"', config.clone()); + + Ok(DataSourceExec::from_data_source(source)) } /// Returns file groups [`Vec`] for scanning `partitions` of `filename` diff --git a/datafusion/core/src/test_util/parquet.rs b/datafusion/core/src/test_util/parquet.rs index eb4c61c02524..594834b2c28f 100644 --- a/datafusion/core/src/test_util/parquet.rs +++ b/datafusion/core/src/test_util/parquet.rs @@ -37,7 +37,6 @@ use crate::physical_plan::metrics::MetricsSet; use crate::physical_plan::ExecutionPlan; use crate::prelude::{Expr, SessionConfig, SessionContext}; -use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_datasource::source::DataSourceExec; use object_store::path::Path; @@ -156,11 +155,9 @@ impl TestParquetFile { maybe_filter: Option, ) -> Result> { let parquet_options = ctx.copied_table_options().parquet; - let source = Arc::new(ParquetSource::new(parquet_options.clone())); let scan_config_builder = FileScanConfigBuilder::new( self.object_store_url.clone(), Arc::clone(&self.schema), - source, ) .with_file(PartitionedFile { object_meta: self.object_meta.clone(), @@ -182,19 +179,20 @@ impl TestParquetFile { let physical_filter_expr = create_physical_expr(&filter, &df_schema, &ExecutionProps::default())?; - let source = Arc::new( - ParquetSource::new(parquet_options) - .with_predicate(Arc::clone(&physical_filter_expr)), - ) - .with_schema(Arc::clone(&self.schema)); - let config = scan_config_builder.with_source(source).build(); - let parquet_exec = DataSourceExec::from_data_source(config); + let config = scan_config_builder.build(); + let source = ParquetSource::new(parquet_options, config.clone()) + .with_predicate(Arc::clone(&physical_filter_expr)); + + let parquet_exec = DataSourceExec::from_data_source(source); let exec = Arc::new(FilterExec::try_new(physical_filter_expr, parquet_exec)?); Ok(exec) } else { let config = scan_config_builder.build(); - Ok(DataSourceExec::from_data_source(config)) + + let source = ParquetSource::new(parquet_options, config.clone()); + + Ok(DataSourceExec::from_data_source(source)) } } diff --git a/datafusion/core/tests/fuzz_cases/pruning.rs b/datafusion/core/tests/fuzz_cases/pruning.rs index c6e30c0722fc..2d8088fba8ed 100644 --- a/datafusion/core/tests/fuzz_cases/pruning.rs +++ b/datafusion/core/tests/fuzz_cases/pruning.rs @@ -24,6 +24,7 @@ use datafusion::{ datasource::{listing::PartitionedFile, physical_plan::ParquetSource}, prelude::*, }; +use datafusion_common::config::TableParquetOptions; use datafusion_common::DFSchema; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_datasource::source::DataSourceExec; @@ -275,15 +276,9 @@ async fn execute_with_predicate( schema: Arc, ctx: &SessionContext, ) -> Vec { - let parquet_source = if prune_stats { - ParquetSource::default().with_predicate(predicate.clone()) - } else { - ParquetSource::default() - }; let config = FileScanConfigBuilder::new( ObjectStoreUrl::parse("memory://").unwrap(), schema.clone(), - Arc::new(parquet_source), ) .with_file_group( files @@ -294,7 +289,15 @@ async fn execute_with_predicate( .collect(), ) .build(); - let exec = DataSourceExec::from_data_source(config); + + let parquet_source = if prune_stats { + ParquetSource::new(TableParquetOptions::default(), config.clone()) + .with_predicate(predicate.clone()) + } else { + ParquetSource::new(TableParquetOptions::default(), config.clone()) + }; + + let exec = DataSourceExec::from_data_source(parquet_source); let exec = Arc::new(FilterExec::try_new(predicate, exec).unwrap()) as Arc; diff --git a/datafusion/core/tests/parquet/custom_reader.rs b/datafusion/core/tests/parquet/custom_reader.rs index f7e48fa9cb91..e844db6d1ae5 100644 --- a/datafusion/core/tests/parquet/custom_reader.rs +++ b/datafusion/core/tests/parquet/custom_reader.rs @@ -36,6 +36,7 @@ use datafusion_common::test_util::batches_to_sort_string; use datafusion_common::Result; use bytes::Bytes; +use datafusion_common::config::TableParquetOptions; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_datasource::source::DataSourceExec; use datafusion_datasource_parquet::ObjectStoreFetch; @@ -80,23 +81,21 @@ async fn route_data_access_ops_to_parquet_file_reader_factory() { }) .collect(); - let source = Arc::new( - ParquetSource::default() - // prepare the scan - .with_parquet_file_reader_factory(Arc::new( - InMemoryParquetFileReaderFactory(Arc::clone(&in_memory_object_store)), - )), - ); let base_config = FileScanConfigBuilder::new( // just any url that doesn't point to in memory object store ObjectStoreUrl::local_filesystem(), file_schema, - source, ) .with_file_group(file_group) .build(); - let parquet_exec = DataSourceExec::from_data_source(base_config); + let source = ParquetSource::new(TableParquetOptions::default(), base_config.clone()) + // prepare the scan + .with_parquet_file_reader_factory(Arc::new(InMemoryParquetFileReaderFactory( + Arc::clone(&in_memory_object_store), + ))); + + let parquet_exec = DataSourceExec::from_data_source(source); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); diff --git a/datafusion/core/tests/parquet/external_access_plan.rs b/datafusion/core/tests/parquet/external_access_plan.rs index a5397c5a397c..3d1f97ef892a 100644 --- a/datafusion/core/tests/parquet/external_access_plan.rs +++ b/datafusion/core/tests/parquet/external_access_plan.rs @@ -36,6 +36,7 @@ use datafusion_expr::{col, lit, Expr}; use datafusion_physical_plan::metrics::MetricsSet; use datafusion_physical_plan::ExecutionPlan; +use datafusion_common::config::TableParquetOptions; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_datasource::source::DataSourceExec; use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; @@ -342,19 +343,27 @@ impl TestFull { // Create a DataSourceExec to read the file let object_store_url = ObjectStoreUrl::local_filesystem(); + + let config = FileScanConfigBuilder::new(object_store_url, schema.clone()) + .with_file(partitioned_file) + .build(); + // add the predicate, if requested let source = if let Some(predicate) = predicate { let df_schema = DFSchema::try_from(schema.clone())?; let predicate = ctx.create_physical_expr(predicate, &df_schema)?; - Arc::new(ParquetSource::default().with_predicate(predicate)) + Arc::new( + ParquetSource::new(TableParquetOptions::default(), config.clone()) + .with_predicate(predicate), + ) } else { - Arc::new(ParquetSource::default()) + Arc::new(ParquetSource::new( + TableParquetOptions::default(), + config.clone(), + )) }; - let config = FileScanConfigBuilder::new(object_store_url, schema.clone(), source) - .with_file(partitioned_file) - .build(); - let plan: Arc = DataSourceExec::from_data_source(config); + let plan: Arc = Arc::new(DataSourceExec::new(source)); // run the DataSourceExec and collect the results let results = diff --git a/datafusion/core/tests/parquet/file_statistics.rs b/datafusion/core/tests/parquet/file_statistics.rs index 64ee92eda254..b95b46b93b9f 100644 --- a/datafusion/core/tests/parquet/file_statistics.rs +++ b/datafusion/core/tests/parquet/file_statistics.rs @@ -37,8 +37,9 @@ use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnvBuilder; use datafusion_expr::{col, lit, Expr}; -use datafusion::datasource::physical_plan::FileScanConfig; use datafusion_common::config::ConfigOptions; +use datafusion_datasource::file::FileSource; +use datafusion_datasource_parquet::source::ParquetSource; use datafusion_physical_optimizer::filter_pushdown::FilterPushdown; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::filter::FilterExec; @@ -200,11 +201,11 @@ async fn list_files_with_session_level_cache() { let data_source = data_source_exec.data_source(); let parquet1 = data_source .as_any() - .downcast_ref::() + .downcast_ref::() .unwrap(); assert_eq!(get_list_file_cache_size(&state1), 1); - let fg = &parquet1.file_groups; + let fg = &parquet1.config().file_groups; assert_eq!(fg.len(), 1); assert_eq!(fg.first().unwrap().len(), 1); @@ -216,11 +217,11 @@ async fn list_files_with_session_level_cache() { let data_source = data_source_exec.data_source(); let parquet2 = data_source .as_any() - .downcast_ref::() + .downcast_ref::() .unwrap(); assert_eq!(get_list_file_cache_size(&state2), 1); - let fg2 = &parquet2.file_groups; + let fg2 = &parquet2.config().file_groups; assert_eq!(fg2.len(), 1); assert_eq!(fg2.first().unwrap().len(), 1); @@ -232,11 +233,11 @@ async fn list_files_with_session_level_cache() { let data_source = data_source_exec.data_source(); let parquet3 = data_source .as_any() - .downcast_ref::() + .downcast_ref::() .unwrap(); assert_eq!(get_list_file_cache_size(&state1), 1); - let fg = &parquet3.file_groups; + let fg = &parquet3.config().file_groups; assert_eq!(fg.len(), 1); assert_eq!(fg.first().unwrap().len(), 1); // List same file no increase diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 27bee10234b5..8489621e8939 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -15,10 +15,9 @@ // specific language governing permissions and limitations // under the License. -use std::sync::Arc; - use crate::parquet::Unit::Page; use crate::parquet::{ContextWithParquet, Scenario}; +use std::sync::Arc; use arrow::array::RecordBatch; use datafusion::datasource::file_format::parquet::ParquetFormat; @@ -36,6 +35,7 @@ use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::{col, lit, Expr}; use datafusion_physical_expr::create_physical_expr; +use datafusion_common::config::TableParquetOptions; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use futures::StreamExt; use object_store::path::Path; @@ -80,17 +80,15 @@ async fn get_parquet_exec( let execution_props = ExecutionProps::new(); let predicate = create_physical_expr(&filter, &df_schema, &execution_props).unwrap(); - let source = Arc::new( - ParquetSource::default() - .with_predicate(predicate) - .with_enable_page_index(true) - .with_pushdown_filters(pushdown_filters), - ); - let base_config = FileScanConfigBuilder::new(object_store_url, schema, source) + let base_config = FileScanConfigBuilder::new(object_store_url, schema) .with_file(partitioned_file) .build(); + let source = ParquetSource::new(TableParquetOptions::default(), base_config.clone()) + .with_predicate(predicate) + .with_enable_page_index(true) + .with_pushdown_filters(pushdown_filters); - DataSourceExec::new(Arc::new(base_config)) + DataSourceExec::new(Arc::new(source)) } async fn get_filter_results( diff --git a/datafusion/core/tests/parquet/schema_coercion.rs b/datafusion/core/tests/parquet/schema_coercion.rs index 59cbf4b0872e..4754e1dd9800 100644 --- a/datafusion/core/tests/parquet/schema_coercion.rs +++ b/datafusion/core/tests/parquet/schema_coercion.rs @@ -30,6 +30,7 @@ use datafusion_common::test_util::batches_to_sort_string; use datafusion_common::Result; use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion_common::config::TableParquetOptions; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_datasource::source::DataSourceExec; use insta::assert_snapshot; @@ -62,16 +63,13 @@ async fn multi_parquet_coercion() { Field::new("c2", DataType::Int32, true), Field::new("c3", DataType::Float64, true), ])); - let source = Arc::new(ParquetSource::default()); - let conf = FileScanConfigBuilder::new( - ObjectStoreUrl::local_filesystem(), - file_schema, - source, - ) - .with_file_group(file_group) - .build(); + let conf = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_schema) + .with_file_group(file_group) + .build(); - let parquet_exec = DataSourceExec::from_data_source(conf); + let source = ParquetSource::new(TableParquetOptions::default(), conf.clone()); + let parquet_exec = DataSourceExec::from_data_source(source); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); @@ -120,16 +118,15 @@ async fn multi_parquet_coercion_projection() { Field::new("c2", DataType::Int32, true), Field::new("c3", DataType::Float64, true), ])); - let config = FileScanConfigBuilder::new( - ObjectStoreUrl::local_filesystem(), - file_schema, - Arc::new(ParquetSource::default()), - ) - .with_file_group(file_group) - .with_projection(Some(vec![1, 0, 2])) - .build(); - - let parquet_exec = DataSourceExec::from_data_source(config); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_schema) + .with_file_group(file_group) + .with_projection(Some(vec![1, 0, 2])) + .build(); + + let source = ParquetSource::new(TableParquetOptions::default(), config.clone()); + + let parquet_exec = DataSourceExec::from_data_source(source); let session_ctx = SessionContext::new(); let task_ctx = session_ctx.task_ctx(); diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index fd847763124a..1bd7f42ad38b 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -37,6 +37,7 @@ use datafusion::datasource::physical_plan::{CsvSource, ParquetSource}; use datafusion::datasource::source::DataSourceExec; use datafusion::datasource::MemTable; use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion_common::config::TableParquetOptions; use datafusion_common::error::Result; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::ScalarValue; @@ -184,19 +185,17 @@ fn parquet_exec_multiple() -> Arc { fn parquet_exec_multiple_sorted( output_ordering: Vec, ) -> Arc { - let config = FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - Arc::new(ParquetSource::default()), - ) - .with_file_groups(vec![ - FileGroup::new(vec![PartitionedFile::new("x".to_string(), 100)]), - FileGroup::new(vec![PartitionedFile::new("y".to_string(), 100)]), - ]) - .with_output_ordering(output_ordering) - .build(); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file_groups(vec![ + FileGroup::new(vec![PartitionedFile::new("x".to_string(), 100)]), + FileGroup::new(vec![PartitionedFile::new("y".to_string(), 100)]), + ]) + .with_output_ordering(output_ordering) + .build(); - DataSourceExec::from_data_source(config) + let source = ParquetSource::new(TableParquetOptions::default(), config.clone()); + DataSourceExec::from_data_source(source) } fn csv_exec() -> Arc { @@ -204,16 +203,14 @@ fn csv_exec() -> Arc { } fn csv_exec_with_sort(output_ordering: Vec) -> Arc { - let config = FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - Arc::new(CsvSource::new(false, b',', b'"')), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering) - .build(); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(output_ordering) + .build(); + let source = CsvSource::new(false, b',', b'"', config.clone()); - DataSourceExec::from_data_source(config) + DataSourceExec::from_data_source(source) } fn csv_exec_multiple() -> Arc { @@ -222,19 +219,18 @@ fn csv_exec_multiple() -> Arc { // Created a sorted parquet exec with multiple files fn csv_exec_multiple_sorted(output_ordering: Vec) -> Arc { - let config = FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - Arc::new(CsvSource::new(false, b',', b'"')), - ) - .with_file_groups(vec![ - FileGroup::new(vec![PartitionedFile::new("x".to_string(), 100)]), - FileGroup::new(vec![PartitionedFile::new("y".to_string(), 100)]), - ]) - .with_output_ordering(output_ordering) - .build(); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file_groups(vec![ + FileGroup::new(vec![PartitionedFile::new("x".to_string(), 100)]), + FileGroup::new(vec![PartitionedFile::new("y".to_string(), 100)]), + ]) + .with_output_ordering(output_ordering) + .build(); + + let source = CsvSource::new(false, b',', b'"', config.clone()); - DataSourceExec::from_data_source(config) + DataSourceExec::from_data_source(source) } fn projection_exec_with_alias( @@ -2534,17 +2530,17 @@ fn parallelization_compressed_csv() -> Result<()> { &expected_partitioned[..] }; + let config = FileScanConfigBuilder::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_file_compression_type(compression_type) + .build(); + + let source = CsvSource::new(false, b',', b'"', config.clone()); let plan = aggregate_exec_with_alias( - DataSourceExec::from_data_source( - FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - Arc::new(CsvSource::new(false, b',', b'"')), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_file_compression_type(compression_type) - .build(), - ), + DataSourceExec::from_data_source(source), vec![("a".to_string(), "a".to_string())], ); let test_config = TestConfig::default() diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index d10459ce86ae..22924edba80b 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -29,45 +29,45 @@ use crate::physical_optimizer::test_utils::{ union_exec, RequirementsTestExec, }; +use arrow::array::{Int32Array, RecordBatch}; use arrow::compute::SortOptions; +use arrow::datatypes::Field; use arrow::datatypes::{DataType, SchemaRef}; +use arrow_schema::Schema; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::physical_plan::CsvSource; +use datafusion::prelude::*; +use datafusion_catalog::streaming::StreamingTable; use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{TreeNode, TransformedResult}; +use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_common::{Result, ScalarValue, TableReference}; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_datasource::source::DataSourceExec; -use datafusion_expr_common::operator::Operator; -use datafusion_expr::{JoinType, SortExpr, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition}; use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion_execution::TaskContext; +use datafusion_expr::{JoinType, SortExpr, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition}; +use datafusion_expr_common::operator::Operator; use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::min_max::{max_udaf, min_udaf}; +use datafusion_physical_expr::expressions::{col, BinaryExpr, Column, NotExpr}; +use datafusion_physical_expr::{Distribution, Partitioning}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{ - LexOrdering, PhysicalSortExpr, PhysicalSortRequirement, OrderingRequirements + LexOrdering, OrderingRequirements, PhysicalSortExpr, PhysicalSortRequirement }; -use datafusion_physical_expr::{Distribution, Partitioning}; -use datafusion_physical_expr::expressions::{col, BinaryExpr, Column, NotExpr}; +use datafusion_physical_optimizer::enforce_distribution::EnforceDistribution; +use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; +use datafusion_physical_optimizer::enforce_sorting::sort_pushdown::{assign_initial_requirements, pushdown_sorts, SortPushDown}; +use datafusion_physical_optimizer::enforce_sorting::{ensure_sorting, parallelize_sorts, EnforceSorting, PlanWithCorrespondingCoalescePartitions, PlanWithCorrespondingSort}; +use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; +use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::windows::{create_window_expr, BoundedWindowAggExec, WindowAggExec}; use datafusion_physical_plan::{displayable, get_plan_string, ExecutionPlan, InputOrderMode}; -use datafusion::datasource::physical_plan::CsvSource; -use datafusion::datasource::listing::PartitionedFile; -use datafusion_physical_optimizer::enforce_sorting::{EnforceSorting, PlanWithCorrespondingCoalescePartitions, PlanWithCorrespondingSort, parallelize_sorts, ensure_sorting}; -use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; -use datafusion_physical_optimizer::enforce_sorting::sort_pushdown::{SortPushDown, assign_initial_requirements, pushdown_sorts}; -use datafusion_physical_optimizer::enforce_distribution::EnforceDistribution; -use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; -use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion::prelude::*; -use arrow::array::{Int32Array, RecordBatch}; -use arrow::datatypes::{Field}; -use arrow_schema::Schema; -use datafusion_execution::TaskContext; -use datafusion_catalog::streaming::StreamingTable; use futures::StreamExt; use rstest::rstest; @@ -80,7 +80,6 @@ fn csv_exec_sorted( let mut builder = FileScanConfigBuilder::new( ObjectStoreUrl::parse("test:///").unwrap(), schema.clone(), - Arc::new(CsvSource::new(false, 0, 0)), ) .with_file(PartitionedFile::new("x".to_string(), 100)); if let Some(ordering) = LexOrdering::new(sort_exprs) { @@ -88,7 +87,7 @@ fn csv_exec_sorted( } let config = builder.build(); - DataSourceExec::from_data_source(config) + DataSourceExec::from_data_source(CsvSource::new(false, 0, 0, config.clone())) } /// Runs the sort enforcement optimizer and asserts the plan diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index acb2b808ef8f..1f0c1f6ce96d 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -19,7 +19,8 @@ use arrow::datatypes::SchemaRef; use arrow::error::ArrowError; use arrow::{array::RecordBatch, compute::concat_batches}; use datafusion::{datasource::object_store::ObjectStoreUrl, physical_plan::PhysicalExpr}; -use datafusion_common::{config::ConfigOptions, internal_err, Result, Statistics}; +use datafusion_common::{config::ConfigOptions, internal_err, Result}; +use datafusion_datasource::source::DataSource; use datafusion_datasource::{ file::FileSource, file_meta::FileMeta, file_scan_config::FileScanConfig, file_scan_config::FileScanConfigBuilder, file_stream::FileOpenFuture, @@ -49,6 +50,7 @@ use std::{ sync::Arc, task::{Context, Poll}, }; + pub struct TestOpener { batches: Vec, batch_size: Option, @@ -103,42 +105,56 @@ impl FileOpener for TestOpener { } /// A placeholder data source that accepts filter pushdown -#[derive(Clone, Default)] +#[derive(Clone)] pub struct TestSource { support: bool, predicate: Option>, - statistics: Option, - batch_size: Option, batches: Vec, - schema: Option, metrics: ExecutionPlanMetricsSet, - projection: Option>, schema_adapter_factory: Option>, + + config: FileScanConfig, } impl TestSource { - fn new(support: bool, batches: Vec) -> Self { + fn new(support: bool, batches: Vec, config: FileScanConfig) -> Self { Self { support, metrics: ExecutionPlanMetricsSet::new(), batches, - ..Default::default() + schema_adapter_factory: None, + predicate: None, + config, } } } impl FileSource for TestSource { + fn config(&self) -> &FileScanConfig { + &self.config + } + + fn with_config(&self, config: FileScanConfig) -> Arc { + let mut this = self.clone(); + this.config = config; + + Arc::new(this) + } + + fn as_data_source(&self) -> Arc { + Arc::new(self.clone()) + } + fn create_file_opener( &self, _object_store: Arc, - _base_config: &FileScanConfig, _partition: usize, ) -> Arc { Arc::new(TestOpener { batches: self.batches.clone(), - batch_size: self.batch_size, - schema: self.schema.clone(), - projection: self.projection.clone(), + batch_size: self.config().batch_size, + schema: Some(Arc::clone(&self.config().file_schema)), + projection: self.config().projection.clone(), }) } @@ -146,46 +162,10 @@ impl FileSource for TestSource { todo!("should not be called") } - fn with_batch_size(&self, batch_size: usize) -> Arc { - Arc::new(TestSource { - batch_size: Some(batch_size), - ..self.clone() - }) - } - - fn with_schema(&self, schema: SchemaRef) -> Arc { - Arc::new(TestSource { - schema: Some(schema), - ..self.clone() - }) - } - - fn with_projection(&self, config: &FileScanConfig) -> Arc { - Arc::new(TestSource { - projection: config.projection.clone(), - ..self.clone() - }) - } - - fn with_statistics(&self, statistics: Statistics) -> Arc { - Arc::new(TestSource { - statistics: Some(statistics), - ..self.clone() - }) - } - - fn metrics(&self) -> &ExecutionPlanMetricsSet { + fn metrics_inner(&self) -> &ExecutionPlanMetricsSet { &self.metrics } - fn statistics(&self) -> Result { - Ok(self - .statistics - .as_ref() - .expect("statistics not set") - .clone()) - } - fn file_type(&self) -> &str { "test" } @@ -254,6 +234,15 @@ impl FileSource for TestSource { } } +impl std::fmt::Debug for TestSource { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "{} {{ ", self.file_type())?; + write!(f, "statistics={:?}, ", self.file_source_statistics())?; + write!(f, "config={:?}, ", self.config())?; + write!(f, " }}") + } +} + #[derive(Debug, Clone)] pub struct TestScanBuilder { support: bool, @@ -281,15 +270,18 @@ impl TestScanBuilder { } pub fn build(self) -> Arc { - let source = Arc::new(TestSource::new(self.support, self.batches)); let base_config = FileScanConfigBuilder::new( ObjectStoreUrl::parse("test://").unwrap(), Arc::clone(&self.schema), - source, ) .with_file(PartitionedFile::new("test.parquet", 123)) .build(); - DataSourceExec::from_data_source(base_config) + + DataSourceExec::from_data_source(TestSource::new( + self.support, + self.batches, + base_config.clone(), + )) } } diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index 149c50557c3a..ce84ef3bf8f2 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -376,16 +376,13 @@ fn create_simple_csv_exec() -> Arc { Field::new("d", DataType::Int32, true), Field::new("e", DataType::Int32, true), ])); - let config = FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema, - Arc::new(CsvSource::new(false, 0, 0)), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_projection(Some(vec![0, 1, 2, 3, 4])) - .build(); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_projection(Some(vec![0, 1, 2, 3, 4])) + .build(); - DataSourceExec::from_data_source(config) + DataSourceExec::from_data_source(CsvSource::new(false, 0, 0, config.clone())) } fn create_projecting_csv_exec() -> Arc { @@ -395,16 +392,13 @@ fn create_projecting_csv_exec() -> Arc { Field::new("c", DataType::Int32, true), Field::new("d", DataType::Int32, true), ])); - let config = FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema, - Arc::new(CsvSource::new(false, 0, 0)), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_projection(Some(vec![3, 2, 1])) - .build(); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_projection(Some(vec![3, 2, 1])) + .build(); - DataSourceExec::from_data_source(config) + DataSourceExec::from_data_source(CsvSource::new(false, 0, 0, config.clone())) } fn create_projecting_memory_exec() -> Arc { @@ -1572,14 +1566,13 @@ fn partitioned_data_source() -> Arc { let config = FileScanConfigBuilder::new( ObjectStoreUrl::parse("test:///").unwrap(), file_schema.clone(), - Arc::new(CsvSource::default()), ) .with_file(PartitionedFile::new("x".to_string(), 100)) .with_table_partition_cols(vec![Field::new("partition_col", DataType::Utf8, true)]) .with_projection(Some(vec![0, 1, 2])) .build(); - DataSourceExec::from_data_source(config) + DataSourceExec::from_data_source(CsvSource::new(false, 0, 0, config.clone())) } #[test] diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 7f7926060edc..9e2d0003f7fa 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -29,11 +29,12 @@ use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::physical_plan::ParquetSource; use datafusion::datasource::source::DataSourceExec; -use datafusion_common::config::ConfigOptions; +use datafusion_common::config::{ConfigOptions, TableParquetOptions}; use datafusion_common::stats::Precision; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; use datafusion_common::{ColumnStatistics, JoinType, NullEquality, Result, Statistics}; +use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; @@ -71,15 +72,14 @@ use datafusion_physical_plan::{ /// Create a non sorted parquet exec pub fn parquet_exec(schema: SchemaRef) -> Arc { - let config = FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema, - Arc::new(ParquetSource::default()), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .build(); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .build(); + + let source = ParquetSource::new(TableParquetOptions::default(), config.clone()); - DataSourceExec::from_data_source(config) + DataSourceExec::from_data_source(source) } /// Create a single parquet file that is sorted @@ -87,16 +87,14 @@ pub(crate) fn parquet_exec_with_sort( schema: SchemaRef, output_ordering: Vec, ) -> Arc { - let config = FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema, - Arc::new(ParquetSource::default()), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering) - .build(); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::parse("test:///").unwrap(), schema) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(output_ordering) + .build(); - DataSourceExec::from_data_source(config) + let source = ParquetSource::new(TableParquetOptions::default(), config.clone()); + DataSourceExec::from_data_source(source) } fn int64_stats() -> ColumnStatistics { @@ -125,20 +123,20 @@ pub(crate) fn parquet_exec_with_stats(file_size: u64) -> Arc { statistics.num_rows = Precision::Inexact(10000); statistics.column_statistics = column_stats(); - let config = FileScanConfigBuilder::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - Arc::new(ParquetSource::new(Default::default())), - ) - .with_file(PartitionedFile::new("x".to_string(), file_size)) - .with_statistics(statistics) - .build(); + let config = + FileScanConfigBuilder::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file(PartitionedFile::new("x".to_string(), file_size)) + .with_file_source_projected_statistics(statistics) + .build(); + + let source = ParquetSource::new(Default::default(), config.clone()); assert_eq!( - config.file_source.statistics().unwrap().num_rows, + source.file_source_statistics().num_rows, Precision::Inexact(10000) ); - DataSourceExec::from_data_source(config) + + DataSourceExec::from_data_source(source) } pub fn schema() -> SchemaRef { diff --git a/datafusion/core/tests/schema_adapter/schema_adapter_integration_tests.rs b/datafusion/core/tests/schema_adapter/schema_adapter_integration_tests.rs index c3c92a9028d6..a7bc542c33ad 100644 --- a/datafusion/core/tests/schema_adapter/schema_adapter_integration_tests.rs +++ b/datafusion/core/tests/schema_adapter/schema_adapter_integration_tests.rs @@ -27,6 +27,7 @@ use datafusion::datasource::physical_plan::{ }; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; +use datafusion_common::config::TableParquetOptions; use datafusion_common::ColumnStatistics; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_datasource::schema_adapter::{ @@ -182,22 +183,22 @@ async fn test_parquet_integration_with_schema_adapter() -> Result<()> { let ctx = SessionContext::new(); ctx.register_object_store(store_url.as_ref(), Arc::clone(&store)); - // Create a ParquetSource with the adapter factory - let file_source = ParquetSource::default() - .with_schema_adapter_factory(Arc::new(UppercaseAdapterFactory {}))?; - // Create a table schema with uppercase column names let table_schema = Arc::new(Schema::new(vec![ Field::new("ID", DataType::Int32, false), Field::new("NAME", DataType::Utf8, true), ])); - let config = FileScanConfigBuilder::new(store_url, table_schema.clone(), file_source) + let config = FileScanConfigBuilder::new(store_url, table_schema.clone()) .with_file(PartitionedFile::new(path, file_size)) .build(); + // Create a ParquetSource with the adapter factory + let file_source = ParquetSource::new(TableParquetOptions::default(), config.clone()) + .with_schema_adapter_factory(Arc::new(UppercaseAdapterFactory {}))?; + // Create a data source executor - let exec = DataSourceExec::from_data_source(config); + let exec = Arc::new(DataSourceExec::new(file_source.as_data_source())); // Collect results let task_ctx = ctx.task_ctx(); @@ -244,16 +245,16 @@ async fn test_parquet_integration_with_schema_adapter_and_expression_rewriter( let ctx = SessionContext::new(); ctx.register_object_store(store_url.as_ref(), Arc::clone(&store)); - // Create a ParquetSource with the adapter factory - let file_source = ParquetSource::default() - .with_schema_adapter_factory(Arc::new(UppercaseAdapterFactory {}))?; - - let config = FileScanConfigBuilder::new(store_url, batch.schema(), file_source) + let config = FileScanConfigBuilder::new(store_url, batch.schema()) .with_file(PartitionedFile::new(path, file_size)) .build(); + // Create a ParquetSource with the adapter factory + let file_source = ParquetSource::new(TableParquetOptions::default(), config.clone()) + .with_schema_adapter_factory(Arc::new(UppercaseAdapterFactory {}))?; + // Create a data source executor - let exec = DataSourceExec::from_data_source(config); + let exec = Arc::new(DataSourceExec::new(file_source.as_data_source())); // Collect results let task_ctx = ctx.task_ctx(); @@ -279,12 +280,18 @@ async fn test_multi_source_schema_adapter_reuse() -> Result<()> { // 2. The factory can be shared and cloned efficiently using Arc // 3. Various data source implementations correctly implement the schema adapter factory pattern + let config = FileScanConfigBuilder::new( + ObjectStoreUrl::local_filesystem(), + Arc::new(Schema::empty()), + ) + .build(); + // Create a test factory let factory = Arc::new(UppercaseAdapterFactory {}); // Test ArrowSource { - let source = ArrowSource::default(); + let source = ArrowSource::new(config.clone()); let source_with_adapter = source .clone() .with_schema_adapter_factory(factory.clone()) @@ -304,7 +311,7 @@ async fn test_multi_source_schema_adapter_reuse() -> Result<()> { // Test ParquetSource #[cfg(feature = "parquet")] { - let source = ParquetSource::default(); + let source = ParquetSource::new(TableParquetOptions::default(), config.clone()); let source_with_adapter = source .clone() .with_schema_adapter_factory(factory.clone()) @@ -323,7 +330,7 @@ async fn test_multi_source_schema_adapter_reuse() -> Result<()> { // Test CsvSource { - let source = CsvSource::default(); + let source = CsvSource::new(false, 0, 0, config.clone()); let source_with_adapter = source .clone() .with_schema_adapter_factory(factory.clone()) @@ -342,7 +349,7 @@ async fn test_multi_source_schema_adapter_reuse() -> Result<()> { // Test JsonSource { - let source = JsonSource::default(); + let source = JsonSource::new(config.clone()); let source_with_adapter = source .clone() .with_schema_adapter_factory(factory.clone()) diff --git a/datafusion/datasource-avro/src/file_format.rs b/datafusion/datasource-avro/src/file_format.rs index 60c361b42e77..3cf1b3aa4e51 100644 --- a/datafusion/datasource-avro/src/file_format.rs +++ b/datafusion/datasource-avro/src/file_format.rs @@ -35,9 +35,7 @@ use datafusion_common::{Result, Statistics}; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_compression_type::FileCompressionType; use datafusion_datasource::file_format::{FileFormat, FileFormatFactory}; -use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; -use datafusion_datasource::source::DataSourceExec; -use datafusion_physical_plan::ExecutionPlan; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_session::Session; use async_trait::async_trait; @@ -149,18 +147,7 @@ impl FileFormat for AvroFormat { Ok(Statistics::new_unknown(&table_schema)) } - async fn create_physical_plan( - &self, - _state: &dyn Session, - conf: FileScanConfig, - ) -> Result> { - let config = FileScanConfigBuilder::from(conf) - .with_source(self.file_source()) - .build(); - Ok(DataSourceExec::from_data_source(config)) - } - - fn file_source(&self) -> Arc { - Arc::new(AvroSource::new()) + fn file_source(&self, config: FileScanConfig) -> Arc { + Arc::new(AvroSource::new(config)) } } diff --git a/datafusion/datasource-avro/src/source.rs b/datafusion/datasource-avro/src/source.rs index 948049f5a747..33981bce775c 100644 --- a/datafusion/datasource-avro/src/source.rs +++ b/datafusion/datasource-avro/src/source.rs @@ -24,7 +24,6 @@ use crate::avro_to_arrow::Reader as AvroReader; use arrow::datatypes::SchemaRef; use datafusion_common::error::Result; -use datafusion_common::Statistics; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::file_stream::FileOpener; @@ -32,44 +31,67 @@ use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion_datasource::source::DataSource; use object_store::ObjectStore; /// AvroSource holds the extra configuration that is necessary for opening avro files -#[derive(Clone, Default)] +#[derive(Clone)] pub struct AvroSource { - schema: Option, - batch_size: Option, - projection: Option>, metrics: ExecutionPlanMetricsSet, - projected_statistics: Option, schema_adapter_factory: Option>, + + config: FileScanConfig, } impl AvroSource { /// Initialize an AvroSource with default values - pub fn new() -> Self { - Self::default() + pub fn new(config: FileScanConfig) -> Self { + Self { + metrics: Default::default(), + schema_adapter_factory: None, + config, + } } - fn open(&self, reader: R) -> Result> { + fn open( + &self, + reader: R, + file_schema: SchemaRef, + batch_size: Option, + projected_file_column_names: Option>, + ) -> Result> { AvroReader::try_new( reader, - Arc::clone(self.schema.as_ref().expect("Schema must set before open")), - self.batch_size.expect("Batch size must set before open"), - self.projection.clone(), + file_schema, + batch_size.expect("Batch size must set before open"), + projected_file_column_names, ) } } impl FileSource for AvroSource { + fn config(&self) -> &FileScanConfig { + &self.config + } + + fn with_config(&self, config: FileScanConfig) -> Arc { + let mut this = self.clone(); + this.config = config; + + Arc::new(this) + } + + fn as_data_source(&self) -> Arc { + Arc::new(self.clone()) + } + fn create_file_opener( &self, object_store: Arc, - _base_config: &FileScanConfig, _partition: usize, ) -> Arc { Arc::new(private::AvroOpener { - config: Arc::new(self.clone()), + source: Arc::new(self.clone()), object_store, }) } @@ -78,51 +100,20 @@ impl FileSource for AvroSource { self } - fn with_batch_size(&self, batch_size: usize) -> Arc { - let mut conf = self.clone(); - conf.batch_size = Some(batch_size); - Arc::new(conf) - } - - fn with_schema(&self, schema: SchemaRef) -> Arc { - let mut conf = self.clone(); - conf.schema = Some(schema); - Arc::new(conf) - } - fn with_statistics(&self, statistics: Statistics) -> Arc { - let mut conf = self.clone(); - conf.projected_statistics = Some(statistics); - Arc::new(conf) - } - - fn with_projection(&self, config: &FileScanConfig) -> Arc { - let mut conf = self.clone(); - conf.projection = config.projected_file_column_names(); - Arc::new(conf) - } - - fn metrics(&self) -> &ExecutionPlanMetricsSet { + fn metrics_inner(&self) -> &ExecutionPlanMetricsSet { &self.metrics } - fn statistics(&self) -> Result { - let statistics = &self.projected_statistics; - Ok(statistics - .clone() - .expect("projected_statistics must be set")) - } - fn file_type(&self) -> &str { "avro" } - fn repartitioned( + fn repartitioned_inner( &self, _target_partitions: usize, _repartition_file_min_size: usize, _output_ordering: Option, - _config: &FileScanConfig, - ) -> Result> { + ) -> Result>> { Ok(None) } @@ -141,6 +132,15 @@ impl FileSource for AvroSource { } } +impl std::fmt::Debug for AvroSource { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{} {{ ", self.file_type())?; + write!(f, "statistics={:?}, ", self.file_source_statistics())?; + write!(f, "config={:?}, ", self.config())?; + write!(f, " }}") + } +} + mod private { use super::*; @@ -152,7 +152,7 @@ mod private { use object_store::{GetResultPayload, ObjectStore}; pub struct AvroOpener { - pub config: Arc, + pub source: Arc, pub object_store: Arc, } @@ -162,18 +162,33 @@ mod private { file_meta: FileMeta, _file: PartitionedFile, ) -> Result { - let config = Arc::clone(&self.config); + let source = Arc::clone(&self.source); let object_store = Arc::clone(&self.object_store); + let file_schema = Arc::clone(&self.source.config().file_schema); + let batch_size = self.source.config().batch_size; + let projected_file_names = + self.source.config().projected_file_column_names().clone(); + Ok(Box::pin(async move { let r = object_store.get(file_meta.location()).await?; match r.payload { GetResultPayload::File(file, _) => { - let reader = config.open(file)?; + let reader = source.open( + file, + file_schema, + batch_size, + projected_file_names, + )?; Ok(futures::stream::iter(reader).boxed()) } GetResultPayload::Stream(_) => { let bytes = r.bytes().await?; - let reader = config.open(bytes.reader())?; + let reader = source.open( + bytes.reader(), + file_schema, + batch_size, + projected_file_names, + )?; Ok(futures::stream::iter(reader).boxed()) } } diff --git a/datafusion/datasource-csv/src/file_format.rs b/datafusion/datasource-csv/src/file_format.rs index 4eeb431584ba..23de68556b6c 100644 --- a/datafusion/datasource-csv/src/file_format.rs +++ b/datafusion/datasource-csv/src/file_format.rs @@ -410,7 +410,7 @@ impl FileFormat for CsvFormat { async fn create_physical_plan( &self, state: &dyn Session, - conf: FileScanConfig, + source: Arc, ) -> Result> { // Consult configuration options for default values let has_header = self @@ -422,20 +422,23 @@ impl FileFormat for CsvFormat { .newlines_in_values .unwrap_or_else(|| state.config_options().catalog.newlines_in_values); - let conf_builder = FileScanConfigBuilder::from(conf) + let conf_builder = FileScanConfigBuilder::from(source.config().to_owned()) .with_file_compression_type(self.options.compression.into()) .with_newlines_in_values(newlines_in_values); - let source = Arc::new( - CsvSource::new(has_header, self.options.delimiter, self.options.quote) - .with_escape(self.options.escape) - .with_terminator(self.options.terminator) - .with_comment(self.options.comment), - ); + let config = conf_builder.build(); - let config = conf_builder.with_source(source).build(); + let source = CsvSource::new( + has_header, + self.options.delimiter, + self.options.quote, + config.clone(), + ) + .with_escape(self.options.escape) + .with_terminator(self.options.terminator) + .with_comment(self.options.comment); - Ok(DataSourceExec::from_data_source(config)) + Ok(DataSourceExec::from_data_source(source)) } async fn create_writer_physical_plan( @@ -475,8 +478,13 @@ impl FileFormat for CsvFormat { Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) } - fn file_source(&self) -> Arc { - Arc::new(CsvSource::default()) + fn file_source(&self, config: FileScanConfig) -> Arc { + Arc::new(CsvSource::new( + self.has_header().unwrap_or(false), + self.delimiter(), + self.quote(), + config, + )) } } diff --git a/datafusion/datasource-csv/src/mod.rs b/datafusion/datasource-csv/src/mod.rs index 90538d0808b1..c942af4a0e15 100644 --- a/datafusion/datasource-csv/src/mod.rs +++ b/datafusion/datasource-csv/src/mod.rs @@ -22,12 +22,10 @@ pub mod file_format; pub mod source; -use std::sync::Arc; - use arrow::datatypes::SchemaRef; use datafusion_datasource::file_groups::FileGroup; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; -use datafusion_datasource::{file::FileSource, file_scan_config::FileScanConfig}; use datafusion_execution::object_store::ObjectStoreUrl; pub use file_format::*; @@ -35,9 +33,8 @@ pub use file_format::*; pub fn partitioned_csv_config( schema: SchemaRef, file_groups: Vec, - file_source: Arc, ) -> FileScanConfig { - FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), schema, file_source) + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), schema) .with_file_groups(file_groups) .build() } diff --git a/datafusion/datasource-csv/src/source.rs b/datafusion/datasource-csv/src/source.rs index 6c994af940d1..ce5f57706a79 100644 --- a/datafusion/datasource-csv/src/source.rs +++ b/datafusion/datasource-csv/src/source.rs @@ -19,13 +19,11 @@ use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use std::any::Any; -use std::fmt; use std::io::{Read, Seek, SeekFrom}; use std::sync::Arc; use std::task::Poll; use datafusion_datasource::decoder::{deserialize_stream, DecoderDeserializer}; -use datafusion_datasource::file_compression_type::FileCompressionType; use datafusion_datasource::file_meta::FileMeta; use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; use datafusion_datasource::{ @@ -35,7 +33,7 @@ use datafusion_datasource::{ use arrow::csv; use arrow::datatypes::SchemaRef; -use datafusion_common::{DataFusionError, Result, Statistics}; +use datafusion_common::{DataFusionError, Result}; use datafusion_common_runtime::JoinSet; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; @@ -46,6 +44,7 @@ use datafusion_physical_plan::{ }; use crate::file_format::CsvDecoder; +use datafusion_datasource::source::DataSource; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; use object_store::{GetOptions, GetResultPayload, ObjectStore}; @@ -66,25 +65,24 @@ use tokio::io::AsyncWriteExt; /// # let object_store_url = ObjectStoreUrl::local_filesystem(); /// # let file_schema = Arc::new(Schema::empty()); /// -/// let source = Arc::new(CsvSource::new( +/// // Create a DataSourceExec for reading the first 100MB of `file1.csv` +/// let config = FileScanConfigBuilder::new(object_store_url, file_schema) +/// .with_file(PartitionedFile::new("file1.csv", 100*1024*1024)) +/// .with_newlines_in_values(true) // The file contains newlines in values; +/// .build(); +/// +/// let source = CsvSource::new( /// true, /// b',', /// b'"', +/// config /// ) /// .with_terminator(Some(b'#') -/// )); -/// // Create a DataSourceExec for reading the first 100MB of `file1.csv` -/// let config = FileScanConfigBuilder::new(object_store_url, file_schema, source) -/// .with_file(PartitionedFile::new("file1.csv", 100*1024*1024)) -/// .with_newlines_in_values(true) // The file contains newlines in values; -/// .build(); -/// let exec = (DataSourceExec::from_data_source(config)); +/// ); +/// let exec = (DataSourceExec::from_data_source(source)); /// ``` -#[derive(Debug, Clone, Default)] +#[derive(Clone)] pub struct CsvSource { - batch_size: Option, - file_schema: Option, - file_projection: Option>, pub(crate) has_header: bool, delimiter: u8, quote: u8, @@ -92,18 +90,29 @@ pub struct CsvSource { escape: Option, comment: Option, metrics: ExecutionPlanMetricsSet, - projected_statistics: Option, schema_adapter_factory: Option>, + + config: FileScanConfig, } impl CsvSource { /// Returns a [`CsvSource`] - pub fn new(has_header: bool, delimiter: u8, quote: u8) -> Self { + pub fn new( + has_header: bool, + delimiter: u8, + quote: u8, + config: FileScanConfig, + ) -> Self { Self { has_header, delimiter, quote, - ..Self::default() + terminator: None, + escape: None, + comment: None, + metrics: Default::default(), + schema_adapter_factory: None, + config, } } @@ -159,27 +168,35 @@ impl CsvSource { } impl CsvSource { - fn open(&self, reader: R) -> Result> { - Ok(self.builder().build(reader)?) + fn open( + &self, + reader: R, + file_schema: SchemaRef, + batch_size: Option, + file_column_projection_indices: Option>, + ) -> Result> { + Ok(self + .builder(file_schema, batch_size, file_column_projection_indices) + .build(reader)?) } - fn builder(&self) -> csv::ReaderBuilder { - let mut builder = csv::ReaderBuilder::new(Arc::clone( - self.file_schema - .as_ref() - .expect("Schema must be set before initializing builder"), - )) - .with_delimiter(self.delimiter) - .with_batch_size( - self.batch_size - .expect("Batch size must be set before initializing builder"), - ) - .with_header(self.has_header) - .with_quote(self.quote); + fn builder( + &self, + file_schema: SchemaRef, + batch_size: Option, + file_column_projection_indices: Option>, + ) -> csv::ReaderBuilder { + let mut builder = csv::ReaderBuilder::new(Arc::clone(&file_schema)) + .with_delimiter(self.delimiter) + .with_batch_size( + batch_size.expect("Batch size must be set before initializing builder"), + ) + .with_header(self.has_header) + .with_quote(self.quote); if let Some(terminator) = self.terminator { builder = builder.with_terminator(terminator); } - if let Some(proj) = &self.file_projection { + if let Some(proj) = &file_column_projection_indices { builder = builder.with_projection(proj.clone()); } if let Some(escape) = self.escape { @@ -195,21 +212,15 @@ impl CsvSource { /// A [`FileOpener`] that opens a CSV file and yields a [`FileOpenFuture`] pub struct CsvOpener { - config: Arc, - file_compression_type: FileCompressionType, + source: Arc, object_store: Arc, } impl CsvOpener { /// Returns a [`CsvOpener`] - pub fn new( - config: Arc, - file_compression_type: FileCompressionType, - object_store: Arc, - ) -> Self { + pub fn new(source: Arc, object_store: Arc) -> Self { Self { - config, - file_compression_type, + source, object_store, } } @@ -222,15 +233,28 @@ impl From for Arc { } impl FileSource for CsvSource { + fn config(&self) -> &FileScanConfig { + &self.config + } + + fn with_config(&self, config: FileScanConfig) -> Arc { + let mut this = self.clone(); + this.config = config; + + Arc::new(this) + } + + fn as_data_source(&self) -> Arc { + Arc::new(self.clone()) + } + fn create_file_opener( &self, object_store: Arc, - base_config: &FileScanConfig, _partition: usize, ) -> Arc { Arc::new(CsvOpener { - config: Arc::new(self.clone()), - file_compression_type: base_config.file_compression_type, + source: Arc::new(self.clone()), object_store, }) } @@ -239,43 +263,18 @@ impl FileSource for CsvSource { self } - fn with_batch_size(&self, batch_size: usize) -> Arc { - let mut conf = self.clone(); - conf.batch_size = Some(batch_size); - Arc::new(conf) - } - - fn with_schema(&self, schema: SchemaRef) -> Arc { - let mut conf = self.clone(); - conf.file_schema = Some(schema); - Arc::new(conf) - } - - fn with_statistics(&self, statistics: Statistics) -> Arc { - let mut conf = self.clone(); - conf.projected_statistics = Some(statistics); - Arc::new(conf) - } - - fn with_projection(&self, config: &FileScanConfig) -> Arc { - let mut conf = self.clone(); - conf.file_projection = config.file_column_projection_indices(); - Arc::new(conf) - } - - fn metrics(&self) -> &ExecutionPlanMetricsSet { + fn metrics_inner(&self) -> &ExecutionPlanMetricsSet { &self.metrics } - fn statistics(&self) -> Result { - let statistics = &self.projected_statistics; - Ok(statistics - .clone() - .expect("projected_statistics must be set")) - } + fn file_type(&self) -> &str { "csv" } - fn fmt_extra(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + fn fmt_extra( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { write!(f, ", has_header={}", self.has_header) @@ -299,6 +298,15 @@ impl FileSource for CsvSource { } } +impl std::fmt::Debug for CsvSource { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{} {{ ", self.file_type())?; + write!(f, "statistics={:?}, ", self.file_source_statistics())?; + write!(f, "config={:?}, ", self.config())?; + write!(f, " }}") + } +} + impl FileOpener for CsvOpener { /// Open a partitioned CSV file. /// @@ -331,19 +339,19 @@ impl FileOpener for CsvOpener { // `self.config.has_header` controls whether to skip reading the 1st line header // If the .csv file is read in parallel and this `CsvOpener` is only reading some middle // partition, then don't skip first line - let mut csv_has_header = self.config.has_header; + let mut csv_has_header = self.source.has_header; if let Some(FileRange { start, .. }) = file_meta.range { if start != 0 { csv_has_header = false; } } - let config = CsvSource { + let source = CsvSource { has_header: csv_has_header, - ..(*self.config).clone() + ..(*self.source).clone() }; - let file_compression_type = self.file_compression_type.to_owned(); + let file_compression_type = source.config.file_compression_type.to_owned(); if file_meta.range.is_some() { assert!( @@ -353,7 +361,12 @@ impl FileOpener for CsvOpener { } let store = Arc::clone(&self.object_store); - let terminator = self.config.terminator; + let terminator = self.source.terminator; + + let file_schema = Arc::clone(&source.config.file_schema); + let batch_size = source.config.batch_size; + let file_column_projection_indices = + source.config.file_column_projection_indices().clone(); Ok(Box::pin(async move { // Current partition contains bytes [start_byte, end_byte) (might contain incomplete lines at boundaries) @@ -392,10 +405,18 @@ impl FileOpener for CsvOpener { )? }; - Ok(futures::stream::iter(config.open(decoder)?).boxed()) + Ok(futures::stream::iter(source.open( + decoder, + file_schema, + batch_size, + file_column_projection_indices, + )?) + .boxed()) } GetResultPayload::Stream(s) => { - let decoder = config.builder().build_decoder(); + let decoder = source + .builder(file_schema, batch_size, file_column_projection_indices) + .build_decoder(); let s = s.map_err(DataFusionError::from); let input = file_compression_type.convert_stream(s.boxed())?.fuse(); diff --git a/datafusion/datasource-json/src/file_format.rs b/datafusion/datasource-json/src/file_format.rs index 51f4bd7e963e..1fb806890aaa 100644 --- a/datafusion/datasource-json/src/file_format.rs +++ b/datafusion/datasource-json/src/file_format.rs @@ -251,16 +251,16 @@ impl FileFormat for JsonFormat { async fn create_physical_plan( &self, _state: &dyn Session, - conf: FileScanConfig, + source: Arc, ) -> Result> { - let source = Arc::new(JsonSource::new()); - let conf = FileScanConfigBuilder::from(conf) + let conf = FileScanConfigBuilder::from(source.config().to_owned()) .with_file_compression_type(FileCompressionType::from( self.options.compression, )) - .with_source(source) .build(); - Ok(DataSourceExec::from_data_source(conf)) + + let source = JsonSource::new(conf.clone()); + Ok(DataSourceExec::from_data_source(source)) } async fn create_writer_physical_plan( @@ -281,8 +281,8 @@ impl FileFormat for JsonFormat { Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) } - fn file_source(&self) -> Arc { - Arc::new(JsonSource::default()) + fn file_source(&self, config: FileScanConfig) -> Arc { + Arc::new(JsonSource::new(config)) } } diff --git a/datafusion/datasource-json/src/source.rs b/datafusion/datasource-json/src/source.rs index d318928e5c6b..3451a5652356 100644 --- a/datafusion/datasource-json/src/source.rs +++ b/datafusion/datasource-json/src/source.rs @@ -38,12 +38,12 @@ use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; -use datafusion_common::Statistics; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_execution::TaskContext; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion_datasource::source::DataSource; use futures::{StreamExt, TryStreamExt}; use object_store::buffered::BufWriter; use object_store::{GetOptions, GetResultPayload, ObjectStore}; @@ -75,18 +75,22 @@ impl JsonOpener { } /// JsonSource holds the extra configuration that is necessary for [`JsonOpener`] -#[derive(Clone, Default)] +#[derive(Clone)] pub struct JsonSource { - batch_size: Option, metrics: ExecutionPlanMetricsSet, - projected_statistics: Option, schema_adapter_factory: Option>, + + config: FileScanConfig, } impl JsonSource { /// Initialize a JsonSource with default values - pub fn new() -> Self { - Self::default() + pub fn new(config: FileScanConfig) -> Self { + Self { + metrics: Default::default(), + schema_adapter_factory: None, + config, + } } } @@ -97,18 +101,33 @@ impl From for Arc { } impl FileSource for JsonSource { + fn config(&self) -> &FileScanConfig { + &self.config + } + + fn with_config(&self, config: FileScanConfig) -> Arc { + let mut this = self.clone(); + this.config = config; + + Arc::new(this) + } + + fn as_data_source(&self) -> Arc { + Arc::new(self.clone()) + } + fn create_file_opener( &self, object_store: Arc, - base_config: &FileScanConfig, _partition: usize, ) -> Arc { Arc::new(JsonOpener { batch_size: self + .config .batch_size .expect("Batch size must set before creating opener"), - projected_schema: base_config.projected_file_schema(), - file_compression_type: base_config.file_compression_type, + projected_schema: self.config.projected_file_schema(), + file_compression_type: self.config.file_compression_type, object_store, }) } @@ -117,36 +136,10 @@ impl FileSource for JsonSource { self } - fn with_batch_size(&self, batch_size: usize) -> Arc { - let mut conf = self.clone(); - conf.batch_size = Some(batch_size); - Arc::new(conf) - } - - fn with_schema(&self, _schema: SchemaRef) -> Arc { - Arc::new(Self { ..self.clone() }) - } - fn with_statistics(&self, statistics: Statistics) -> Arc { - let mut conf = self.clone(); - conf.projected_statistics = Some(statistics); - Arc::new(conf) - } - - fn with_projection(&self, _config: &FileScanConfig) -> Arc { - Arc::new(Self { ..self.clone() }) - } - - fn metrics(&self) -> &ExecutionPlanMetricsSet { + fn metrics_inner(&self) -> &ExecutionPlanMetricsSet { &self.metrics } - fn statistics(&self) -> Result { - let statistics = &self.projected_statistics; - Ok(statistics - .clone() - .expect("projected_statistics must be set to call")) - } - fn file_type(&self) -> &str { "json" } @@ -166,6 +159,15 @@ impl FileSource for JsonSource { } } +impl std::fmt::Debug for JsonSource { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{} {{ ", self.file_type())?; + write!(f, "statistics={:?}, ", self.file_source_statistics())?; + write!(f, "config={:?}, ", self.config())?; + write!(f, " }}") + } +} + impl FileOpener for JsonOpener { /// Open a partitioned NDJSON file. /// diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 56718534a558..08efa3fb44d8 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -53,7 +53,7 @@ use datafusion_common::{HashMap, Statistics}; use datafusion_common_runtime::{JoinSet, SpawnedTask}; use datafusion_datasource::display::FileGroupDisplay; use datafusion_datasource::file::FileSource; -use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; +use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::sink::{DataSink, DataSinkExec}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; @@ -475,7 +475,7 @@ impl FileFormat for ParquetFormat { async fn create_physical_plan( &self, state: &dyn Session, - conf: FileScanConfig, + source: Arc, ) -> Result> { let mut metadata_size_hint = None; @@ -483,13 +483,17 @@ impl FileFormat for ParquetFormat { metadata_size_hint = Some(metadata); } - let mut source = ParquetSource::new(self.options.clone()); + let mut source = source + .as_any() + .downcast_ref::() + .unwrap() + .clone(); // Use the CachedParquetFileReaderFactory let metadata_cache = state.runtime_env().cache_manager.get_file_metadata_cache(); let store = state .runtime_env() - .object_store(conf.object_store_url.clone())?; + .object_store(source.config().object_store_url.clone())?; let cached_parquet_read_factory = Arc::new(CachedParquetFileReaderFactory::new(store, metadata_cache)); source = source.with_parquet_file_reader_factory(cached_parquet_read_factory); @@ -500,13 +504,7 @@ impl FileFormat for ParquetFormat { source = self.set_source_encryption_factory(source, state)?; - // Apply schema adapter factory before building the new config - let file_source = source.apply_schema_adapter(&conf)?; - - let conf = FileScanConfigBuilder::from(conf) - .with_source(file_source) - .build(); - Ok(DataSourceExec::from_data_source(conf)) + Ok(Arc::new(DataSourceExec::new(source.as_data_source()))) } async fn create_writer_physical_plan( @@ -525,8 +523,8 @@ impl FileFormat for ParquetFormat { Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _) } - fn file_source(&self) -> Arc { - Arc::new(ParquetSource::default()) + fn file_source(&self, config: FileScanConfig) -> Arc { + Arc::new(ParquetSource::new(self.options.clone(), config)) } } diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index caec7db0ce0b..216cfec02e30 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -17,8 +17,7 @@ //! ParquetSource implementation for reading parquet files use std::any::Any; -use std::fmt::Debug; -use std::fmt::Formatter; +use std::fmt::{self, Debug, Formatter}; use std::sync::Arc; use crate::opener::build_pruning_predicates; @@ -35,7 +34,7 @@ use datafusion_datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapterFactory, }; -use arrow::datatypes::{SchemaRef, TimeUnit}; +use arrow::datatypes::TimeUnit; use datafusion_common::config::TableParquetOptions; use datafusion_common::{DataFusionError, Statistics}; use datafusion_datasource::file::FileSource; @@ -53,6 +52,7 @@ use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::DisplayFormatType; use datafusion_common::encryption::map_config_decryption_to_decryption; +use datafusion_datasource::source::DataSource; #[cfg(feature = "parquet_encryption")] use datafusion_execution::parquet_encryption::EncryptionFactory; use itertools::Itertools; @@ -98,18 +98,20 @@ use object_store::ObjectStore; /// # use datafusion_physical_expr::expressions::lit; /// # use datafusion_datasource::source::DataSourceExec; /// # use datafusion_common::config::TableParquetOptions; +/// use datafusion_datasource::file::FileSource; /// /// # let file_schema = Arc::new(Schema::empty()); /// # let object_store_url = ObjectStoreUrl::local_filesystem(); /// # let predicate = lit(true); -/// let source = Arc::new( -/// ParquetSource::default() -/// .with_predicate(predicate) -/// ); /// // Create a DataSourceExec for reading `file1.parquet` with a file size of 100MB -/// let config = FileScanConfigBuilder::new(object_store_url, file_schema, source) +/// let config = FileScanConfigBuilder::new(object_store_url, file_schema) /// .with_file(PartitionedFile::new("file1.parquet", 100*1024*1024)).build(); -/// let exec = DataSourceExec::from_data_source(config); +/// +/// let source = ParquetSource::new(TableParquetOptions::default(), config) +/// .with_predicate(predicate) +/// .as_data_source(); +/// +/// let exec = Arc::new(DataSourceExec::new(source)); /// ``` /// /// # Features @@ -173,15 +175,18 @@ use object_store::ObjectStore; /// ```no_run /// # use std::sync::Arc; /// # use arrow::datatypes::Schema; +/// # use datafusion_datasource::file::FileSource; /// # use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; /// # use datafusion_datasource::PartitionedFile; /// # use datafusion_datasource::source::DataSourceExec; +/// # use datafusion_datasource_parquet::source::ParquetSource; /// -/// # fn parquet_exec() -> DataSourceExec { unimplemented!() } +/// fn parquet_exec() -> DataSourceExec { unimplemented!() } /// // Split a single DataSourceExec into multiple DataSourceExecs, one for each file /// let exec = parquet_exec(); /// let data_source = exec.data_source(); -/// let base_config = data_source.as_any().downcast_ref::().unwrap(); +/// let parquet_source = data_source.as_any().downcast_ref::().unwrap(); +/// let base_config = parquet_source.config(); /// let existing_file_groups = &base_config.file_groups; /// let new_execs = existing_file_groups /// .iter() @@ -191,7 +196,9 @@ use object_store::ObjectStore; /// .with_file_groups(vec![file_group.clone()]) /// .build(); /// -/// (DataSourceExec::from_data_source(new_config)) +/// let source = parquet_source.with_config(new_config).as_data_source(); +/// +/// Arc::new(DataSourceExec::new(source)) /// }) /// .collect::>(); /// ``` @@ -212,6 +219,7 @@ use object_store::ObjectStore; /// ``` /// # use std::sync::Arc; /// # use arrow::datatypes::{Schema, SchemaRef}; +/// # use datafusion_common::config::TableParquetOptions; /// # use datafusion_datasource::PartitionedFile; /// # use datafusion_datasource_parquet::ParquetAccessPlan; /// # use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; @@ -230,11 +238,14 @@ use object_store::ObjectStore; /// let partitioned_file = PartitionedFile::new("my_file.parquet", 1234) /// .with_extensions(Arc::new(access_plan)); /// // create a FileScanConfig to scan this file -/// let config = FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), schema(), Arc::new(ParquetSource::default())) +/// let config = FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), schema()) /// .with_file(partitioned_file).build(); +/// +/// let source = ParquetSource::new(TableParquetOptions::default(), config); +/// /// // this parquet DataSourceExec will not even try to read row groups 2 and 4. Additional /// // pruning based on predicates may also happen -/// let exec = DataSourceExec::from_data_source(config); +/// let exec = DataSourceExec::from_data_source(source); /// ``` /// /// For a complete example, see the [`advanced_parquet_index` example]). @@ -265,39 +276,44 @@ use object_store::ObjectStore; /// [`RecordBatch`]: arrow::record_batch::RecordBatch /// [`SchemaAdapter`]: datafusion_datasource::schema_adapter::SchemaAdapter /// [`ParquetMetadata`]: parquet::file::metadata::ParquetMetaData -#[derive(Clone, Default, Debug)] +#[derive(Clone)] pub struct ParquetSource { /// Options for reading Parquet files pub(crate) table_parquet_options: TableParquetOptions, /// Optional metrics pub(crate) metrics: ExecutionPlanMetricsSet, - /// The schema of the file. - /// In particular, this is the schema of the table without partition columns, - /// *not* the physical schema of the file. - pub(crate) file_schema: Option, /// Optional predicate for row filtering during parquet scan pub(crate) predicate: Option>, /// Optional user defined parquet file reader factory pub(crate) parquet_file_reader_factory: Option>, /// Optional user defined schema adapter pub(crate) schema_adapter_factory: Option>, - /// Batch size configuration - pub(crate) batch_size: Option, /// Optional hint for the size of the parquet metadata pub(crate) metadata_size_hint: Option, - pub(crate) projected_statistics: Option, #[cfg(feature = "parquet_encryption")] pub(crate) encryption_factory: Option>, + + pub(crate) config: FileScanConfig, } impl ParquetSource { /// Create a new ParquetSource to read the data specified in the file scan /// configuration with the provided `TableParquetOptions`. /// if default values are going to be used, use `ParguetConfig::default()` instead - pub fn new(table_parquet_options: TableParquetOptions) -> Self { + pub fn new( + table_parquet_options: TableParquetOptions, + config: FileScanConfig, + ) -> Self { Self { table_parquet_options, - ..Self::default() + metrics: Default::default(), + predicate: None, + parquet_file_reader_factory: None, + schema_adapter_factory: None, + metadata_size_hint: None, + #[cfg(feature = "parquet_encryption")] + encryption_factory: None, + config, } } @@ -432,16 +448,13 @@ impl ParquetSource { /// * `conf` - FileScanConfig that may contain a schema adapter factory /// # Returns /// The converted FileSource with schema adapter factory applied if provided - pub fn apply_schema_adapter( - self, - conf: &FileScanConfig, - ) -> datafusion_common::Result> { - let file_source: Arc = self.into(); + pub fn apply_schema_adapter(self) -> datafusion_common::Result> { + let file_source: Arc = self.clone().into(); // If the FileScanConfig.file_source() has a schema adapter factory, apply it - if let Some(factory) = conf.file_source().schema_adapter_factory() { + if let Some(factory) = &self.schema_adapter_factory() { file_source.with_schema_adapter_factory( - Arc::::clone(&factory), + Arc::::clone(factory), ) } else { Ok(file_source) @@ -488,12 +501,28 @@ impl From for Arc { } impl FileSource for ParquetSource { + fn config(&self) -> &FileScanConfig { + &self.config + } + + fn with_config(&self, config: FileScanConfig) -> Arc { + let mut this = self.clone(); + this.config = config; + + Arc::new(this) + } + + fn as_data_source(&self) -> Arc { + Arc::new(self.clone()) + } + fn create_file_opener( &self, object_store: Arc, - base_config: &FileScanConfig, partition: usize, ) -> Arc { + let base_config = self.config(); + let projection = base_config .file_column_projection_indices() .unwrap_or_else(|| (0..base_config.file_schema.fields().len()).collect()); @@ -559,7 +588,7 @@ impl FileSource for ParquetSource { Arc::new(ParquetOpener { partition_index: partition, projection: Arc::from(projection), - batch_size: self + batch_size: base_config .batch_size .expect("Batch size must set before creating ParquetOpener"), limit: base_config.limit, @@ -567,7 +596,7 @@ impl FileSource for ParquetSource { logical_file_schema: Arc::clone(&base_config.file_schema), partition_fields: base_config.table_partition_cols.clone(), metadata_size_hint: self.metadata_size_hint, - metrics: self.metrics().clone(), + metrics: self.metrics_inner().clone(), parquet_file_reader_factory, pushdown_filters: self.pushdown_filters(), reorder_filters: self.reorder_filters(), @@ -587,55 +616,30 @@ impl FileSource for ParquetSource { self } - fn with_batch_size(&self, batch_size: usize) -> Arc { - let mut conf = self.clone(); - conf.batch_size = Some(batch_size); - Arc::new(conf) - } - - fn with_schema(&self, schema: SchemaRef) -> Arc { - Arc::new(Self { - file_schema: Some(schema), - ..self.clone() - }) - } - - fn with_statistics(&self, statistics: Statistics) -> Arc { - let mut conf = self.clone(); - conf.projected_statistics = Some(statistics); - Arc::new(conf) - } - - fn with_projection(&self, _config: &FileScanConfig) -> Arc { - Arc::new(Self { ..self.clone() }) - } - - fn metrics(&self) -> &ExecutionPlanMetricsSet { + fn metrics_inner(&self) -> &ExecutionPlanMetricsSet { &self.metrics } - fn statistics(&self) -> datafusion_common::Result { - let statistics = &self.projected_statistics; - let statistics = statistics - .clone() - .expect("projected_statistics must be set"); + fn file_source_statistics(&self) -> Statistics { + let statistics = self.config.file_source_projected_statistics.clone(); + // When filters are pushed down, we have no way of knowing the exact statistics. // Note that pruning predicate is also a kind of filter pushdown. // (bloom filters use `pruning_predicate` too). // Because filter pushdown may happen dynamically as long as there is a predicate // if we have *any* predicate applied, we can't guarantee the statistics are exact. if self.predicate().is_some() { - Ok(statistics.to_inexact()) - } else { - Ok(statistics) + return statistics.to_inexact(); } + + statistics } fn file_type(&self) -> &str { "parquet" } - fn fmt_extra(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { + fn fmt_extra(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { let predicate_string = self @@ -653,8 +657,8 @@ impl FileSource for ParquetSource { // the actual predicates are built in reference to the physical schema of // each file, which we do not have at this point and hence cannot use. // Instead we use the logical schema of the file (the table schema without partition columns). - if let (Some(file_schema), Some(predicate)) = - (&self.file_schema, &self.predicate) + if let (file_schema, Some(predicate)) = + (&self.config.file_schema, &self.predicate) { let predicate_creation_errors = Count::new(); if let (Some(pruning_predicate), _) = build_pruning_predicates( @@ -692,11 +696,8 @@ impl FileSource for ParquetSource { filters: Vec>, config: &ConfigOptions, ) -> datafusion_common::Result>> { - let Some(file_schema) = self.file_schema.clone() else { - return Ok(FilterPushdownPropagation::with_parent_pushdown_result( - vec![PushedDown::No; filters.len()], - )); - }; + let file_schema = Arc::clone(&self.config.file_schema); + // Determine if based on configs we should push filters down. // If either the table / scan itself or the config has pushdown enabled, // we will push down the filters. @@ -773,3 +774,12 @@ impl FileSource for ParquetSource { self.schema_adapter_factory.clone() } } + +impl Debug for ParquetSource { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + write!(f, "{} {{ ", self.file_type())?; + write!(f, "statistics={:?}, ", self.file_source_statistics())?; + write!(f, "config={:?} ", self.config())?; + write!(f, " }}") + } +} diff --git a/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs index e9288a5f80f6..528ccb3c9de1 100644 --- a/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs +++ b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs @@ -20,6 +20,7 @@ mod parquet_adapter_tests { datatypes::{DataType, Field, Schema, SchemaRef}, record_batch::RecordBatch, }; + use datafusion_common::config::TableParquetOptions; use datafusion_common::{ColumnStatistics, DataFusionError, Result}; use datafusion_datasource::{ file::FileSource, @@ -133,25 +134,31 @@ mod parquet_adapter_tests { Field::new("name", DataType::Utf8, true), ])); + let config = FileScanConfigBuilder::new( + ObjectStoreUrl::local_filesystem(), + schema.clone(), + ) + .build(); + // Create a parquet source - let source = ParquetSource::default(); + let source = ParquetSource::new(TableParquetOptions::default(), config); // Create a file scan config with source that has a schema adapter factory let factory = Arc::new(PrefixAdapterFactory { prefix: "test_".to_string(), }); - let file_source = source.clone().with_schema_adapter_factory(factory).unwrap(); - - let config = FileScanConfigBuilder::new( - ObjectStoreUrl::local_filesystem(), - schema.clone(), - file_source, - ) - .build(); + let file_source = source + .clone() + .with_schema_adapter_factory(factory) + .unwrap() + .as_any() + .downcast_ref::() + .unwrap() + .clone(); // Apply schema adapter to a new source - let result_source = source.apply_schema_adapter(&config).unwrap(); + let result_source = file_source.apply_schema_adapter().unwrap(); // Verify the adapter was applied assert!(result_source.schema_adapter_factory().is_some()); @@ -183,22 +190,17 @@ mod parquet_adapter_tests { Field::new("name", DataType::Utf8, true), ])); - // Create a parquet source - let source = ParquetSource::default(); - - // Convert to Arc - let file_source: Arc = Arc::new(source.clone()); - // Create a file scan config without a schema adapter factory let config = FileScanConfigBuilder::new( ObjectStoreUrl::local_filesystem(), schema.clone(), - file_source, ) .build(); + // Create a parquet source + let source = ParquetSource::new(TableParquetOptions::default(), config); // Apply schema adapter function - should pass through the source unchanged - let result_source = source.apply_schema_adapter(&config).unwrap(); + let result_source = source.apply_schema_adapter().unwrap(); // Verify no adapter was applied assert!(result_source.schema_adapter_factory().is_none()); diff --git a/datafusion/datasource/src/display.rs b/datafusion/datasource/src/display.rs index c9e979535963..92765bf0fa08 100644 --- a/datafusion/datasource/src/display.rs +++ b/datafusion/datasource/src/display.rs @@ -27,7 +27,7 @@ use std::fmt::{Debug, Formatter, Result as FmtResult}; /// {NUM_GROUPS groups: [[file1, file2,...], [fileN, fileM, ...], ...]} /// ``` #[derive(Debug)] -pub(crate) struct FileGroupsDisplay<'a>(pub(crate) &'a [FileGroup]); +pub struct FileGroupsDisplay<'a>(pub(crate) &'a [FileGroup]); impl DisplayAs for FileGroupsDisplay<'_> { fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult { diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 29fa38a8ee36..2f315b27e60a 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -17,23 +17,39 @@ //! Common behaviors that every file format needs to implement -use std::any::Any; -use std::fmt; -use std::fmt::Formatter; -use std::sync::Arc; - use crate::file_groups::FileGroupPartitioner; -use crate::file_scan_config::FileScanConfig; -use crate::file_stream::FileOpener; +use crate::file_scan_config::{ + get_projected_output_ordering, FileScanConfig, FileScanConfigBuilder, +}; +use crate::file_stream::{FileOpener, FileStream}; use crate::schema_adapter::SchemaAdapterFactory; use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; -use datafusion_common::{not_impl_err, Result, Statistics}; -use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; +use datafusion_common::{ + not_impl_err, ColumnStatistics, Constraints, Result, Statistics, +}; +use datafusion_physical_expr::{ + EquivalenceProperties, LexOrdering, Partitioning, PhysicalExpr, +}; use datafusion_physical_plan::filter_pushdown::{FilterPushdownPropagation, PushedDown}; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; -use datafusion_physical_plan::DisplayFormatType; +use datafusion_physical_plan::DisplayAs; +use datafusion_physical_plan::{DisplayFormatType, ExecutionPlan}; +use std::any::Any; +use std::fmt; +use std::fmt::Formatter; +use std::sync::Arc; +use crate::display::FileGroupsDisplay; +use crate::source::{DataSource, DataSourceExec}; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_plan::coop::cooperative; +use datafusion_physical_plan::display::{display_orderings, ProjectSchemaDisplay}; +use datafusion_physical_plan::execution_plan::SchedulingType; +use datafusion_physical_plan::projection::{ + all_alias_free_columns, new_projections_for_columns, ProjectionExec, +}; use object_store::ObjectStore; /// Helper function to convert any type implementing FileSource to Arc<dyn FileSource> @@ -51,28 +67,27 @@ pub fn as_file_source(source: T) -> Arc /// * [`ParquetSource`](https://docs.rs/datafusion/latest/datafusion/datasource/physical_plan/struct.ParquetSource.html) /// /// [`DataSource`]: crate::source::DataSource -pub trait FileSource: Send + Sync { +pub trait FileSource: fmt::Debug + Send + Sync { + fn config(&self) -> &FileScanConfig; + + fn with_config(&self, config: FileScanConfig) -> Arc; + + fn as_data_source(&self) -> Arc; + /// Creates a `dyn FileOpener` based on given parameters fn create_file_opener( &self, object_store: Arc, - base_config: &FileScanConfig, partition: usize, ) -> Arc; /// Any fn as_any(&self) -> &dyn Any; - /// Initialize new type with batch size configuration - fn with_batch_size(&self, batch_size: usize) -> Arc; - /// Initialize new instance with a new schema - fn with_schema(&self, schema: SchemaRef) -> Arc; - /// Initialize new instance with projection information - fn with_projection(&self, config: &FileScanConfig) -> Arc; - /// Initialize new instance with projected statistics - fn with_statistics(&self, statistics: Statistics) -> Arc; /// Return execution plan metrics - fn metrics(&self) -> &ExecutionPlanMetricsSet; + fn metrics_inner(&self) -> &ExecutionPlanMetricsSet; /// Return projected statistics - fn statistics(&self) -> Result; + fn file_source_statistics(&self) -> Statistics { + self.config().file_source_projected_statistics.clone() + } /// String representation of file source such as "csv", "json", "parquet" fn file_type(&self) -> &str; /// Format FileType specific information @@ -86,13 +101,14 @@ pub trait FileSource: Send + Sync { /// /// The default implementation uses [`FileGroupPartitioner`]. See that /// struct for more details. - fn repartitioned( + fn repartitioned_inner( &self, target_partitions: usize, repartition_file_min_size: usize, output_ordering: Option, - config: &FileScanConfig, - ) -> Result> { + ) -> Result>> { + let config = self.config(); + if config.file_compression_type.is_compressed() || config.new_lines_in_values { return Ok(None); } @@ -104,9 +120,9 @@ pub trait FileSource: Send + Sync { .repartition_file_groups(&config.file_groups); if let Some(repartitioned_file_groups) = repartitioned_file_groups_option { - let mut source = config.clone(); - source.file_groups = repartitioned_file_groups; - return Ok(Some(source)); + let mut config = config.clone(); + config.file_groups = repartitioned_file_groups; + return Ok(Some(self.with_config(config))); } Ok(None) } @@ -150,4 +166,236 @@ pub trait FileSource: Send + Sync { fn schema_adapter_factory(&self) -> Option> { None } + + /// Project the schema, constraints, and the statistics on the given column indices + fn project(&self) -> (SchemaRef, Constraints, Statistics, Vec) { + let config = self.config(); + + if config.projection.is_none() && config.table_partition_cols.is_empty() { + return ( + Arc::clone(&config.file_schema), + config.constraints.clone(), + self.file_source_statistics(), + config.output_ordering.clone(), + ); + } + + let schema = config.projected_schema(); + let constraints = config.projected_constraints(); + let stats = self.as_data_source().data_source_statistics(); + + let output_ordering = get_projected_output_ordering(config, &schema); + + (schema, constraints, stats, output_ordering) + } +} + +impl DataSource for T { + fn open( + &self, + partition: usize, + context: Arc, + ) -> Result { + let object_store = context + .runtime_env() + .object_store(&self.config().object_store_url)?; + + let batch_size = self + .config() + .batch_size + .unwrap_or_else(|| context.session_config().batch_size()); + + let config = FileScanConfigBuilder::from(self.config().clone()) + .with_batch_size(Some(batch_size)) + .build(); + + let source = self.with_config(config); + + let opener = source.create_file_opener(object_store, partition); + + let stream = + FileStream::new(source.config(), partition, opener, source.metrics_inner())?; + Ok(Box::pin(cooperative(stream))) + } + + 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 schema = self.config().projected_schema(); + let orderings = get_projected_output_ordering(self.config(), &schema); + + write!(f, "file_groups=")?; + FileGroupsDisplay(&self.config().file_groups).fmt_as(t, f)?; + + if !schema.fields().is_empty() { + write!(f, ", projection={}", ProjectSchemaDisplay(&schema))?; + } + + if let Some(limit) = self.config().limit { + write!(f, ", limit={limit}")?; + } + + display_orderings(f, &orderings)?; + + if !self.config().constraints.is_empty() { + write!(f, ", {}", self.config().constraints)?; + } + + write!(f, ", file_type={}", self.file_type())?; + self.fmt_extra(t, f) + } + DisplayFormatType::TreeRender => { + writeln!(f, "format={}", self.file_type())?; + self.fmt_extra(t, f)?; + let num_files = self + .config() + .file_groups + .iter() + .map(|fg| fg.len()) + .sum::(); + writeln!(f, "files={num_files}")?; + Ok(()) + } + } + } + + /// If supported by the underlying [`FileSource`], redistribute files across partitions according to their size. + fn repartitioned( + &self, + target_partitions: usize, + repartition_file_min_size: usize, + output_ordering: Option, + ) -> Result>> { + let source = self.repartitioned_inner( + target_partitions, + repartition_file_min_size, + output_ordering, + )?; + + Ok(source.map(|s| s.as_data_source())) + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(self.config().file_groups.len()) + } + + fn eq_properties(&self) -> EquivalenceProperties { + let (schema, constraints, _, orderings) = self.project(); + EquivalenceProperties::new_with_orderings(schema, orderings) + .with_constraints(constraints) + } + + fn scheduling_type(&self) -> SchedulingType { + SchedulingType::Cooperative + } + + fn data_source_statistics(&self) -> Statistics { + let file_source_statistics = self.file_source_statistics(); + + let table_cols_stats = self + .config() + .projection_indices() + .into_iter() + .map(|idx| { + if idx < self.config().file_schema.fields().len() { + file_source_statistics.column_statistics[idx].clone() + } else { + // TODO provide accurate stat for partition column (#1186) + ColumnStatistics::new_unknown() + } + }) + .collect(); + + Statistics { + num_rows: file_source_statistics.num_rows, + // TODO correct byte size: https://github.com/apache/datafusion/issues/14936 + total_byte_size: file_source_statistics.total_byte_size, + column_statistics: table_cols_stats, + } + } + + fn with_fetch(&self, limit: Option) -> Option> { + let config = FileScanConfigBuilder::from(self.config().clone()) + .with_limit(limit) + .build(); + + Some(self.with_config(config).as_data_source()) + } + + fn fetch(&self) -> Option { + self.config().limit + } + + fn metrics(&self) -> ExecutionPlanMetricsSet { + self.metrics_inner().clone() + } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + // This process can be moved into CsvExec, but it would be an overlap of their responsibility. + + // Must be all column references, with no table partition columns (which can not be projected) + let partitioned_columns_in_proj = projection.expr().iter().any(|(expr, _)| { + expr.as_any() + .downcast_ref::() + .map(|expr| expr.index() >= self.config().file_schema.fields().len()) + .unwrap_or(false) + }); + + // If there is any non-column or alias-carrier expression, Projection should not be removed. + let no_aliases = all_alias_free_columns(projection.expr()); + + Ok((no_aliases && !partitioned_columns_in_proj).then(|| { + let new_projections = new_projections_for_columns( + projection, + &self.config().projection.clone().unwrap_or_else(|| { + (0..self.config().file_schema.fields().len()).collect() + }), + ); + + let source = self + .with_config( + FileScanConfigBuilder::from(self.config().to_owned()) + // Assign projected statistics to source + .with_projection(Some(new_projections)) + .build(), + ) + .as_data_source(); + + Arc::new(DataSourceExec::new(source)) as Arc + })) + } + + fn try_pushdown_filters( + &self, + filters: Vec>, + config: &ConfigOptions, + ) -> Result>> { + let result = self.try_pushdown_filters(filters, config)?; + match result.updated_node { + Some(new_file_source) => Ok(FilterPushdownPropagation { + filters: result.filters, + updated_node: Some(new_file_source.as_data_source()), + }), + None => { + // If the file source does not support filter pushdown, return the original config + Ok(FilterPushdownPropagation { + filters: result.filters, + updated_node: None, + }) + } + } + } + + fn as_file_source(&self) -> Option> { + // just trigger a clone here + let this = self.with_config(self.config().to_owned()); + Some(this) + } } diff --git a/datafusion/datasource/src/file_format.rs b/datafusion/datasource/src/file_format.rs index 23f68636c156..71a8135d872d 100644 --- a/datafusion/datasource/src/file_format.rs +++ b/datafusion/datasource/src/file_format.rs @@ -35,6 +35,7 @@ use datafusion_physical_expr::LexRequirement; use datafusion_physical_plan::ExecutionPlan; use datafusion_session::Session; +use crate::source::DataSourceExec; use async_trait::async_trait; use object_store::{ObjectMeta, ObjectStore}; @@ -94,9 +95,11 @@ pub trait FileFormat: Send + Sync + fmt::Debug { /// according to this file format. async fn create_physical_plan( &self, - state: &dyn Session, - conf: FileScanConfig, - ) -> Result>; + _state: &dyn Session, + file_source: Arc, + ) -> Result> { + Ok(Arc::new(DataSourceExec::new(file_source.as_data_source()))) + } /// Take a list of files and the configuration to convert it to the /// appropriate writer executor according to this file format. @@ -111,7 +114,7 @@ pub trait FileFormat: Send + Sync + fmt::Debug { } /// Return the related FileSource such as `CsvSource`, `JsonSource`, etc. - fn file_source(&self) -> Arc; + fn file_source(&self, config: FileScanConfig) -> Arc; } /// Factory for creating [`FileFormat`] instances based on session and command level options diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 7088f811bbce..1c100f18ab6d 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -15,11 +15,12 @@ // specific language governing permissions and limitations // under the License. -//! [`FileScanConfig`] to configure scanning of possibly partitioned -//! file sources. +//! [`FileScanConfig`] for configuring file-based data sources that implement [`DataSource`]. +//! +//! [`DataSource`]: crate::source::DataSource use std::{ - any::Any, borrow::Cow, collections::HashMap, fmt::Debug, fmt::Formatter, + borrow::Cow, collections::HashMap, fmt::Debug, fmt::Formatter, fmt::Result as FmtResult, marker::PhantomData, sync::Arc, }; @@ -27,13 +28,8 @@ use crate::file_groups::FileGroup; #[allow(unused_imports)] use crate::schema_adapter::SchemaAdapterFactory; use crate::{ - display::FileGroupsDisplay, - file::FileSource, - file_compression_type::FileCompressionType, - file_stream::FileStream, - source::{DataSource, DataSourceExec}, - statistics::MinMaxStatistics, - PartitionedFile, + display::FileGroupsDisplay, file_compression_type::FileCompressionType, + statistics::MinMaxStatistics, PartitionedFile, }; use arrow::datatypes::FieldRef; use arrow::{ @@ -44,39 +40,33 @@ use arrow::{ buffer::Buffer, datatypes::{ArrowNativeType, DataType, Field, Schema, SchemaRef, UInt16Type}, }; -use datafusion_common::config::ConfigOptions; use datafusion_common::{ - exec_err, ColumnStatistics, Constraints, DataFusionError, Result, ScalarValue, - Statistics, -}; -use datafusion_execution::{ - object_store::ObjectStoreUrl, SendableRecordBatchStream, TaskContext, + exec_err, Constraints, DataFusionError, Result, ScalarValue, Statistics, }; +use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::{EquivalenceProperties, Partitioning}; use datafusion_physical_expr_adapter::PhysicalExprAdapterFactory; -use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; -use datafusion_physical_plan::filter_pushdown::FilterPushdownPropagation; use datafusion_physical_plan::{ display::{display_orderings, ProjectSchemaDisplay}, - metrics::ExecutionPlanMetricsSet, - projection::{all_alias_free_columns, new_projections_for_columns, ProjectionExec}, - DisplayAs, DisplayFormatType, ExecutionPlan, + DisplayAs, DisplayFormatType, }; -use datafusion_physical_plan::coop::cooperative; -use datafusion_physical_plan::execution_plan::SchedulingType; use log::{debug, warn}; -/// The base configurations for a [`DataSourceExec`], the a physical plan for -/// any given file format. +/// Configuration for file-based data sources. +/// +/// Owned by file sources that implement [`DataSource`]. Use [`DataSourceExec::from_data_source`] +/// to create a [`DataSourceExec`] from a data source that owns this config. /// -/// Use [`DataSourceExec::from_data_source`] to create a [`DataSourceExec`] from a ``FileScanConfig`. +/// [`DataSourceExec::from_data_source`]: crate::source::DataSourceExec::from_data_source +/// [`DataSourceExec`]: crate::source::DataSourceExec +/// [`DataSource`]: crate::source::DataSource /// /// # Example /// ``` /// # use std::any::Any; +/// # use std::fmt::{Debug, Formatter}; /// # use std::sync::Arc; /// # use arrow::datatypes::{Field, Fields, DataType, Schema, SchemaRef}; /// # use object_store::ObjectStore; @@ -87,7 +77,7 @@ use log::{debug, warn}; /// # use datafusion_datasource::PartitionedFile; /// # use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder}; /// # use datafusion_datasource::file_stream::FileOpener; -/// # use datafusion_datasource::source::DataSourceExec; +/// # use datafusion_datasource::source::{DataSource, DataSourceExec}; /// # use datafusion_execution::object_store::ObjectStoreUrl; /// # use datafusion_physical_plan::ExecutionPlan; /// # use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; @@ -102,28 +92,28 @@ use log::{debug, warn}; /// #[derive(Clone)] /// # struct ParquetSource { /// # projected_statistics: Option, -/// # schema_adapter_factory: Option> +/// # schema_adapter_factory: Option>, +/// # config: FileScanConfig, /// # }; -/// # impl FileSource for ParquetSource { -/// # fn create_file_opener(&self, _: Arc, _: &FileScanConfig, _: usize) -> Arc { unimplemented!() } +/// # impl Debug for ParquetSource {fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { unimplemented!() }} +/// +/// impl FileSource for ParquetSource { +/// # fn config(&self) -> &FileScanConfig { &self.config } +/// # fn with_config(&self, config: FileScanConfig) -> Arc { unimplemented!() } +/// # fn as_data_source(&self) -> Arc { Arc::new(self.clone()) } +/// # fn create_file_opener(&self, _: Arc, _: usize) -> Arc { unimplemented!() } /// # fn as_any(&self) -> &dyn Any { self } -/// # fn with_batch_size(&self, _: usize) -> Arc { unimplemented!() } -/// # fn with_schema(&self, _: SchemaRef) -> Arc { Arc::new(self.clone()) as Arc } -/// # fn with_projection(&self, _: &FileScanConfig) -> Arc { unimplemented!() } -/// # fn with_statistics(&self, statistics: Statistics) -> Arc { Arc::new(Self {projected_statistics: Some(statistics), schema_adapter_factory: self.schema_adapter_factory.clone()} ) } -/// # fn metrics(&self) -> &ExecutionPlanMetricsSet { unimplemented!() } -/// # fn statistics(&self) -> Result { Ok(self.projected_statistics.clone().expect("projected_statistics should be set")) } +/// # fn metrics_inner(&self) -> &ExecutionPlanMetricsSet { unimplemented!() } /// # fn file_type(&self) -> &str { "parquet" } -/// # fn with_schema_adapter_factory(&self, factory: Arc) -> Result> { Ok(Arc::new(Self {projected_statistics: self.projected_statistics.clone(), schema_adapter_factory: Some(factory)} )) } +/// # fn with_schema_adapter_factory(&self, factory: Arc) -> Result> { Ok(Arc::new(Self {projected_statistics: self.projected_statistics.clone(), schema_adapter_factory: Some(factory), config: self.config.clone() } )) } /// # fn schema_adapter_factory(&self) -> Option> { self.schema_adapter_factory.clone() } -/// # } +/// # } /// # impl ParquetSource { -/// # fn new() -> Self { Self {projected_statistics: None, schema_adapter_factory: None} } +/// # fn new(config: FileScanConfig) -> Self { Self {projected_statistics: None, schema_adapter_factory: None, config} } /// # } /// // create FileScan config for reading parquet files from file:// /// let object_store_url = ObjectStoreUrl::local_filesystem(); -/// let file_source = Arc::new(ParquetSource::new()); -/// let config = FileScanConfigBuilder::new(object_store_url, file_schema, file_source) +/// let config = FileScanConfigBuilder::new(object_store_url, file_schema) /// .with_limit(Some(1000)) // read only the first 1000 records /// .with_projection(Some(vec![2, 3])) // project columns 2 and 3 /// // Read /tmp/file1.parquet with known size of 1234 bytes in a single group @@ -134,8 +124,8 @@ use log::{debug, warn}; /// PartitionedFile::new("file2.parquet", 56), /// PartitionedFile::new("file3.parquet", 78), /// ])).build(); -/// // create an execution plan from the config -/// let plan: Arc = DataSourceExec::from_data_source(config); +/// // create an execution plan from the data source that owns the config +/// let plan: Arc = DataSourceExec::from_data_source(ParquetSource::new(config)); /// ``` #[derive(Clone)] pub struct FileScanConfig { @@ -151,12 +141,10 @@ pub struct FileScanConfig { /// [`RuntimeEnv::register_object_store`]: datafusion_execution::runtime_env::RuntimeEnv::register_object_store /// [`RuntimeEnv::object_store`]: datafusion_execution::runtime_env::RuntimeEnv::object_store pub object_store_url: ObjectStoreUrl, - /// Schema before `projection` is applied. It contains the all columns that may - /// appear in the files. It does not include table partition columns - /// that may be added. - /// Note that this is **not** the schema of the physical files. - /// This is the schema that the physical file schema will be - /// mapped onto, and the schema that the [`DataSourceExec`] will return. + /// Schema before `projection` is applied. Contains all columns that may + /// appear in the files, excluding table partition columns that may be added. + /// This is the logical schema that the physical file schema will be + /// mapped onto, and the schema that data sources will expose. pub file_schema: SchemaRef, /// List of files to be processed, grouped into partitions /// @@ -184,17 +172,17 @@ pub struct FileScanConfig { pub file_compression_type: FileCompressionType, /// Are new lines in values supported for CSVOptions pub new_lines_in_values: bool, - /// File source such as `ParquetSource`, `CsvSource`, `JsonSource`, etc. - pub file_source: Arc, /// Batch size while creating new batches /// Defaults to [`datafusion_common::config::ExecutionOptions`] batch_size. pub batch_size: Option, /// Expression adapter used to adapt filters and projections that are pushed down into the scan /// from the logical schema to the physical schema of the file. pub expr_adapter_factory: Option>, + + pub file_source_projected_statistics: Statistics, } -/// A builder for [`FileScanConfig`]'s. +/// A builder for [`FileScanConfig`]. /// /// Example: /// @@ -221,7 +209,6 @@ pub struct FileScanConfig { /// let config = FileScanConfigBuilder::new( /// ObjectStoreUrl::local_filesystem(), /// schema, -/// file_source, /// ) /// // Set a limit of 1000 rows /// .with_limit(Some(1000)) @@ -251,20 +238,19 @@ pub struct FileScanConfigBuilder { /// This schema is used to read the files, but is **not** necessarily the /// schema of the physical files. Rather this is the schema that the /// physical file schema will be mapped onto, and the schema that the - /// [`DataSourceExec`] will return. + /// [`crate::source::DataSourceExec`] will return. /// /// This is usually the same as the table schema as specified by the `TableProvider` minus any partition columns. /// /// This probably would be better named `table_schema` file_schema: SchemaRef, - file_source: Arc, limit: Option, projection: Option>, table_partition_cols: Vec, constraints: Option, file_groups: Vec, - statistics: Option, + file_source_projected_statistics: Option, output_ordering: Vec, file_compression_type: Option, new_lines_in_values: Option, @@ -278,18 +264,12 @@ impl FileScanConfigBuilder { /// # Parameters: /// * `object_store_url`: See [`FileScanConfig::object_store_url`] /// * `file_schema`: See [`FileScanConfig::file_schema`] - /// * `file_source`: See [`FileScanConfig::file_source`] - pub fn new( - object_store_url: ObjectStoreUrl, - file_schema: SchemaRef, - file_source: Arc, - ) -> Self { + pub fn new(object_store_url: ObjectStoreUrl, file_schema: SchemaRef) -> Self { Self { object_store_url, file_schema, - file_source, file_groups: vec![], - statistics: None, + file_source_projected_statistics: None, output_ordering: vec![], file_compression_type: None, new_lines_in_values: None, @@ -309,15 +289,6 @@ impl FileScanConfigBuilder { self } - /// Set the file source for scanning files. - /// - /// This method allows you to change the file source implementation (e.g. ParquetSource, CsvSource, etc.) - /// after the builder has been created. - pub fn with_source(mut self, file_source: Arc) -> Self { - self.file_source = file_source; - self - } - /// Set the columns on which to project the data. Indexes that are higher than the /// number of columns of `file_schema` refer to `table_partition_cols`. pub fn with_projection(mut self, projection: Option>) -> Self { @@ -342,8 +313,11 @@ impl FileScanConfigBuilder { /// Set the estimated overall statistics of the files, taking `filters` into account. /// Defaults to [`Statistics::new_unknown`]. - pub fn with_statistics(mut self, statistics: Statistics) -> Self { - self.statistics = Some(statistics); + pub fn with_file_source_projected_statistics( + mut self, + statistics: Statistics, + ) -> Self { + self.file_source_projected_statistics = Some(statistics); self } @@ -429,13 +403,12 @@ impl FileScanConfigBuilder { let Self { object_store_url, file_schema, - file_source, limit, projection, table_partition_cols, constraints, file_groups, - statistics, + file_source_projected_statistics, output_ordering, file_compression_type, new_lines_in_values, @@ -444,12 +417,9 @@ impl FileScanConfigBuilder { } = self; let constraints = constraints.unwrap_or_default(); - let statistics = - statistics.unwrap_or_else(|| Statistics::new_unknown(&file_schema)); + let file_source_projected_statistics = file_source_projected_statistics + .unwrap_or_else(|| Statistics::new_unknown(&file_schema)); - let file_source = file_source - .with_statistics(statistics.clone()) - .with_schema(Arc::clone(&file_schema)); let file_compression_type = file_compression_type.unwrap_or(FileCompressionType::UNCOMPRESSED); let new_lines_in_values = new_lines_in_values.unwrap_or(false); @@ -457,7 +427,6 @@ impl FileScanConfigBuilder { FileScanConfig { object_store_url, file_schema, - file_source, limit, projection, table_partition_cols, @@ -468,6 +437,7 @@ impl FileScanConfigBuilder { new_lines_in_values, batch_size, expr_adapter_factory: expr_adapter, + file_source_projected_statistics, } } } @@ -477,9 +447,10 @@ impl From for FileScanConfigBuilder { Self { object_store_url: config.object_store_url, file_schema: config.file_schema, - file_source: Arc::::clone(&config.file_source), file_groups: config.file_groups, - statistics: config.file_source.statistics().ok(), + file_source_projected_statistics: Some( + config.file_source_projected_statistics, + ), output_ordering: config.output_ordering, file_compression_type: Some(config.file_compression_type), new_lines_in_values: Some(config.new_lines_in_values), @@ -493,183 +464,8 @@ impl From for FileScanConfigBuilder { } } -impl DataSource for FileScanConfig { - fn open( - &self, - partition: usize, - context: Arc, - ) -> Result { - let object_store = context.runtime_env().object_store(&self.object_store_url)?; - let batch_size = self - .batch_size - .unwrap_or_else(|| context.session_config().batch_size()); - - let source = self - .file_source - .with_batch_size(batch_size) - .with_projection(self); - - let opener = source.create_file_opener(object_store, self, partition); - - let stream = FileStream::new(self, partition, opener, source.metrics())?; - Ok(Box::pin(cooperative(stream))) - } - - fn as_any(&self) -> &dyn Any { - self - } - - fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult { - match t { - DisplayFormatType::Default | DisplayFormatType::Verbose => { - let schema = self.projected_schema(); - let orderings = get_projected_output_ordering(self, &schema); - - write!(f, "file_groups=")?; - FileGroupsDisplay(&self.file_groups).fmt_as(t, f)?; - - if !schema.fields().is_empty() { - write!(f, ", projection={}", ProjectSchemaDisplay(&schema))?; - } - - if let Some(limit) = self.limit { - write!(f, ", limit={limit}")?; - } - - display_orderings(f, &orderings)?; - - if !self.constraints.is_empty() { - write!(f, ", {}", self.constraints)?; - } - - self.fmt_file_source(t, f) - } - DisplayFormatType::TreeRender => { - writeln!(f, "format={}", self.file_source.file_type())?; - self.file_source.fmt_extra(t, f)?; - let num_files = self.file_groups.iter().map(|fg| fg.len()).sum::(); - writeln!(f, "files={num_files}")?; - Ok(()) - } - } - } - - /// If supported by the underlying [`FileSource`], redistribute files across partitions according to their size. - fn repartitioned( - &self, - target_partitions: usize, - repartition_file_min_size: usize, - output_ordering: Option, - ) -> Result>> { - let source = self.file_source.repartitioned( - target_partitions, - repartition_file_min_size, - output_ordering, - self, - )?; - - Ok(source.map(|s| Arc::new(s) as _)) - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(self.file_groups.len()) - } - - fn eq_properties(&self) -> EquivalenceProperties { - let (schema, constraints, _, orderings) = self.project(); - EquivalenceProperties::new_with_orderings(schema, orderings) - .with_constraints(constraints) - } - - fn scheduling_type(&self) -> SchedulingType { - SchedulingType::Cooperative - } - - fn statistics(&self) -> Result { - Ok(self.projected_stats()) - } - - fn with_fetch(&self, limit: Option) -> Option> { - let source = FileScanConfigBuilder::from(self.clone()) - .with_limit(limit) - .build(); - Some(Arc::new(source)) - } - - fn fetch(&self) -> Option { - self.limit - } - - fn metrics(&self) -> ExecutionPlanMetricsSet { - self.file_source.metrics().clone() - } - - fn try_swapping_with_projection( - &self, - projection: &ProjectionExec, - ) -> Result>> { - // This process can be moved into CsvExec, but it would be an overlap of their responsibility. - - // Must be all column references, with no table partition columns (which can not be projected) - let partitioned_columns_in_proj = projection.expr().iter().any(|(expr, _)| { - expr.as_any() - .downcast_ref::() - .map(|expr| expr.index() >= self.file_schema.fields().len()) - .unwrap_or(false) - }); - - // If there is any non-column or alias-carrier expression, Projection should not be removed. - let no_aliases = all_alias_free_columns(projection.expr()); - - Ok((no_aliases && !partitioned_columns_in_proj).then(|| { - let file_scan = self.clone(); - let source = Arc::clone(&file_scan.file_source); - let new_projections = new_projections_for_columns( - projection, - &file_scan - .projection - .clone() - .unwrap_or_else(|| (0..self.file_schema.fields().len()).collect()), - ); - DataSourceExec::from_data_source( - FileScanConfigBuilder::from(file_scan) - // Assign projected statistics to source - .with_projection(Some(new_projections)) - .with_source(source) - .build(), - ) as _ - })) - } - - fn try_pushdown_filters( - &self, - filters: Vec>, - config: &ConfigOptions, - ) -> Result>> { - let result = self.file_source.try_pushdown_filters(filters, config)?; - match result.updated_node { - Some(new_file_source) => { - let file_scan_config = FileScanConfigBuilder::from(self.clone()) - .with_source(new_file_source) - .build(); - Ok(FilterPushdownPropagation { - filters: result.filters, - updated_node: Some(Arc::new(file_scan_config) as _), - }) - } - None => { - // If the file source does not support filter pushdown, return the original config - Ok(FilterPushdownPropagation { - filters: result.filters, - updated_node: None, - }) - } - } - } -} - impl FileScanConfig { - fn projection_indices(&self) -> Vec { + pub(crate) fn projection_indices(&self) -> Vec { match &self.projection { Some(proj) => proj.clone(), None => (0..self.file_schema.fields().len() @@ -678,30 +474,6 @@ impl FileScanConfig { } } - pub fn projected_stats(&self) -> Statistics { - let statistics = self.file_source.statistics().unwrap(); - - let table_cols_stats = self - .projection_indices() - .into_iter() - .map(|idx| { - if idx < self.file_schema.fields().len() { - statistics.column_statistics[idx].clone() - } else { - // TODO provide accurate stat for partition column (#1186) - ColumnStatistics::new_unknown() - } - }) - .collect(); - - Statistics { - num_rows: statistics.num_rows, - // TODO correct byte size: https://github.com/apache/datafusion/issues/14936 - total_byte_size: statistics.total_byte_size, - column_statistics: table_cols_stats, - } - } - pub fn projected_schema(&self) -> Arc { let table_fields: Vec<_> = self .projection_indices() @@ -739,27 +511,6 @@ impl FileScanConfig { pub fn newlines_in_values(&self) -> bool { self.new_lines_in_values } - - /// Project the schema, constraints, and the statistics on the given column indices - pub fn project(&self) -> (SchemaRef, Constraints, Statistics, Vec) { - if self.projection.is_none() && self.table_partition_cols.is_empty() { - return ( - Arc::clone(&self.file_schema), - self.constraints.clone(), - self.file_source.statistics().unwrap().clone(), - self.output_ordering.clone(), - ); - } - - let schema = self.projected_schema(); - let constraints = self.projected_constraints(); - let stats = self.projected_stats(); - - let output_ordering = get_projected_output_ordering(self, &schema); - - (schema, constraints, stats, output_ordering) - } - pub fn projected_file_column_names(&self) -> Option> { self.projection.as_ref().map(|p| { p.iter() @@ -957,17 +708,6 @@ impl FileScanConfig { }) .collect()) } - - /// Write the data_type based on file_source - fn fmt_file_source(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult { - write!(f, ", file_type={}", self.file_source.file_type())?; - self.file_source.fmt_extra(t, f) - } - - /// Returns the file_source - pub fn file_source(&self) -> &Arc { - &self.file_source - } } impl Debug for FileScanConfig { @@ -975,12 +715,6 @@ impl Debug for FileScanConfig { write!(f, "FileScanConfig {{")?; write!(f, "object_store_url={:?}, ", self.object_store_url)?; - write!( - f, - "statistics={:?}, ", - self.file_source.statistics().unwrap() - )?; - DisplayAs::fmt_as(self, DisplayFormatType::Verbose, f)?; write!(f, "}}") } @@ -1319,7 +1053,7 @@ fn create_output_array( /// /// DataSourceExec ///``` -fn get_projected_output_ordering( +pub fn get_projected_output_ordering( base_config: &FileScanConfig, projected_schema: &SchemaRef, ) -> Vec { @@ -1412,9 +1146,11 @@ mod tests { verify_sort_integrity, }; + use crate::file::FileSource; + use crate::source::DataSource; use arrow::array::{Int32Array, RecordBatch}; use datafusion_common::stats::Precision; - use datafusion_common::{assert_batches_eq, internal_err}; + use datafusion_common::{assert_batches_eq, internal_err, ColumnStatistics}; use datafusion_expr::SortExpr; use datafusion_physical_expr::create_physical_sort_expr; @@ -1426,7 +1162,7 @@ mod tests { #[test] fn physical_plan_config_no_projection() { let file_schema = aggr_test_schema(); - let conf = config_for_projection( + let source = source_for_projection( Arc::clone(&file_schema), None, Statistics::new_unknown(&file_schema), @@ -1436,7 +1172,7 @@ mod tests { )]), ); - let (proj_schema, _, proj_statistics, _) = conf.project(); + let (proj_schema, _, proj_statistics, _) = source.project(); assert_eq!(proj_schema.fields().len(), file_schema.fields().len() + 1); assert_eq!( proj_schema.field(file_schema.fields().len()).name(), @@ -1449,10 +1185,10 @@ mod tests { ); // TODO implement tests for partition column statistics once implemented - let col_names = conf.projected_file_column_names(); + let col_names = source.config().projected_file_column_names(); assert_eq!(col_names, None); - let col_indices = conf.file_column_projection_indices(); + let col_indices = source.config().file_column_projection_indices(); assert_eq!(col_indices, None); } @@ -1468,7 +1204,7 @@ mod tests { "value_whatever".to_owned(), )])); - let conf = config_for_projection( + let source = source_for_projection( Arc::clone(&file_schema), None, Statistics::new_unknown(&file_schema), @@ -1476,7 +1212,7 @@ mod tests { ); // verify the proj_schema includes the last column and exactly the same the field it is defined - let proj_schema = conf.projected_schema(); + let proj_schema = source.config().projected_schema(); assert_eq!(proj_schema.fields().len(), file_schema.fields().len() + 1); assert_eq!( *proj_schema.field(file_schema.fields().len()), @@ -1488,7 +1224,7 @@ mod tests { #[test] fn physical_plan_config_with_projection() { let file_schema = aggr_test_schema(); - let conf = config_for_projection( + let source = source_for_projection( Arc::clone(&file_schema), Some(vec![file_schema.fields().len(), 0]), Statistics { @@ -1509,7 +1245,7 @@ mod tests { )]), ); - let (proj_schema, _, proj_statistics, _) = conf.project(); + let (proj_schema, _, proj_statistics, _) = source.project(); assert_eq!( columns(&proj_schema), vec!["date".to_owned(), "c1".to_owned()] @@ -1520,10 +1256,10 @@ mod tests { // statistics are implemented assert_eq!(proj_stat_cols[1].distinct_count, Precision::Inexact(0)); - let col_names = conf.projected_file_column_names(); + let col_names = source.config().projected_file_column_names(); assert_eq!(col_names, Some(vec!["c1".to_owned()])); - let col_indices = conf.file_column_projection_indices(); + let col_indices = source.config().file_column_projection_indices(); assert_eq!(col_indices, Some(vec![0])); } @@ -1555,7 +1291,7 @@ mod tests { column_statistics: Statistics::unknown_column(&file_batch.schema()), }; - let conf = config_for_projection( + let source = source_for_projection( file_batch.schema(), // keep all cols from file and 2 from partitioning Some(vec![ @@ -1569,8 +1305,8 @@ mod tests { to_partition_cols(partition_cols.clone()), ); - let source_statistics = conf.file_source.statistics().unwrap(); - let conf_stats = conf.statistics().unwrap(); + let source_statistics = source.file_source_statistics(); + let conf_stats = source.data_source_statistics(); // projection should be reflected in the file source statistics assert_eq!(conf_stats.num_rows, Precision::Inexact(3)); @@ -1582,7 +1318,7 @@ mod tests { assert_eq!(source_statistics, statistics); assert_eq!(source_statistics.column_statistics.len(), 3); - let proj_schema = conf.projected_schema(); + let proj_schema = source.config().projected_schema(); // created a projector for that projected schema let mut proj = PartitionColumnProjector::new( proj_schema, @@ -1717,13 +1453,14 @@ mod tests { ]; // Projected file schema for config with projection including partition column - let projection = config_for_projection( + let source = source_for_projection( schema.clone(), Some(vec![0, 3, 5, schema.fields().len()]), Statistics::new_unknown(&schema), to_partition_cols(partition_cols), - ) - .projected_file_schema(); + ); + + let projection = source.config().projected_file_schema(); // Assert partition column filtered out in projected file schema let expected_columns = vec!["c1", "c4", "c6"]; @@ -1750,13 +1487,14 @@ mod tests { ]; // Projected file schema for config without projection - let projection = config_for_projection( + let source = source_for_projection( schema.clone(), None, Statistics::new_unknown(&schema), to_partition_cols(partition_cols), - ) - .projected_file_schema(); + ); + + let projection = source.config().projected_file_schema(); // Assert projected file schema is equal to file schema assert_eq!(projection.fields(), schema.fields()); @@ -2068,21 +1806,22 @@ mod tests { } // sets default for configs that play no role in projections - fn config_for_projection( + fn source_for_projection( file_schema: SchemaRef, projection: Option>, statistics: Statistics, table_partition_cols: Vec, - ) -> FileScanConfig { - FileScanConfigBuilder::new( + ) -> MockSource { + let config = FileScanConfigBuilder::new( ObjectStoreUrl::parse("test:///").unwrap(), file_schema, - Arc::new(MockSource::default()), ) .with_projection(projection) - .with_statistics(statistics) + .with_file_source_projected_statistics(statistics) .with_table_partition_cols(table_partition_cols) - .build() + .build(); + + MockSource::new(config) } /// Convert partition columns from Vec to Vec @@ -2120,13 +1859,11 @@ mod tests { fn test_file_scan_config_builder() { let file_schema = aggr_test_schema(); let object_store_url = ObjectStoreUrl::parse("test:///").unwrap(); - let file_source: Arc = Arc::new(MockSource::default()); // Create a builder with required parameters let builder = FileScanConfigBuilder::new( object_store_url.clone(), Arc::clone(&file_schema), - Arc::clone(&file_source), ); // Build with various configurations @@ -2138,7 +1875,7 @@ mod tests { wrap_partition_type_in_dict(DataType::Utf8), false, )]) - .with_statistics(Statistics::new_unknown(&file_schema)) + .with_file_source_projected_statistics(Statistics::new_unknown(&file_schema)) .with_file_groups(vec![FileGroup::new(vec![PartitionedFile::new( "test.parquet".to_string(), 1024, @@ -2176,16 +1913,16 @@ mod tests { fn test_file_scan_config_builder_defaults() { let file_schema = aggr_test_schema(); let object_store_url = ObjectStoreUrl::parse("test:///").unwrap(); - let file_source: Arc = Arc::new(MockSource::default()); // Create a builder with only required parameters and build without any additional configurations let config = FileScanConfigBuilder::new( object_store_url.clone(), Arc::clone(&file_schema), - Arc::clone(&file_source), ) .build(); + let file_source: Arc = Arc::new(MockSource::new(config.clone())); + // Verify default values assert_eq!(config.object_store_url, object_store_url); assert_eq!(config.file_schema, file_schema); @@ -2203,23 +1940,18 @@ mod tests { // Verify statistics are set to unknown assert_eq!( - config.file_source.statistics().unwrap().num_rows, + file_source.file_source_statistics().num_rows, Precision::Absent ); assert_eq!( - config.file_source.statistics().unwrap().total_byte_size, + file_source.file_source_statistics().total_byte_size, Precision::Absent ); assert_eq!( - config - .file_source - .statistics() - .unwrap() - .column_statistics - .len(), + file_source.file_source_statistics().column_statistics.len(), file_schema.fields().len() ); - for stat in config.file_source.statistics().unwrap().column_statistics { + for stat in file_source.file_source_statistics().column_statistics { assert_eq!(stat.distinct_count, Precision::Absent); assert_eq!(stat.min_value, Precision::Absent); assert_eq!(stat.max_value, Precision::Absent); @@ -2231,7 +1963,6 @@ mod tests { fn test_file_scan_config_builder_new_from() { let schema = aggr_test_schema(); let object_store_url = ObjectStoreUrl::parse("test:///").unwrap(); - let file_source: Arc = Arc::new(MockSource::default()); let partition_cols = vec![Field::new( "date", wrap_partition_type_in_dict(DataType::Utf8), @@ -2240,18 +1971,15 @@ mod tests { let file = PartitionedFile::new("test_file.parquet", 100); // Create a config with non-default values - let original_config = FileScanConfigBuilder::new( - object_store_url.clone(), - Arc::clone(&schema), - Arc::clone(&file_source), - ) - .with_projection(Some(vec![0, 2])) - .with_limit(Some(10)) - .with_table_partition_cols(partition_cols.clone()) - .with_file(file.clone()) - .with_constraints(Constraints::default()) - .with_newlines_in_values(true) - .build(); + let original_config = + FileScanConfigBuilder::new(object_store_url.clone(), Arc::clone(&schema)) + .with_projection(Some(vec![0, 2])) + .with_limit(Some(10)) + .with_table_partition_cols(partition_cols.clone()) + .with_file(file.clone()) + .with_constraints(Constraints::default()) + .with_newlines_in_values(true) + .build(); // Create a new builder from the config let new_builder = FileScanConfigBuilder::from(original_config); diff --git a/datafusion/datasource/src/file_stream.rs b/datafusion/datasource/src/file_stream.rs index 868b980b6476..cecfd5980b98 100644 --- a/datafusion/datasource/src/file_stream.rs +++ b/datafusion/datasource/src/file_stream.rs @@ -536,7 +536,6 @@ mod tests { use crate::file_meta::FileMeta; use crate::file_stream::{FileOpenFuture, FileOpener, FileStream, OnError}; - use crate::test_util::MockSource; use arrow::array::RecordBatch; use arrow::datatypes::Schema; @@ -664,11 +663,11 @@ mod tests { let config = FileScanConfigBuilder::new( ObjectStoreUrl::parse("test:///").unwrap(), file_schema, - Arc::new(MockSource::default()), ) .with_file_group(file_group) .with_limit(self.limit) .build(); + let metrics_set = ExecutionPlanMetricsSet::new(); let file_stream = FileStream::new(&config, 0, Arc::new(self.opener), &metrics_set) diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 673c1b9dd45d..957985163afe 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -194,12 +194,12 @@ impl DataSource for MemorySourceConfig { SchedulingType::Cooperative } - fn statistics(&self) -> Result { - Ok(common::compute_record_batch_statistics( + fn data_source_statistics(&self) -> Statistics { + common::compute_record_batch_statistics( &self.partitions, &self.schema, self.projection.clone(), - )) + ) } fn with_fetch(&self, limit: Option) -> Option> { diff --git a/datafusion/datasource/src/source.rs b/datafusion/datasource/src/source.rs index 153d03b3ab49..2cb8ace32b26 100644 --- a/datafusion/datasource/src/source.rs +++ b/datafusion/datasource/src/source.rs @@ -35,7 +35,7 @@ use datafusion_physical_plan::{ }; use itertools::Itertools; -use crate::file_scan_config::FileScanConfig; +use crate::file::FileSource; use datafusion_common::config::ConfigOptions; use datafusion_common::{Constraints, Result, Statistics}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; @@ -155,7 +155,7 @@ pub trait DataSource: Send + Sync + Debug { fn scheduling_type(&self) -> SchedulingType { SchedulingType::NonCooperative } - fn statistics(&self) -> Result; + fn data_source_statistics(&self) -> Statistics; /// Return a copy of this DataSource with a new fetch limit fn with_fetch(&self, _limit: Option) -> Option>; fn fetch(&self) -> Option; @@ -179,6 +179,10 @@ pub trait DataSource: Send + Sync + Debug { vec![PushedDown::No; filters.len()], )) } + + fn as_file_source(&self) -> Option> { + None + } } /// [`ExecutionPlan`] that reads one or more files @@ -291,10 +295,9 @@ impl ExecutionPlan for DataSourceExec { fn partition_statistics(&self, partition: Option) -> Result { if let Some(partition) = partition { let mut statistics = Statistics::new_unknown(&self.schema()); - if let Some(file_config) = - self.data_source.as_any().downcast_ref::() - { - if let Some(file_group) = file_config.file_groups.get(partition) { + if let Some(file_source) = self.data_source.as_file_source() { + if let Some(file_group) = file_source.config().file_groups.get(partition) + { if let Some(stat) = file_group.file_statistics(None) { statistics = stat.clone(); } @@ -302,7 +305,7 @@ impl ExecutionPlan for DataSourceExec { } Ok(statistics) } else { - Ok(self.data_source.statistics()?) + Ok(self.data_source.data_source_statistics()) } } @@ -465,18 +468,9 @@ impl DataSourceExec { /// /// Returns `None` if /// 1. the datasource is not scanning files (`FileScanConfig`) - /// 2. The [`FileScanConfig::file_source`] is not of type `T` - pub fn downcast_to_file_source(&self) -> Option<(&FileScanConfig, &T)> { - self.data_source() - .as_any() - .downcast_ref::() - .and_then(|file_scan_conf| { - file_scan_conf - .file_source() - .as_any() - .downcast_ref::() - .map(|source| (file_scan_conf, source)) - }) + /// 2. the file source is not of type `T` + pub fn downcast_to_file_source(&self) -> Option<&T> { + self.data_source().as_any().downcast_ref::() } } diff --git a/datafusion/datasource/src/test_util.rs b/datafusion/datasource/src/test_util.rs index e4a5114aa073..2f7246444fa5 100644 --- a/datafusion/datasource/src/test_util.rs +++ b/datafusion/datasource/src/test_util.rs @@ -22,25 +22,52 @@ use crate::{ use std::sync::Arc; -use arrow::datatypes::{Schema, SchemaRef}; -use datafusion_common::{Result, Statistics}; +use crate::source::DataSource; +use arrow::datatypes::Schema; +use datafusion_common::Result; use datafusion_physical_expr::{expressions::Column, PhysicalExpr}; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion_physical_plan::DisplayFormatType; use object_store::ObjectStore; /// Minimal [`crate::file::FileSource`] implementation for use in tests. -#[derive(Clone, Default)] +#[derive(Clone)] pub(crate) struct MockSource { metrics: ExecutionPlanMetricsSet, - projected_statistics: Option, schema_adapter_factory: Option>, + + config: FileScanConfig, +} + +impl MockSource { + pub fn new(config: FileScanConfig) -> Self { + Self { + metrics: Default::default(), + schema_adapter_factory: None, + config, + } + } } impl FileSource for MockSource { + fn config(&self) -> &FileScanConfig { + &self.config + } + + fn with_config(&self, config: FileScanConfig) -> Arc { + let mut this = self.clone(); + this.config = config; + + Arc::new(this) + } + + fn as_data_source(&self) -> Arc { + Arc::new(self.clone()) + } + fn create_file_opener( &self, _object_store: Arc, - _base_config: &FileScanConfig, _partition: usize, ) -> Arc { unimplemented!() @@ -50,36 +77,10 @@ impl FileSource for MockSource { self } - fn with_batch_size(&self, _batch_size: usize) -> Arc { - Arc::new(Self { ..self.clone() }) - } - - fn with_schema(&self, _schema: SchemaRef) -> Arc { - Arc::new(Self { ..self.clone() }) - } - - fn with_projection(&self, _config: &FileScanConfig) -> Arc { - Arc::new(Self { ..self.clone() }) - } - - fn with_statistics(&self, statistics: Statistics) -> Arc { - let mut source = self.clone(); - source.projected_statistics = Some(statistics); - Arc::new(source) - } - - fn metrics(&self) -> &ExecutionPlanMetricsSet { + fn metrics_inner(&self) -> &ExecutionPlanMetricsSet { &self.metrics } - fn statistics(&self) -> Result { - Ok(self - .projected_statistics - .as_ref() - .expect("projected_statistics must be set") - .clone()) - } - fn file_type(&self) -> &str { "mock" } @@ -99,6 +100,15 @@ impl FileSource for MockSource { } } +impl std::fmt::Debug for MockSource { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{} {{ ", self.file_type())?; + write!(f, "statistics={:?}, ", self.file_source_statistics())?; + ::fmt_as(self, DisplayFormatType::Verbose, f)?; + write!(f, " }}") + } +} + /// Create a column expression pub(crate) fn col(name: &str, schema: &Schema) -> Result> { Ok(Arc::new(Column::new_with_schema(name, schema)?)) diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 39ad52d46a80..f4299d89fca5 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -34,7 +34,7 @@ use datafusion::datasource::file_format::parquet::ParquetSink; use datafusion::datasource::listing::{FileRange, ListingTableUrl, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{ - FileGroup, FileScanConfig, FileScanConfigBuilder, FileSinkConfig, FileSource, + FileGroup, FileScanConfig, FileScanConfigBuilder, FileSinkConfig, }; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::WindowFunctionDefinition; @@ -490,7 +490,6 @@ pub fn parse_protobuf_file_scan_config( proto: &protobuf::FileScanExecConf, ctx: &SessionContext, codec: &dyn PhysicalExtensionCodec, - file_source: Arc, ) -> Result { let schema: Arc = parse_protobuf_file_scan_schema(proto)?; let projection = proto @@ -543,10 +542,10 @@ pub fn parse_protobuf_file_scan_config( output_ordering.extend(LexOrdering::new(sort_exprs)); } - let config = FileScanConfigBuilder::new(object_store_url, file_schema, file_source) + let config = FileScanConfigBuilder::new(object_store_url, file_schema) .with_file_groups(file_groups) .with_constraints(constraints) - .with_statistics(statistics) + .with_file_source_projected_statistics(statistics) .with_projection(Some(projection)) .with_limit(proto.limit.as_ref().map(|sl| sl.limit as usize)) .with_table_partition_cols(table_partition_cols) diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index fb86e380557f..b38622f15f27 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -50,7 +50,7 @@ use datafusion::datasource::physical_plan::AvroSource; #[cfg(feature = "parquet")] use datafusion::datasource::physical_plan::ParquetSource; use datafusion::datasource::physical_plan::{ - CsvSource, FileScanConfig, FileScanConfigBuilder, JsonSource, + CsvSource, FileScanConfigBuilder, FileSource, JsonSource, }; use datafusion::datasource::sink::DataSinkExec; use datafusion::datasource::source::DataSourceExec; @@ -646,26 +646,25 @@ impl protobuf::PhysicalPlanNode { None }; - let source = Arc::new( - CsvSource::new( - scan.has_header, - str_to_byte(&scan.delimiter, "delimiter")?, - 0, - ) - .with_escape(escape) - .with_comment(comment), - ); - let conf = FileScanConfigBuilder::from(parse_protobuf_file_scan_config( scan.base_conf.as_ref().unwrap(), ctx, extension_codec, - source, )?) .with_newlines_in_values(scan.newlines_in_values) .with_file_compression_type(FileCompressionType::UNCOMPRESSED) .build(); - Ok(DataSourceExec::from_data_source(conf)) + + let source = CsvSource::new( + scan.has_header, + str_to_byte(&scan.delimiter, "delimiter")?, + 0, + conf.clone(), + ) + .with_escape(escape) + .with_comment(comment); + + Ok(DataSourceExec::from_data_source(source)) } fn try_into_json_scan_physical_plan( @@ -679,9 +678,11 @@ impl protobuf::PhysicalPlanNode { scan.base_conf.as_ref().unwrap(), ctx, extension_codec, - Arc::new(JsonSource::new()), )?; - Ok(DataSourceExec::from_data_source(scan_conf)) + + let source = JsonSource::new(scan_conf.clone()); + + Ok(DataSourceExec::from_data_source(source)) } #[cfg_attr(not(feature = "parquet"), allow(unused_variables))] @@ -728,18 +729,17 @@ impl protobuf::PhysicalPlanNode { if let Some(table_options) = scan.parquet_options.as_ref() { options = table_options.try_into()?; } - let mut source = ParquetSource::new(options); + + let base_config = + parse_protobuf_file_scan_config(base_conf, ctx, extension_codec)?; + + let mut source = ParquetSource::new(options, base_config.clone()); if let Some(predicate) = predicate { source = source.with_predicate(predicate); } - let base_config = parse_protobuf_file_scan_config( - base_conf, - ctx, - extension_codec, - Arc::new(source), - )?; - Ok(DataSourceExec::from_data_source(base_config)) + + Ok(DataSourceExec::from_data_source(source)) } #[cfg(not(feature = "parquet"))] panic!("Unable to process a Parquet PhysicalPlan when `parquet` feature is not enabled") @@ -759,9 +759,10 @@ impl protobuf::PhysicalPlanNode { scan.base_conf.as_ref().unwrap(), ctx, extension_codec, - Arc::new(AvroSource::new()), )?; - Ok(DataSourceExec::from_data_source(conf)) + + let source = AvroSource::new(conf.clone()); + Ok(DataSourceExec::from_data_source(source)) } #[cfg(not(feature = "avro"))] panic!("Unable to process a Avro PhysicalPlan when `avro` feature is not enabled") @@ -2315,67 +2316,56 @@ impl protobuf::PhysicalPlanNode { extension_codec: &dyn PhysicalExtensionCodec, ) -> Result> { let data_source = data_source_exec.data_source(); - if let Some(maybe_csv) = data_source.as_any().downcast_ref::() { - let source = maybe_csv.file_source(); - if let Some(csv_config) = source.as_any().downcast_ref::() { - return Ok(Some(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::CsvScan( - protobuf::CsvScanExecNode { - base_conf: Some(serialize_file_scan_config( - maybe_csv, - extension_codec, - )?), - has_header: csv_config.has_header(), - delimiter: byte_to_string( - csv_config.delimiter(), - "delimiter", - )?, - quote: byte_to_string(csv_config.quote(), "quote")?, - optional_escape: if let Some(escape) = csv_config.escape() { - Some( - protobuf::csv_scan_exec_node::OptionalEscape::Escape( - byte_to_string(escape, "escape")?, - ), - ) - } else { - None - }, - optional_comment: if let Some(comment) = csv_config.comment() - { - Some(protobuf::csv_scan_exec_node::OptionalComment::Comment( - byte_to_string(comment, "comment")?, - )) - } else { - None - }, - newlines_in_values: maybe_csv.newlines_in_values(), + if let Some(csv_source) = data_source.as_any().downcast_ref::() { + return Ok(Some(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::CsvScan( + protobuf::CsvScanExecNode { + base_conf: Some(serialize_file_scan_config( + csv_source.config(), + csv_source.file_source_statistics(), + extension_codec, + )?), + has_header: csv_source.has_header(), + delimiter: byte_to_string(csv_source.delimiter(), "delimiter")?, + quote: byte_to_string(csv_source.quote(), "quote")?, + optional_escape: if let Some(escape) = csv_source.escape() { + Some(protobuf::csv_scan_exec_node::OptionalEscape::Escape( + byte_to_string(escape, "escape")?, + )) + } else { + None }, - )), - })); - } + optional_comment: if let Some(comment) = csv_source.comment() { + Some(protobuf::csv_scan_exec_node::OptionalComment::Comment( + byte_to_string(comment, "comment")?, + )) + } else { + None + }, + newlines_in_values: csv_source.config().newlines_in_values(), + }, + )), + })); } - if let Some(scan_conf) = data_source.as_any().downcast_ref::() { - let source = scan_conf.file_source(); - if let Some(_json_source) = source.as_any().downcast_ref::() { - return Ok(Some(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::JsonScan( - protobuf::JsonScanExecNode { - base_conf: Some(serialize_file_scan_config( - scan_conf, - extension_codec, - )?), - }, - )), - })); - } + if let Some(json_source) = data_source.as_any().downcast_ref::() { + return Ok(Some(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::JsonScan( + protobuf::JsonScanExecNode { + base_conf: Some(serialize_file_scan_config( + json_source.config(), + json_source.file_source_statistics(), + extension_codec, + )?), + }, + )), + })); } #[cfg(feature = "parquet")] - if let Some((maybe_parquet, conf)) = - data_source_exec.downcast_to_file_source::() + if let Some(parquet_source) = data_source.as_any().downcast_ref::() { - let predicate = conf + let predicate = parquet_source .predicate() .map(|pred| serialize_physical_expr(pred, extension_codec)) .transpose()?; @@ -2383,31 +2373,32 @@ impl protobuf::PhysicalPlanNode { physical_plan_type: Some(PhysicalPlanType::ParquetScan( protobuf::ParquetScanExecNode { base_conf: Some(serialize_file_scan_config( - maybe_parquet, + parquet_source.config(), + parquet_source.file_source_statistics(), extension_codec, )?), predicate, - parquet_options: Some(conf.table_parquet_options().try_into()?), + parquet_options: Some( + parquet_source.table_parquet_options().try_into()?, + ), }, )), })); } #[cfg(feature = "avro")] - if let Some(maybe_avro) = data_source.as_any().downcast_ref::() { - let source = maybe_avro.file_source(); - if source.as_any().downcast_ref::().is_some() { - return Ok(Some(protobuf::PhysicalPlanNode { - physical_plan_type: Some(PhysicalPlanType::AvroScan( - protobuf::AvroScanExecNode { - base_conf: Some(serialize_file_scan_config( - maybe_avro, - extension_codec, - )?), - }, - )), - })); - } + if let Some(avro_source) = data_source.as_any().downcast_ref::() { + return Ok(Some(protobuf::PhysicalPlanNode { + physical_plan_type: Some(PhysicalPlanType::AvroScan( + protobuf::AvroScanExecNode { + base_conf: Some(serialize_file_scan_config( + avro_source.config(), + avro_source.file_source_statistics(), + extension_codec, + )?), + }, + )), + })); } Ok(None) diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 64960e39f75d..1fb5300bf3e6 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -496,6 +496,7 @@ impl TryFrom<&[PartitionedFile]> for protobuf::FileGroup { pub fn serialize_file_scan_config( conf: &FileScanConfig, + file_source_statistics: datafusion_common::Statistics, codec: &dyn PhysicalExtensionCodec, ) -> Result { let file_groups = conf @@ -523,7 +524,7 @@ pub fn serialize_file_scan_config( Ok(protobuf::FileScanExecConf { file_groups, - statistics: Some((&conf.file_source.statistics().unwrap()).into()), + statistics: Some((&file_source_statistics).into()), limit: conf.limit.map(|l| protobuf::ScanLimit { limit: l as u32 }), projection: conf .projection diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 1547b7087d5e..f90152c033d7 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -49,7 +49,7 @@ use datafusion::datasource::listing::{ListingTableUrl, PartitionedFile}; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{ wrap_partition_type_in_dict, wrap_partition_value_in_dict, FileGroup, - FileScanConfigBuilder, FileSinkConfig, FileSource, ParquetSource, + FileScanConfigBuilder, FileSinkConfig, ParquetSource, }; use datafusion::datasource::sink::DataSinkExec; use datafusion::datasource::source::DataSourceExec; @@ -871,27 +871,25 @@ fn roundtrip_parquet_exec_with_pruning_predicate() -> Result<()> { let mut options = TableParquetOptions::new(); options.global.pushdown_filters = true; - let file_source = Arc::new(ParquetSource::new(options).with_predicate(predicate)); + let scan_config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_schema) + .with_file_groups(vec![FileGroup::new(vec![PartitionedFile::new( + "/path/to/file.parquet".to_string(), + 1024, + )])]) + .with_file_source_projected_statistics(Statistics { + num_rows: Precision::Inexact(100), + total_byte_size: Precision::Inexact(1024), + column_statistics: Statistics::unknown_column(&Arc::new(Schema::new( + vec![Field::new("col", DataType::Utf8, false)], + ))), + }) + .build(); - let scan_config = FileScanConfigBuilder::new( - ObjectStoreUrl::local_filesystem(), - file_schema, - file_source, - ) - .with_file_groups(vec![FileGroup::new(vec![PartitionedFile::new( - "/path/to/file.parquet".to_string(), - 1024, - )])]) - .with_statistics(Statistics { - num_rows: Precision::Inexact(100), - total_byte_size: Precision::Inexact(1024), - column_statistics: Statistics::unknown_column(&Arc::new(Schema::new(vec![ - Field::new("col", DataType::Utf8, false), - ]))), - }) - .build(); - - roundtrip_test(DataSourceExec::from_data_source(scan_config)) + let file_source = + ParquetSource::new(options, scan_config.clone()).with_predicate(predicate); + + roundtrip_test(DataSourceExec::from_data_source(file_source)) } #[tokio::test] @@ -902,23 +900,21 @@ async fn roundtrip_parquet_exec_with_table_partition_cols() -> Result<()> { vec![wrap_partition_value_in_dict(ScalarValue::Int64(Some(0)))]; let schema = Arc::new(Schema::new(vec![Field::new("col", DataType::Utf8, false)])); - let file_source = Arc::new(ParquetSource::default()); - let scan_config = FileScanConfigBuilder::new( - ObjectStoreUrl::local_filesystem(), - schema, - file_source, - ) - .with_projection(Some(vec![0, 1])) - .with_file_group(FileGroup::new(vec![file_group])) - .with_table_partition_cols(vec![Field::new( - "part".to_string(), - wrap_partition_type_in_dict(DataType::Int16), - false, - )]) - .with_newlines_in_values(false) - .build(); + let scan_config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), schema) + .with_projection(Some(vec![0, 1])) + .with_file_group(FileGroup::new(vec![file_group])) + .with_table_partition_cols(vec![Field::new( + "part".to_string(), + wrap_partition_type_in_dict(DataType::Int16), + false, + )]) + .with_newlines_in_values(false) + .build(); - roundtrip_test(DataSourceExec::from_data_source(scan_config)) + let file_source = + ParquetSource::new(TableParquetOptions::default(), scan_config.clone()); + roundtrip_test(DataSourceExec::from_data_source(file_source)) } #[test] @@ -930,26 +926,20 @@ fn roundtrip_parquet_exec_with_custom_predicate_expr() -> Result<()> { inner: Arc::new(Column::new("col", 1)), }); - let file_source = - Arc::new(ParquetSource::default().with_predicate(custom_predicate_expr)); - - let scan_config = FileScanConfigBuilder::new( - ObjectStoreUrl::local_filesystem(), - file_schema, - file_source, - ) - .with_file_groups(vec![FileGroup::new(vec![PartitionedFile::new( - "/path/to/file.parquet".to_string(), - 1024, - )])]) - .with_statistics(Statistics { - num_rows: Precision::Inexact(100), - total_byte_size: Precision::Inexact(1024), - column_statistics: Statistics::unknown_column(&Arc::new(Schema::new(vec![ - Field::new("col", DataType::Utf8, false), - ]))), - }) - .build(); + let scan_config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), file_schema) + .with_file_groups(vec![FileGroup::new(vec![PartitionedFile::new( + "/path/to/file.parquet".to_string(), + 1024, + )])]) + .with_file_source_projected_statistics(Statistics { + num_rows: Precision::Inexact(100), + total_byte_size: Precision::Inexact(1024), + column_statistics: Statistics::unknown_column(&Arc::new(Schema::new( + vec![Field::new("col", DataType::Utf8, false)], + ))), + }) + .build(); #[derive(Debug, Clone, Eq)] struct CustomPredicateExpr { @@ -1060,7 +1050,11 @@ fn roundtrip_parquet_exec_with_custom_predicate_expr() -> Result<()> { } } - let exec_plan = DataSourceExec::from_data_source(scan_config); + let file_source = + ParquetSource::new(TableParquetOptions::default(), scan_config.clone()) + .with_predicate(custom_predicate_expr); + + let exec_plan = DataSourceExec::from_data_source(file_source); let ctx = SessionContext::new(); roundtrip_test_and_return(exec_plan, &ctx, &CustomPhysicalExtensionCodec {})?; @@ -1788,24 +1782,23 @@ async fn roundtrip_projection_source() -> Result<()> { let statistics = Statistics::new_unknown(&schema); - let file_source = ParquetSource::default().with_statistics(statistics.clone()); - let scan_config = FileScanConfigBuilder::new( - ObjectStoreUrl::local_filesystem(), - schema.clone(), - file_source, - ) - .with_file_groups(vec![FileGroup::new(vec![PartitionedFile::new( - "/path/to/file.parquet".to_string(), - 1024, - )])]) - .with_statistics(statistics) - .with_projection(Some(vec![0, 1, 2])) - .build(); + let scan_config = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), schema.clone()) + .with_file_groups(vec![FileGroup::new(vec![PartitionedFile::new( + "/path/to/file.parquet".to_string(), + 1024, + )])]) + .with_file_source_projected_statistics(statistics) + .with_projection(Some(vec![0, 1, 2])) + .build(); + let file_source = + ParquetSource::new(TableParquetOptions::default(), scan_config.clone()); + let exec = DataSourceExec::from_data_source(file_source); let filter = Arc::new( FilterExec::try_new( Arc::new(BinaryExpr::new(col("c", &schema)?, Operator::Eq, lit(1))), - DataSourceExec::from_data_source(scan_config), + exec, )? .with_projection(Some(vec![0, 1]))?, ); diff --git a/datafusion/substrait/src/physical_plan/consumer.rs b/datafusion/substrait/src/physical_plan/consumer.rs index 4990054ac7fc..e510427a3d8b 100644 --- a/datafusion/substrait/src/physical_plan/consumer.rs +++ b/datafusion/substrait/src/physical_plan/consumer.rs @@ -35,6 +35,7 @@ use crate::variation_const::{ }; use async_recursion::async_recursion; use chrono::DateTime; +use datafusion::config::TableParquetOptions; use datafusion::datasource::memory::DataSourceExec; use object_store::ObjectMeta; use substrait::proto::r#type::{Kind, Nullability}; @@ -53,7 +54,6 @@ pub async fn from_substrait_rel( ) -> Result> { let mut base_config_builder; - let source = Arc::new(ParquetSource::default()); match &rel.rel_type { Some(RelType::Read(read)) => { if read.filter.is_some() || read.best_effort_filter.is_some() { @@ -83,7 +83,6 @@ pub async fn from_substrait_rel( base_config_builder = FileScanConfigBuilder::new( ObjectStoreUrl::local_filesystem(), Arc::new(Schema::new(fields)), - source, ); } Err(e) => return Err(e), @@ -156,10 +155,13 @@ pub async fn from_substrait_rel( } } - Ok( - DataSourceExec::from_data_source(base_config_builder.build()) - as Arc, - ) + let conf = base_config_builder.build(); + + let source = + ParquetSource::new(TableParquetOptions::default(), conf.clone()); + + Ok(DataSourceExec::from_data_source(source) + as Arc) } _ => not_impl_err!( "Only LocalFile reads are supported when parsing physical" diff --git a/datafusion/substrait/src/physical_plan/producer.rs b/datafusion/substrait/src/physical_plan/producer.rs index cb725a7277fd..c5e4a70076dd 100644 --- a/datafusion/substrait/src/physical_plan/producer.rs +++ b/datafusion/substrait/src/physical_plan/producer.rs @@ -27,7 +27,7 @@ use datafusion::datasource::source::DataSourceExec; use datafusion::error::{DataFusionError, Result}; use datafusion::physical_plan::{displayable, ExecutionPlan}; -use datafusion::datasource::physical_plan::ParquetSource; +use datafusion::datasource::physical_plan::{FileSource, ParquetSource}; use substrait::proto::expression::mask_expression::{StructItem, StructSelect}; use substrait::proto::expression::MaskExpression; use substrait::proto::r#type::{ @@ -52,9 +52,10 @@ pub fn to_substrait_rel( ), ) -> Result> { if let Some(data_source_exec) = plan.as_any().downcast_ref::() { - if let Some((file_config, _)) = + if let Some(parquet_source) = data_source_exec.downcast_to_file_source::() { + let file_config = parquet_source.config(); let mut substrait_files = vec![]; for (partition_index, files) in file_config.file_groups.iter().enumerate() { for file in files.iter() { diff --git a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs index 64599465f96f..c0c1f407e852 100644 --- a/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_physical_plan.rs @@ -30,17 +30,15 @@ use datafusion::physical_plan::{displayable, ExecutionPlan}; use datafusion::prelude::{ParquetReadOptions, SessionContext}; use datafusion_substrait::physical_plan::{consumer, producer}; +use datafusion::config::TableParquetOptions; use datafusion::datasource::memory::DataSourceExec; use substrait::proto::extensions; #[tokio::test] async fn parquet_exec() -> Result<()> { - let source = Arc::new(ParquetSource::default()); - let scan_config = FileScanConfigBuilder::new( ObjectStoreUrl::local_filesystem(), Arc::new(Schema::empty()), - source, ) .with_file_groups(vec![ FileGroup::new(vec![PartitionedFile::new( @@ -53,8 +51,9 @@ async fn parquet_exec() -> Result<()> { )]), ]) .build(); - let parquet_exec: Arc = - DataSourceExec::from_data_source(scan_config); + + let source = ParquetSource::new(TableParquetOptions::default(), scan_config.clone()); + let parquet_exec: Arc = DataSourceExec::from_data_source(source); let mut extension_info: ( Vec, diff --git a/docs/source/library-user-guide/upgrading.md b/docs/source/library-user-guide/upgrading.md index 05c0de011868..b23b6ff32234 100644 --- a/docs/source/library-user-guide/upgrading.md +++ b/docs/source/library-user-guide/upgrading.md @@ -774,7 +774,7 @@ let config = FileScanConfigBuilder::new(url, schema, Arc::new(file_source)) .with_statistics(stats) ... .build(); -let scan = DataSourceExec::from_data_source(config); +let scan =DataSourceExec::from_data_source(source); # */ ```