From b23cd58644b1dc5964815db3bfd6ef2065246557 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 15:08:04 +0800 Subject: [PATCH 01/38] Implement schema adapter factory support for file sources --- datafusion/core/src/datasource/mod.rs | 8 ++-- .../datasource/physical_plan/arrow_file.rs | 24 +++++++++++ .../filter_pushdown/util.rs | 40 +++++++++++++------ datafusion/datasource-avro/src/source.rs | 16 ++++++++ 4 files changed, 71 insertions(+), 17 deletions(-) diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 674541ff73a5c..273ac2be8fbf3 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -52,7 +52,7 @@ pub use datafusion_physical_expr::create_ordering; mod tests { use crate::prelude::SessionContext; - + use datafusion_datasource::file::FileSource; use std::fs; use std::sync::Arc; @@ -124,10 +124,8 @@ mod tests { let f2 = Field::new("extra_column", DataType::Utf8, true); let schema = Arc::new(Schema::new(vec![f1.clone(), f2.clone()])); - let source = Arc::new( - ParquetSource::default() - .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})), - ); + let source = ParquetSource::default() + .with_schema_adapter_factory(Arc::new(TestSchemaAdapterFactory {})); let base_conf = FileScanConfigBuilder::new( ObjectStoreUrl::local_filesystem(), schema, diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 897d1c04471c1..f15cda44a7eec 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -20,6 +20,8 @@ use std::sync::Arc; use crate::datasource::physical_plan::{FileMeta, FileOpenFuture, FileOpener}; use crate::error::Result; +use datafusion_datasource::as_file_source; +use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use arrow::buffer::Buffer; use arrow::datatypes::SchemaRef; @@ -39,8 +41,14 @@ use object_store::{GetOptions, GetRange, GetResultPayload, ObjectStore}; pub struct ArrowSource { metrics: ExecutionPlanMetricsSet, projected_statistics: Option, + schema_adapter_factory: Option>, } +impl From for Arc { + fn from(source: ArrowSource) -> Self { + as_file_source(source) + } + impl FileSource for ArrowSource { fn create_file_opener( &self, @@ -89,8 +97,24 @@ impl FileSource for ArrowSource { fn file_type(&self) -> &str { "arrow" } + + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Arc { + Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + }) + } + + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() + } } + + /// The struct arrow that implements `[FileOpener]` trait pub struct ArrowOpener { pub object_store: Arc, diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index 393322a7f3e2e..4e611b03fa777 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -31,15 +31,19 @@ use arrow::{array::RecordBatch, compute::concat_batches}; use datafusion::{datasource::object_store::ObjectStoreUrl, physical_plan::PhysicalExpr}; use datafusion_common::{config::ConfigOptions, Statistics}; use datafusion_common::{internal_err, Result}; -use datafusion_datasource::file_scan_config::FileScanConfigBuilder; -use datafusion_datasource::file_stream::FileOpenFuture; -use datafusion_datasource::source::DataSourceExec; use datafusion_datasource::{ file::FileSource, file_scan_config::FileScanConfig, file_stream::FileOpener, }; use datafusion_datasource::{ file_meta::FileMeta, schema_adapter::DefaultSchemaAdapterFactory, PartitionedFile, }; +use datafusion_datasource::{ + file_scan_config::FileScanConfigBuilder, file_stream::FileOpenFuture, + source::DataSourceExec, +}; +use datafusion_datasource::{ + file_scan_config::FileScanConfigBuilder, schema_adapter::SchemaAdapterFactory, +}; use datafusion_physical_expr::conjunction; use datafusion_physical_expr_common::physical_expr::fmt_sql; use datafusion_physical_optimizer::PhysicalOptimizerRule; @@ -119,19 +123,17 @@ pub struct TestSource { schema: Option, metrics: ExecutionPlanMetricsSet, projection: Option>, + schema_adapter_factory: Option>, } impl TestSource { fn new(support: bool, batches: Vec) -> Self { - Self { - support, - predicate: None, - statistics: None, - batch_size: None, - schema: None, - projection: None, - metrics: ExecutionPlanMetricsSet::new(), - batches, + fn new(support: bool, batches: Vec) -> Self { + Self { + support, + batches, + ..Default::default() + } } } } @@ -243,6 +245,20 @@ impl FileSource for TestSource { Ok(FilterPushdownPropagation::unsupported(filters)) } } + + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Arc { + Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + }) + } + + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() + } } #[derive(Debug, Clone)] diff --git a/datafusion/datasource-avro/src/source.rs b/datafusion/datasource-avro/src/source.rs index abd049a50e26d..f40a0cb021c61 100644 --- a/datafusion/datasource-avro/src/source.rs +++ b/datafusion/datasource-avro/src/source.rs @@ -28,6 +28,7 @@ use datafusion_common::Statistics; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::file_stream::FileOpener; +use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; @@ -41,6 +42,7 @@ pub struct AvroSource { projection: Option>, metrics: ExecutionPlanMetricsSet, projected_statistics: Option, + schema_adapter_factory: Option>, } impl AvroSource { @@ -123,6 +125,20 @@ impl FileSource for AvroSource { ) -> Result> { Ok(None) } + + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Arc { + Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + }) + } + + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() + } } mod private { From cd51b5abe314f2203d37196014083c95b760c6e5 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 15:25:48 +0800 Subject: [PATCH 02/38] Add schema adapter factory support to file sources --- datafusion/datasource-csv/src/source.rs | 24 ++++++++++++++++- datafusion/datasource-json/src/source.rs | 26 +++++++++++++++++- .../datasource-parquet/src/file_format.rs | 27 ++++++++++++++++++- datafusion/datasource-parquet/src/source.rs | 21 +++++++++++++++ datafusion/datasource/src/file.rs | 18 +++++++++++++ datafusion/datasource/src/mod.rs | 1 + 6 files changed, 114 insertions(+), 3 deletions(-) diff --git a/datafusion/datasource-csv/src/source.rs b/datafusion/datasource-csv/src/source.rs index cbadb5dd91af2..81c9eef0ffacf 100644 --- a/datafusion/datasource-csv/src/source.rs +++ b/datafusion/datasource-csv/src/source.rs @@ -17,6 +17,7 @@ //! Execution plan for reading CSV files +use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use std::any::Any; use std::fmt; use std::io::{Read, Seek, SeekFrom}; @@ -28,7 +29,7 @@ use datafusion_datasource::file_compression_type::FileCompressionType; use datafusion_datasource::file_meta::FileMeta; use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; use datafusion_datasource::{ - calculate_range, FileRange, ListingTableUrl, RangeCalculation, + as_file_source, calculate_range, FileRange, ListingTableUrl, RangeCalculation, }; use arrow::csv; @@ -91,6 +92,7 @@ pub struct CsvSource { comment: Option, metrics: ExecutionPlanMetricsSet, projected_statistics: Option, + schema_adapter_factory: Option>, } impl CsvSource { @@ -212,6 +214,12 @@ impl CsvOpener { } } +impl From for Arc { + fn from(source: CsvSource) -> Self { + as_file_source(source) + } +} + impl FileSource for CsvSource { fn create_file_opener( &self, @@ -274,6 +282,20 @@ impl FileSource for CsvSource { DisplayFormatType::TreeRender => Ok(()), } } + + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Arc { + Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + }) + } + + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() + } } impl FileOpener for CsvOpener { diff --git a/datafusion/datasource-json/src/source.rs b/datafusion/datasource-json/src/source.rs index 982b799556ab0..4b1e7dcd74ad8 100644 --- a/datafusion/datasource-json/src/source.rs +++ b/datafusion/datasource-json/src/source.rs @@ -30,7 +30,10 @@ 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::{calculate_range, ListingTableUrl, RangeCalculation}; +use datafusion_datasource::schema_adapter::SchemaAdapterFactory; +use datafusion_datasource::{ + as_file_source, calculate_range, ListingTableUrl, RangeCalculation, +}; use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow::json::ReaderBuilder; @@ -52,6 +55,7 @@ pub struct JsonOpener { projected_schema: SchemaRef, file_compression_type: FileCompressionType, object_store: Arc, + schema_adapter_factory: Option>, } impl JsonOpener { @@ -86,6 +90,12 @@ impl JsonSource { } } +impl From for Arc { + fn from(source: JsonSource) -> Self { + as_file_source(source) + } +} + impl FileSource for JsonSource { fn create_file_opener( &self, @@ -140,6 +150,20 @@ impl FileSource for JsonSource { fn file_type(&self) -> &str { "json" } + + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Arc { + Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + }) + } + + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() + } } impl FileOpener for JsonOpener { diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 83013e5c97835..7de8d139dc586 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -420,9 +420,11 @@ impl FileFormat for ParquetFormat { if let Some(metadata_size_hint) = metadata_size_hint { source = source.with_metadata_size_hint(metadata_size_hint) } + // Apply schema adapter factory before building the new config + let file_source = apply_schema_adapter(source, &conf); let conf = FileScanConfigBuilder::from(conf) - .with_source(Arc::new(source)) + .with_source(file_source) .build(); Ok(DataSourceExec::from_data_source(conf)) } @@ -1576,3 +1578,26 @@ fn create_max_min_accs( .collect(); (max_values, min_values) } + +/// Converts a ParquetSource to an Arc and applies the schema adapter factory +/// from the FileScanConfig if present. +/// +/// # Arguments +/// * `source` - The ParquetSource to convert +/// * `conf` - FileScanConfig that may contain a schema adapter factory +/// # Returns +/// The converted FileSource with schema adapter factory applied if provided +fn apply_schema_adapter( + source: ParquetSource, + conf: &FileScanConfig, +) -> Arc { + // Convert the ParquetSource to Arc + let file_source: Arc = source.into(); + + // If the FileScanConfig.file_source() has a schema adapter factory, apply it + if let Some(factory) = conf.file_source().schema_adapter_factory() { + file_source.with_schema_adapter_factory(factory.clone()) + } else { + file_source + } +} diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 13684db8ea150..05ad547a0c21c 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -446,6 +446,13 @@ pub(crate) fn parse_coerce_int96_string( } } +/// Allows easy conversion from ParquetSource to Arc +impl From for Arc { + fn from(source: ParquetSource) -> Self { + as_file_source(source) + } +} + impl FileSource for ParquetSource { fn create_file_opener( &self, @@ -656,4 +663,18 @@ impl FileSource for ParquetSource { ); Ok(FilterPushdownPropagation::with_filters(filters).with_updated_node(source)) } + + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Arc { + Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + }) + } + + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() + } } diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index c9b5c416f0c0e..6afe534b963b6 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -35,6 +35,11 @@ use datafusion_physical_plan::DisplayFormatType; use object_store::ObjectStore; +/// Helper function to convert any type implementing FileSource to Arc +pub fn as_file_source(source: T) -> Arc { + Arc::new(source) +} + /// file format specific behaviors for elements in [`DataSource`] /// /// See more details on specific implementations: @@ -116,4 +121,17 @@ pub trait FileSource: Send + Sync { ) -> Result>> { Ok(FilterPushdownPropagation::unsupported(filters)) } + + /// Set optional schema adapter factory. + /// + /// [`SchemaAdapterFactory`] allows user to specify how fields from the + /// file get mapped to that of the table schema. The default implementation + /// returns the original source. + fn with_schema_adapter_factory( + &self, + factory: Arc, + ) -> Arc; + + /// Returns the current schema adapter factory if set + fn schema_adapter_factory(&self) -> Option>; } diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index 9e83adc6b9fed..c79efd11fcc5e 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -48,6 +48,7 @@ pub mod test_util; pub mod url; pub mod write; +pub use self::file::as_file_source; pub use self::url::ListingTableUrl; use crate::file_groups::FileGroup; use chrono::TimeZone; From 223345638de54abd250ff678e881c77d5b2bc97c Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 15:26:58 +0800 Subject: [PATCH 03/38] Add SchemaAdapterFactory import to file source module --- datafusion/datasource/src/file.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 6afe534b963b6..04b6b6975a766 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -25,6 +25,7 @@ use std::sync::Arc; use crate::file_groups::FileGroupPartitioner; use crate::file_scan_config::FileScanConfig; use crate::file_stream::FileOpener; +use crate::schema_adapter::SchemaAdapterFactory; use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_common::{Result, Statistics}; From 60ff7e6f3bfe4652dba00ccc69d24a8b725e0961 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 15:28:42 +0800 Subject: [PATCH 04/38] Add schema_adapter_factory field to JsonOpener and JsonSource structs --- datafusion/datasource-json/src/source.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/datasource-json/src/source.rs b/datafusion/datasource-json/src/source.rs index 4b1e7dcd74ad8..5cd06aa776734 100644 --- a/datafusion/datasource-json/src/source.rs +++ b/datafusion/datasource-json/src/source.rs @@ -55,7 +55,6 @@ pub struct JsonOpener { projected_schema: SchemaRef, file_compression_type: FileCompressionType, object_store: Arc, - schema_adapter_factory: Option>, } impl JsonOpener { @@ -81,6 +80,7 @@ pub struct JsonSource { batch_size: Option, metrics: ExecutionPlanMetricsSet, projected_statistics: Option, + schema_adapter_factory: Option>, } impl JsonSource { From 4d6f8f788aacac1824e77b27b84503175a197dbe Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 15:29:37 +0800 Subject: [PATCH 05/38] Add missing import for as_file_source in source.rs --- datafusion/datasource-parquet/src/source.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 05ad547a0c21c..4ff3e32a33736 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -27,6 +27,7 @@ use crate::row_filter::can_expr_be_pushed_down_with_schemas; use crate::DefaultParquetFileReaderFactory; use crate::ParquetFileReaderFactory; use datafusion_common::config::ConfigOptions; +use datafusion_datasource::as_file_source; use datafusion_datasource::file_stream::FileOpener; use datafusion_datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapterFactory, From 011ce03eb9e675208d4c8653f15e61ee7adba257 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 15:30:53 +0800 Subject: [PATCH 06/38] Fix formatting in ArrowSource implementation by removing extra newlines --- datafusion/core/src/datasource/physical_plan/arrow_file.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index f15cda44a7eec..fb10a96473805 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -48,6 +48,7 @@ impl From for Arc { fn from(source: ArrowSource) -> Self { as_file_source(source) } +} impl FileSource for ArrowSource { fn create_file_opener( @@ -113,8 +114,6 @@ impl FileSource for ArrowSource { } } - - /// The struct arrow that implements `[FileOpener]` trait pub struct ArrowOpener { pub object_store: Arc, From 7fde3967c3a487d50c32a84e92cf8f92061ca221 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 15:36:29 +0800 Subject: [PATCH 07/38] Add integration and unit tests for schema adapter factory functionality --- .../schema_adapter_integration_tests.rs | 197 ++++++++++++++++++ .../tests/schema_adapter_factory_tests.rs | 167 +++++++++++++++ .../core/tests/test_source_adapter_tests.rs | 109 ++++++++++ .../tests/apply_schema_adapter_tests.rs | 147 +++++++++++++ 4 files changed, 620 insertions(+) create mode 100644 datafusion/core/tests/integration_tests/schema_adapter_integration_tests.rs create mode 100644 datafusion/core/tests/schema_adapter_factory_tests.rs create mode 100644 datafusion/core/tests/test_source_adapter_tests.rs create mode 100644 datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs diff --git a/datafusion/core/tests/integration_tests/schema_adapter_integration_tests.rs b/datafusion/core/tests/integration_tests/schema_adapter_integration_tests.rs new file mode 100644 index 0000000000000..98ca808eebf4e --- /dev/null +++ b/datafusion/core/tests/integration_tests/schema_adapter_integration_tests.rs @@ -0,0 +1,197 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Integration test for schema adapter factory functionality + +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; +use datafusion::datasource::object_store::ObjectStoreUrl; +use datafusion::datasource::physical_plan::arrow_file::ArrowSource; +use datafusion::prelude::*; +use datafusion_common::Result; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfigBuilder; +use datafusion_datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory}; +use datafusion_datasource::source::DataSourceExec; +use std::sync::Arc; +use tempfile::TempDir; + +#[cfg(feature = "parquet")] +use datafusion_datasource_parquet::ParquetSource; +#[cfg(feature = "parquet")] +use parquet::arrow::ArrowWriter; +#[cfg(feature = "parquet")] +use parquet::file::properties::WriterProperties; + +#[cfg(feature = "csv")] +use datafusion_datasource_csv::CsvSource; + +/// A schema adapter factory that transforms column names to uppercase +#[derive(Debug)] +struct UppercaseAdapterFactory {} + +impl SchemaAdapterFactory for UppercaseAdapterFactory { + fn create(&self, schema: &Schema) -> Result> { + Ok(Box::new(UppercaseAdapter { + input_schema: Arc::new(schema.clone()), + })) + } +} + +/// Schema adapter that transforms column names to uppercase +#[derive(Debug)] +struct UppercaseAdapter { + input_schema: SchemaRef, +} + +impl SchemaAdapter for UppercaseAdapter { + fn adapt(&self, record_batch: RecordBatch) -> Result { + // In a real adapter, we might transform the data too + // For this test, we're just passing through the batch + Ok(record_batch) + } + + fn output_schema(&self) -> SchemaRef { + let fields = self + .input_schema + .fields() + .iter() + .map(|f| { + Field::new( + f.name().to_uppercase().as_str(), + f.data_type().clone(), + f.is_nullable(), + ) + }) + .collect(); + + Arc::new(Schema::new(fields)) + } +} + +#[cfg(feature = "parquet")] +#[tokio::test] +async fn test_parquet_integration_with_schema_adapter() -> Result<()> { + // Create a temporary directory for our test file + let tmp_dir = TempDir::new()?; + let file_path = tmp_dir.path().join("test.parquet"); + let file_path_str = file_path.to_str().unwrap(); + + // Create test data + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, true), + ])); + + let batch = RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(arrow::array::Int32Array::from(vec![1, 2, 3])), + Arc::new(arrow::array::StringArray::from(vec!["a", "b", "c"])), + ], + )?; + + // Write test parquet file + let file = std::fs::File::create(file_path_str)?; + let props = WriterProperties::builder().build(); + let mut writer = ArrowWriter::try_new(file, schema.clone(), Some(props))?; + writer.write(&batch)?; + writer.close()?; + + // Create a session context + let ctx = SessionContext::new(); + + // Create a ParquetSource with the adapter factory + let source = ParquetSource::default() + .with_schema_adapter_factory(Arc::new(UppercaseAdapterFactory {})); + + // Create a scan config + let config = FileScanConfigBuilder::new( + ObjectStoreUrl::parse(&format!("file://{}", file_path_str))?, + schema.clone(), + ) + .with_source(source) + .build(); + + // Create a data source executor + let exec = DataSourceExec::from_data_source(config); + + // Collect results + let task_ctx = ctx.task_ctx(); + let stream = exec.execute(0, task_ctx)?; + let batches = datafusion::physical_plan::common::collect(stream).await?; + + // There should be one batch + assert_eq!(batches.len(), 1); + + // Verify the schema has uppercase column names + let result_schema = batches[0].schema(); + assert_eq!(result_schema.field(0).name(), "ID"); + assert_eq!(result_schema.field(1).name(), "NAME"); + + Ok(()) +} + +#[tokio::test] +async fn test_multi_source_schema_adapter_reuse() -> Result<()> { + // Create a test factory + let factory = Arc::new(UppercaseAdapterFactory {}); + + // Apply the same adapter to different source types + let arrow_source = + ArrowSource::default().with_schema_adapter_factory(factory.clone()); + + #[cfg(feature = "parquet")] + let parquet_source = + ParquetSource::default().with_schema_adapter_factory(factory.clone()); + + #[cfg(feature = "csv")] + let csv_source = CsvSource::default().with_schema_adapter_factory(factory.clone()); + + // Verify adapters were properly set + assert!(arrow_source.schema_adapter_factory().is_some()); + + #[cfg(feature = "parquet")] + assert!(parquet_source.schema_adapter_factory().is_some()); + + #[cfg(feature = "csv")] + assert!(csv_source.schema_adapter_factory().is_some()); + + Ok(()) +} + +// Helper function to test From for Arc implementations +fn test_from_impl> + Default>(expected_file_type: &str) { + let source = T::default(); + let file_source: Arc = source.into(); + assert_eq!(file_source.file_type(), expected_file_type); +} + +#[test] +fn test_from_implementations() { + // Test From implementation for various sources + test_from_impl::("arrow"); + + #[cfg(feature = "parquet")] + test_from_impl::("parquet"); + + #[cfg(feature = "csv")] + test_from_impl::("csv"); + + #[cfg(feature = "json")] + test_from_impl::("json"); +} diff --git a/datafusion/core/tests/schema_adapter_factory_tests.rs b/datafusion/core/tests/schema_adapter_factory_tests.rs new file mode 100644 index 0000000000000..d16a1055ea862 --- /dev/null +++ b/datafusion/core/tests/schema_adapter_factory_tests.rs @@ -0,0 +1,167 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion::datasource::physical_plan::arrow_file::ArrowSource; +use datafusion::prelude::*; +use datafusion_common::Result; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory}; +use std::sync::Arc; + +#[cfg(feature = "parquet")] +use datafusion_datasource_parquet::ParquetSource; + +#[cfg(feature = "avro")] +use datafusion_datasource_avro::AvroSource; + +#[cfg(feature = "json")] +use datafusion_datasource_json::JsonSource; + +#[cfg(feature = "csv")] +use datafusion_datasource_csv::CsvSource; + +/// A test schema adapter factory that adds an extra column +#[derive(Debug)] +struct TestSchemaAdapterFactory {} + +impl SchemaAdapterFactory for TestSchemaAdapterFactory { + fn create(&self, schema: &Schema) -> Result> { + Ok(Box::new(TestSchemaAdapter { + input_schema: Arc::new(schema.clone()), + })) + } +} + +/// A test schema adapter that adds a column +#[derive(Debug)] +struct TestSchemaAdapter { + input_schema: SchemaRef, +} + +impl SchemaAdapter for TestSchemaAdapter { + fn adapt( + &self, + mut record_batch: arrow::record_batch::RecordBatch, + ) -> Result { + // In a real adapter, we would transform the record batch here + // For this test, we're just verifying the adapter was called correctly + Ok(record_batch) + } + + fn output_schema(&self) -> SchemaRef { + // This creates an output schema with one additional column + let mut fields = self.input_schema.fields().clone(); + fields.push(Field::new("adapted_column", DataType::Utf8, true)); + Arc::new(Schema::new(fields)) + } +} + +// General function to test schema adapter factory functionality for any file source +fn test_generic_schema_adapter_factory(file_type: &str) { + let source = T::default(); + + // Test that schema adapter factory is initially None + assert!(source.schema_adapter_factory().is_none()); + + // Add a schema adapter factory + let factory = Arc::new(TestSchemaAdapterFactory {}); + let source_with_adapter = source.with_schema_adapter_factory(factory); + + // Verify schema adapter factory is now set + assert!(source_with_adapter.schema_adapter_factory().is_some()); + + // Check that file_type method returns the expected value + assert_eq!(source_with_adapter.file_type(), file_type); +} + +#[test] +fn test_arrow_source_schema_adapter_factory() { + test_generic_schema_adapter_factory::("arrow"); +} + +#[cfg(feature = "parquet")] +#[test] +fn test_parquet_source_schema_adapter_factory() { + test_generic_schema_adapter_factory::("parquet"); +} + +#[cfg(feature = "avro")] +#[test] +fn test_avro_source_schema_adapter_factory() { + test_generic_schema_adapter_factory::("avro"); +} + +#[cfg(feature = "json")] +#[test] +fn test_json_source_schema_adapter_factory() { + test_generic_schema_adapter_factory::("json"); +} + +#[cfg(feature = "csv")] +#[test] +fn test_csv_source_schema_adapter_factory() { + test_generic_schema_adapter_factory::("csv"); +} + +#[test] +fn test_file_source_conversion() { + // Test the as_file_source function + let arrow_source = ArrowSource::default(); + let file_source = datafusion_datasource::as_file_source(arrow_source); + assert_eq!(file_source.file_type(), "arrow"); + + // Test the From implementation for ArrowSource + let arrow_source = ArrowSource::default(); + let file_source: Arc = arrow_source.into(); + assert_eq!(file_source.file_type(), "arrow"); +} + +#[cfg(feature = "parquet")] +#[test] +fn test_apply_schema_adapter() { + use datafusion::datasource::object_store::ObjectStoreUrl; + use datafusion_datasource::file_scan_config::{ + FileScanConfig, FileScanConfigBuilder, + }; + use datafusion_datasource_parquet::file_format::apply_schema_adapter; + + // Create a test schema + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, true), + ])); + + // Create a basic FileScanConfig + let config_builder = FileScanConfigBuilder::new( + ObjectStoreUrl::parse("file:///path/to/parquet").unwrap(), + schema.clone(), + ); + + // Create source and apply adapter + let source = ParquetSource::default(); + let factory = Arc::new(TestSchemaAdapterFactory {}); + let file_source = source.with_schema_adapter_factory(factory); + let config = config_builder.with_source(file_source).build(); + + // Test that apply_schema_adapter preserves the adapter + let source = ParquetSource::default(); + let result = apply_schema_adapter(source, &config); + + // Verify the schema adapter factory was preserved + assert!(result.schema_adapter_factory().is_some()); +} diff --git a/datafusion/core/tests/test_source_adapter_tests.rs b/datafusion/core/tests/test_source_adapter_tests.rs new file mode 100644 index 0000000000000..2d6c8d4c39bf3 --- /dev/null +++ b/datafusion/core/tests/test_source_adapter_tests.rs @@ -0,0 +1,109 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion::physical_optimizer::filter_pushdown::util::TestSource; +use datafusion_common::Result; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory}; +use std::sync::Arc; + +/// A simple schema adapter factory for testing +#[derive(Debug)] +struct TestFilterPushdownAdapterFactory {} + +impl SchemaAdapterFactory for TestFilterPushdownAdapterFactory { + fn create(&self, schema: &Schema) -> Result> { + Ok(Box::new(TestFilterPushdownAdapter { + input_schema: Arc::new(schema.clone()), + })) + } +} + +/// A simple schema adapter for testing +#[derive(Debug)] +struct TestFilterPushdownAdapter { + input_schema: SchemaRef, +} + +impl SchemaAdapter for TestFilterPushdownAdapter { + fn adapt(&self, record_batch: arrow::record_batch::RecordBatch) -> Result { + Ok(record_batch) + } + + fn output_schema(&self) -> SchemaRef { + // Add a suffix to column names + let fields = self + .input_schema + .fields() + .iter() + .map(|f| { + Field::new( + format!("{}_modified", f.name()).as_str(), + f.data_type().clone(), + f.is_nullable(), + ) + }) + .collect(); + + Arc::new(Schema::new(fields)) + } +} + +#[test] +fn test_test_source_schema_adapter_factory() { + // Create a TestSource instance + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("value", DataType::Utf8, true), + ])); + + let batches = vec![]; // Empty for this test + let source = TestSource::new(true, batches); + + // Verify initial state has no adapter + assert!(source.schema_adapter_factory().is_none()); + + // Apply an adapter factory + let factory = Arc::new(TestFilterPushdownAdapterFactory {}); + let source_with_adapter = source.with_schema_adapter_factory(factory); + + // Verify adapter was set + assert!(source_with_adapter.schema_adapter_factory().is_some()); + + // Create an adapter and validate the output schema + let adapter_factory = source_with_adapter.schema_adapter_factory().unwrap(); + let adapter = adapter_factory.create(&schema).unwrap(); + let output_schema = adapter.output_schema(); + + // Check modified column names + assert_eq!(output_schema.field(0).name(), "id_modified"); + assert_eq!(output_schema.field(1).name(), "value_modified"); + + // Check file type remains unchanged + assert_eq!(source_with_adapter.file_type(), "test"); +} + +#[test] +fn test_test_source_default() { + // Create a TestSource with default values for other fields + let batches = vec![]; + let source = TestSource::new(false, batches); + + // Ensure schema_adapter_factory is None by default + assert!(source.schema_adapter_factory().is_none()); +} diff --git a/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs new file mode 100644 index 0000000000000..2a0ab042f69db --- /dev/null +++ b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs @@ -0,0 +1,147 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +#[cfg(feature = "parquet")] +mod parquet_adapter_tests { + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; + use datafusion::datasource::object_store::ObjectStoreUrl; + use datafusion_common::Result; + use datafusion_datasource::file::FileSource; + use datafusion_datasource::file_scan_config::{ + FileScanConfig, FileScanConfigBuilder, + }; + use datafusion_datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory}; + use datafusion_datasource_parquet::file_format::apply_schema_adapter; + use datafusion_datasource_parquet::ParquetSource; + use std::sync::Arc; + + /// A test schema adapter factory that adds prefix to column names + #[derive(Debug)] + struct PrefixAdapterFactory { + prefix: String, + } + + impl SchemaAdapterFactory for PrefixAdapterFactory { + fn create(&self, schema: &Schema) -> Result> { + Ok(Box::new(PrefixAdapter { + input_schema: Arc::new(schema.clone()), + prefix: self.prefix.clone(), + })) + } + } + + /// A test schema adapter that adds prefix to column names + #[derive(Debug)] + struct PrefixAdapter { + input_schema: SchemaRef, + prefix: String, + } + + impl SchemaAdapter for PrefixAdapter { + fn adapt( + &self, + record_batch: arrow::record_batch::RecordBatch, + ) -> Result { + // In a real adapter, we might transform the data + // For this test, we're just verifying the adapter was called correctly + Ok(record_batch) + } + + fn output_schema(&self) -> SchemaRef { + let fields = self + .input_schema + .fields() + .iter() + .map(|f| { + Field::new( + format!("{}{}", self.prefix, f.name()).as_str(), + f.data_type().clone(), + f.is_nullable(), + ) + }) + .collect(); + + Arc::new(Schema::new(fields)) + } + } + + #[test] + fn test_apply_schema_adapter_with_factory() { + // Create a schema + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, true), + ])); + + // Create a parquet source + let source = ParquetSource::default(); + + // 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); + + let config = FileScanConfigBuilder::new( + ObjectStoreUrl::local_filesystem(), + schema.clone(), + ) + .with_source(file_source) + .build(); + + // Apply schema adapter to a new source + let result_source = apply_schema_adapter(source, &config); + + // Verify the adapter was applied + assert!(result_source.schema_adapter_factory().is_some()); + + // Create adapter and test it produces expected schema + let adapter_factory = result_source.schema_adapter_factory().unwrap(); + let adapter = adapter_factory.create(&schema).unwrap(); + let output_schema = adapter.output_schema(); + + // Check the column names have the prefix + assert_eq!(output_schema.field(0).name(), "test_id"); + assert_eq!(output_schema.field(1).name(), "test_name"); + } + + #[test] + fn test_apply_schema_adapter_without_factory() { + // Create a schema + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, true), + ])); + + // Create a parquet source + let source = ParquetSource::default(); + + // Create a file scan config without a schema adapter factory + let config = FileScanConfigBuilder::new( + ObjectStoreUrl::local_filesystem(), + schema.clone(), + ) + .build(); + + // Apply schema adapter function - should pass through the source unchanged + let result_source = apply_schema_adapter(source, &config); + + // Verify no adapter was applied + assert!(result_source.schema_adapter_factory().is_none()); + } +} From dc9b8ac86e29799f31ecaeb6355dbd6a6ff225ba Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 15:38:37 +0800 Subject: [PATCH 08/38] fix tests --- datafusion/core/src/datasource/mod.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 273ac2be8fbf3..b866b5c46d160 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -52,7 +52,6 @@ pub use datafusion_physical_expr::create_ordering; mod tests { use crate::prelude::SessionContext; - use datafusion_datasource::file::FileSource; use std::fs; use std::sync::Arc; @@ -129,7 +128,7 @@ mod tests { let base_conf = FileScanConfigBuilder::new( ObjectStoreUrl::local_filesystem(), schema, - source, + source.into(), ) .with_file(partitioned_file) .build(); From 08b3ce08b02de2ddee2258578bae1e860fa5995f Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 15:51:14 +0800 Subject: [PATCH 09/38] Refactor adapt method signature and improve test assertions for schema adapter factory --- .../core/tests/test_source_adapter_tests.rs | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/datafusion/core/tests/test_source_adapter_tests.rs b/datafusion/core/tests/test_source_adapter_tests.rs index 2d6c8d4c39bf3..0f6f9a6ce0ff1 100644 --- a/datafusion/core/tests/test_source_adapter_tests.rs +++ b/datafusion/core/tests/test_source_adapter_tests.rs @@ -41,13 +41,16 @@ struct TestFilterPushdownAdapter { } impl SchemaAdapter for TestFilterPushdownAdapter { - fn adapt(&self, record_batch: arrow::record_batch::RecordBatch) -> Result { + fn adapt( + &self, + record_batch: arrow::record_batch::RecordBatch, + ) -> Result { Ok(record_batch) } fn output_schema(&self) -> SchemaRef { // Add a suffix to column names - let fields = self + let fields: Vec<_> = self .input_schema .fields() .iter() @@ -71,29 +74,29 @@ fn test_test_source_schema_adapter_factory() { Field::new("id", DataType::Int32, false), Field::new("value", DataType::Utf8, true), ])); - + let batches = vec![]; // Empty for this test let source = TestSource::new(true, batches); - + // Verify initial state has no adapter assert!(source.schema_adapter_factory().is_none()); - + // Apply an adapter factory let factory = Arc::new(TestFilterPushdownAdapterFactory {}); let source_with_adapter = source.with_schema_adapter_factory(factory); - + // Verify adapter was set assert!(source_with_adapter.schema_adapter_factory().is_some()); - + // Create an adapter and validate the output schema let adapter_factory = source_with_adapter.schema_adapter_factory().unwrap(); let adapter = adapter_factory.create(&schema).unwrap(); let output_schema = adapter.output_schema(); - + // Check modified column names assert_eq!(output_schema.field(0).name(), "id_modified"); assert_eq!(output_schema.field(1).name(), "value_modified"); - + // Check file type remains unchanged assert_eq!(source_with_adapter.file_type(), "test"); } @@ -103,7 +106,7 @@ fn test_test_source_default() { // Create a TestSource with default values for other fields let batches = vec![]; let source = TestSource::new(false, batches); - + // Ensure schema_adapter_factory is None by default assert!(source.schema_adapter_factory().is_none()); } From aef5dd3985c6905d42b1755a9fc8453cbf5a4261 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 15:53:52 +0800 Subject: [PATCH 10/38] Simplify constructor in TestSource by removing redundant function definition --- .../tests/physical_optimizer/filter_pushdown/util.rs | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index 4e611b03fa777..7d6465cc3bf70 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -128,12 +128,10 @@ pub struct TestSource { impl TestSource { fn new(support: bool, batches: Vec) -> Self { - fn new(support: bool, batches: Vec) -> Self { - Self { - support, - batches, - ..Default::default() - } + Self { + support, + batches, + ..Default::default() } } } From f964947f5429ab5824edfa67bef15ab132817468 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 15:54:44 +0800 Subject: [PATCH 11/38] Remove redundant import of SchemaAdapterFactory in util.rs --- .../filter_pushdown/util.rs | 4 +- .../tests/schema_adapter_factory_tests.rs | 142 +++++++----- datafusion/core/tests/test_adapter_updated.rs | 201 +++++++++++++++++ .../core/tests/test_source_adapter_tests.rs | 211 ++++++++++++++---- .../tests/apply_schema_adapter_tests.rs | 143 +++++++++--- datafusion/datasource/src/test_util.rs | 16 ++ 6 files changed, 589 insertions(+), 128 deletions(-) create mode 100644 datafusion/core/tests/test_adapter_updated.rs diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index 7d6465cc3bf70..d6e103b1080d0 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -31,6 +31,7 @@ use arrow::{array::RecordBatch, compute::concat_batches}; use datafusion::{datasource::object_store::ObjectStoreUrl, physical_plan::PhysicalExpr}; use datafusion_common::{config::ConfigOptions, Statistics}; use datafusion_common::{internal_err, Result}; +use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use datafusion_datasource::{ file::FileSource, file_scan_config::FileScanConfig, file_stream::FileOpener, }; @@ -41,9 +42,6 @@ use datafusion_datasource::{ file_scan_config::FileScanConfigBuilder, file_stream::FileOpenFuture, source::DataSourceExec, }; -use datafusion_datasource::{ - file_scan_config::FileScanConfigBuilder, schema_adapter::SchemaAdapterFactory, -}; use datafusion_physical_expr::conjunction; use datafusion_physical_expr_common::physical_expr::fmt_sql; use datafusion_physical_optimizer::PhysicalOptimizerRule; diff --git a/datafusion/core/tests/schema_adapter_factory_tests.rs b/datafusion/core/tests/schema_adapter_factory_tests.rs index d16a1055ea862..4c840147abe22 100644 --- a/datafusion/core/tests/schema_adapter_factory_tests.rs +++ b/datafusion/core/tests/schema_adapter_factory_tests.rs @@ -16,34 +16,41 @@ // under the License. use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion::datasource::physical_plan::arrow_file::ArrowSource; -use datafusion::prelude::*; -use datafusion_common::Result; +use datafusion_common::ColumnStatistics; use datafusion_datasource::file::FileSource; use datafusion_datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory}; use std::sync::Arc; +// Import ArrowSource from the correct location +// ArrowSource is part of the core DataFusion package +use datafusion::datasource::physical_plan::ArrowSource; + #[cfg(feature = "parquet")] -use datafusion_datasource_parquet::ParquetSource; +use datafusion_datasource_parquet::source::ParquetSource; #[cfg(feature = "avro")] -use datafusion_datasource_avro::AvroSource; +use datafusion_datasource_avro::source::AvroSource; -#[cfg(feature = "json")] -use datafusion_datasource_json::JsonSource; +// JSON and CSV sources are not available in the current feature set +// #[cfg(feature = "json")] +// use datafusion_datasource_json::JsonSource; -#[cfg(feature = "csv")] -use datafusion_datasource_csv::CsvSource; +// #[cfg(feature = "csv")] +// use datafusion_datasource_csv::CsvSource; /// A test schema adapter factory that adds an extra column #[derive(Debug)] struct TestSchemaAdapterFactory {} impl SchemaAdapterFactory for TestSchemaAdapterFactory { - fn create(&self, schema: &Schema) -> Result> { - Ok(Box::new(TestSchemaAdapter { - input_schema: Arc::new(schema.clone()), - })) + fn create( + &self, + projected_table_schema: SchemaRef, + _table_schema: SchemaRef, + ) -> Box { + Box::new(TestSchemaAdapter { + input_schema: projected_table_schema, + }) } } @@ -54,20 +61,57 @@ struct TestSchemaAdapter { } impl SchemaAdapter for TestSchemaAdapter { - fn adapt( - &self, - mut record_batch: arrow::record_batch::RecordBatch, - ) -> Result { - // In a real adapter, we would transform the record batch here - // For this test, we're just verifying the adapter was called correctly - Ok(record_batch) + fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { + let field = self.input_schema.field(index); + file_schema.fields.find(field.name()).map(|(i, _)| i) } - fn output_schema(&self) -> SchemaRef { - // This creates an output schema with one additional column - let mut fields = self.input_schema.fields().clone(); - fields.push(Field::new("adapted_column", DataType::Utf8, true)); - Arc::new(Schema::new(fields)) + fn map_schema( + &self, + file_schema: &Schema, + ) -> datafusion_common::Result<( + Arc, + Vec, + )> { + let mut projection = Vec::with_capacity(file_schema.fields().len()); + for (file_idx, file_field) in file_schema.fields().iter().enumerate() { + if self.input_schema.fields().find(file_field.name()).is_some() { + projection.push(file_idx); + } + } + + // Create a schema mapper that adds an adapted_column + #[derive(Debug)] + struct TestSchemaMapping { + #[allow(dead_code)] + input_schema: SchemaRef, + } + + impl datafusion_datasource::schema_adapter::SchemaMapper for TestSchemaMapping { + fn map_batch( + &self, + batch: arrow::record_batch::RecordBatch, + ) -> datafusion_common::Result { + // In a real adapter, we would transform the record batch here + // For this test, we're just verifying the adapter was called correctly + Ok(batch) + } + + fn map_column_statistics( + &self, + file_col_statistics: &[ColumnStatistics], + ) -> datafusion_common::Result> { + // For testing, just return the input statistics + Ok(file_col_statistics.to_vec()) + } + } + + Ok(( + Arc::new(TestSchemaMapping { + input_schema: self.input_schema.clone(), + }), + projection, + )) } } @@ -106,17 +150,16 @@ fn test_avro_source_schema_adapter_factory() { test_generic_schema_adapter_factory::("avro"); } -#[cfg(feature = "json")] -#[test] -fn test_json_source_schema_adapter_factory() { - test_generic_schema_adapter_factory::("json"); -} +// JSON and CSV sources are not available in the current feature set +// #[test] +// fn test_json_source_schema_adapter_factory() { +// test_generic_schema_adapter_factory::("json"); +// } -#[cfg(feature = "csv")] -#[test] -fn test_csv_source_schema_adapter_factory() { - test_generic_schema_adapter_factory::("csv"); -} +// #[test] +// fn test_csv_source_schema_adapter_factory() { +// test_generic_schema_adapter_factory::("csv"); +// } #[test] fn test_file_source_conversion() { @@ -133,12 +176,9 @@ fn test_file_source_conversion() { #[cfg(feature = "parquet")] #[test] -fn test_apply_schema_adapter() { +fn test_schema_adapter_preservation() { use datafusion::datasource::object_store::ObjectStoreUrl; - use datafusion_datasource::file_scan_config::{ - FileScanConfig, FileScanConfigBuilder, - }; - use datafusion_datasource_parquet::file_format::apply_schema_adapter; + use datafusion_datasource::file_scan_config::FileScanConfigBuilder; // Create a test schema let schema = Arc::new(Schema::new(vec![ @@ -146,22 +186,20 @@ fn test_apply_schema_adapter() { Field::new("name", DataType::Utf8, true), ])); - // Create a basic FileScanConfig + // Create source with schema adapter factory + let source = ParquetSource::default(); + let factory = Arc::new(TestSchemaAdapterFactory {}); + let file_source = source.with_schema_adapter_factory(factory); + + // Create a FileScanConfig with the source let config_builder = FileScanConfigBuilder::new( ObjectStoreUrl::parse("file:///path/to/parquet").unwrap(), schema.clone(), + file_source.clone().into(), ); - // Create source and apply adapter - let source = ParquetSource::default(); - let factory = Arc::new(TestSchemaAdapterFactory {}); - let file_source = source.with_schema_adapter_factory(factory); - let config = config_builder.with_source(file_source).build(); - - // Test that apply_schema_adapter preserves the adapter - let source = ParquetSource::default(); - let result = apply_schema_adapter(source, &config); + let config = config_builder.build(); - // Verify the schema adapter factory was preserved - assert!(result.schema_adapter_factory().is_some()); + // Verify the schema adapter factory is present in the file source + assert!(config.file_source().schema_adapter_factory().is_some()); } diff --git a/datafusion/core/tests/test_adapter_updated.rs b/datafusion/core/tests/test_adapter_updated.rs new file mode 100644 index 0000000000000..3d56f296f36b0 --- /dev/null +++ b/datafusion/core/tests/test_adapter_updated.rs @@ -0,0 +1,201 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; +use datafusion_common::{ColumnStatistics, DataFusionError, Result, Statistics}; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_datasource::file_stream::FileOpener; +use datafusion_datasource::schema_adapter::{ + SchemaAdapter, SchemaAdapterFactory, SchemaMapper, +}; +use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +use object_store::ObjectStore; +use std::any::Any; +use std::fmt::Debug; +use std::sync::Arc; + +/// A test source for testing schema adapters +#[derive(Debug, Clone)] +struct TestSource { + schema_adapter_factory: Option>, +} + +impl TestSource { + fn new() -> Self { + Self { + schema_adapter_factory: None, + } + } +} + +impl FileSource for TestSource { + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() + } + + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Arc { + let mut new_source = self.clone(); + new_source.schema_adapter_factory = Some(schema_adapter_factory); + Arc::new(new_source) + } + + fn file_type(&self) -> &str { + "test" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn create_file_opener( + &self, + _store: Arc, + _conf: &FileScanConfig, + _index: usize, + ) -> Arc { + unimplemented!("Not needed for this test") + } + + fn with_batch_size(&self, _batch_size: usize) -> Arc { + Arc::new(self.clone()) + } + + fn with_schema(&self, _schema: SchemaRef) -> Arc { + Arc::new(self.clone()) + } + + fn with_projection(&self, _projection: &FileScanConfig) -> Arc { + Arc::new(self.clone()) + } + + fn with_statistics(&self, _statistics: Statistics) -> Arc { + Arc::new(self.clone()) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + unimplemented!("Not needed for this test") + } + + fn statistics(&self) -> Result { + Ok(Statistics::default()) + } +} + +/// A test schema adapter factory +#[derive(Debug)] +struct TestSchemaAdapterFactory {} + +impl SchemaAdapterFactory for TestSchemaAdapterFactory { + fn create( + &self, + projected_table_schema: SchemaRef, + _table_schema: SchemaRef, + ) -> Box { + Box::new(TestSchemaAdapter { + table_schema: projected_table_schema, + }) + } +} + +/// A test schema adapter implementation +#[derive(Debug)] +struct TestSchemaAdapter { + table_schema: SchemaRef, +} + +impl SchemaAdapter for TestSchemaAdapter { + fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { + let field = self.table_schema.field(index); + file_schema.fields.find(field.name()).map(|(i, _)| i) + } + + fn map_schema( + &self, + file_schema: &Schema, + ) -> Result<(Arc, Vec)> { + let mut projection = Vec::with_capacity(file_schema.fields().len()); + for (file_idx, file_field) in file_schema.fields().iter().enumerate() { + if self.table_schema.fields().find(file_field.name()).is_some() { + projection.push(file_idx); + } + } + + Ok((Arc::new(TestSchemaMapping {}), projection)) + } +} + +/// A test schema mapper implementation +#[derive(Debug)] +struct TestSchemaMapping {} + +impl SchemaMapper for TestSchemaMapping { + fn map_batch(&self, batch: RecordBatch) -> Result { + // For testing, just return the original batch + Ok(batch) + } + + fn map_column_statistics( + &self, + stats: &[ColumnStatistics], + ) -> Result> { + // For testing, just return the input statistics + Ok(stats.to_vec()) + } +} + +#[test] +fn test_schema_adapter() { + // Create a test schema + let table_schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, true), + ])); + + // Create a file schema + let file_schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, true), + Field::new("extra", DataType::Int64, true), + ]); + + // Create a TestSource + let source = TestSource::new(); + assert!(source.schema_adapter_factory().is_none()); + + // Add a schema adapter factory + let factory = Arc::new(TestSchemaAdapterFactory {}); + let source_with_adapter = source.with_schema_adapter_factory(factory); + assert!(source_with_adapter.schema_adapter_factory().is_some()); + + // Create a schema adapter + let adapter_factory = source_with_adapter.schema_adapter_factory().unwrap(); + let adapter = + adapter_factory.create(Arc::clone(&table_schema), Arc::clone(&table_schema)); + + // Test mapping column index + assert_eq!(adapter.map_column_index(0, &file_schema), Some(0)); + assert_eq!(adapter.map_column_index(1, &file_schema), Some(1)); + + // Test creating schema mapper + let (_mapper, projection) = adapter.map_schema(&file_schema).unwrap(); + assert_eq!(projection, vec![0, 1]); +} diff --git a/datafusion/core/tests/test_source_adapter_tests.rs b/datafusion/core/tests/test_source_adapter_tests.rs index 0f6f9a6ce0ff1..ee8495c00be12 100644 --- a/datafusion/core/tests/test_source_adapter_tests.rs +++ b/datafusion/core/tests/test_source_adapter_tests.rs @@ -16,21 +16,121 @@ // under the License. use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion::physical_optimizer::filter_pushdown::util::TestSource; -use datafusion_common::Result; +use datafusion_common::{ColumnStatistics, DataFusionError, Result, Statistics}; use datafusion_datasource::file::FileSource; -use datafusion_datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory}; +use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_datasource::file_stream::FileOpener; +use datafusion_datasource::schema_adapter::{ + SchemaAdapter, SchemaAdapterFactory, SchemaMapper, +}; +use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; +use object_store::ObjectStore; +use std::fmt::Debug; use std::sync::Arc; +// Simple TestSource implementation for testing without dependency on private module +#[derive(Clone, Debug)] +struct TestSource { + #[allow(dead_code)] + has_adapter: bool, + schema_adapter_factory: Option>, +} + +impl TestSource { + fn new(has_adapter: bool) -> Self { + Self { + has_adapter, + schema_adapter_factory: None, + } + } +} + +impl FileSource for TestSource { + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() + } + + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Arc { + let mut new_source = self.clone(); + new_source.schema_adapter_factory = Some(schema_adapter_factory); + Arc::new(new_source) + } + + fn file_type(&self) -> &str { + "test" + } + + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn create_file_opener( + &self, + _store: Arc, + _conf: &FileScanConfig, + _index: usize, + ) -> Arc { + unimplemented!("Not needed for this test") + } + + fn with_batch_size(&self, _batch_size: usize) -> Arc { + Arc::new(self.clone()) + } + + fn with_schema(&self, _schema: SchemaRef) -> Arc { + Arc::new(self.clone()) + } + + fn with_projection(&self, _projection: &FileScanConfig) -> Arc { + Arc::new(self.clone()) + } + + fn with_statistics(&self, _statistics: Statistics) -> Arc { + Arc::new(self.clone()) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + unimplemented!("Not needed for this test") + } + + fn statistics(&self) -> Result { + Ok(Statistics::default()) + } +} + +impl DisplayAs for TestSource { + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default + | DisplayFormatType::Verbose + | DisplayFormatType::TreeRender => { + write!(f, "TestSource") + } + } + } +} + /// A simple schema adapter factory for testing #[derive(Debug)] struct TestFilterPushdownAdapterFactory {} impl SchemaAdapterFactory for TestFilterPushdownAdapterFactory { - fn create(&self, schema: &Schema) -> Result> { - Ok(Box::new(TestFilterPushdownAdapter { - input_schema: Arc::new(schema.clone()), - })) + fn create( + &self, + projected_table_schema: SchemaRef, + _table_schema: SchemaRef, + ) -> Box { + Box::new(TestFilterPushdownAdapter { + input_schema: projected_table_schema, + }) } } @@ -41,29 +141,53 @@ struct TestFilterPushdownAdapter { } impl SchemaAdapter for TestFilterPushdownAdapter { - fn adapt( + fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { + let field = self.input_schema.field(index); + file_schema.fields.find(field.name()).map(|(i, _)| i) + } + + fn map_schema( &self, - record_batch: arrow::record_batch::RecordBatch, - ) -> Result { - Ok(record_batch) - } - - fn output_schema(&self) -> SchemaRef { - // Add a suffix to column names - let fields: Vec<_> = self - .input_schema - .fields() - .iter() - .map(|f| { - Field::new( - format!("{}_modified", f.name()).as_str(), - f.data_type().clone(), - f.is_nullable(), - ) - }) - .collect(); - - Arc::new(Schema::new(fields)) + file_schema: &Schema, + ) -> Result<(Arc, Vec)> { + let mut projection = Vec::with_capacity(file_schema.fields().len()); + for (file_idx, file_field) in file_schema.fields().iter().enumerate() { + if self.input_schema.fields().find(file_field.name()).is_some() { + projection.push(file_idx); + } + } + + // Create a schema mapper that modifies column names + #[derive(Debug)] + struct TestSchemaMapping { + #[allow(dead_code)] + input_schema: SchemaRef, + } + + impl SchemaMapper for TestSchemaMapping { + fn map_batch( + &self, + batch: arrow::record_batch::RecordBatch, + ) -> Result { + // For testing, just return the original batch + Ok(batch) + } + + fn map_column_statistics( + &self, + file_col_statistics: &[ColumnStatistics], + ) -> Result> { + // For testing, just return the input statistics + Ok(file_col_statistics.to_vec()) + } + } + + Ok(( + Arc::new(TestSchemaMapping { + input_schema: self.input_schema.clone(), + }), + projection, + )) } } @@ -75,8 +199,7 @@ fn test_test_source_schema_adapter_factory() { Field::new("value", DataType::Utf8, true), ])); - let batches = vec![]; // Empty for this test - let source = TestSource::new(true, batches); + let source = TestSource::new(true); // Verify initial state has no adapter assert!(source.schema_adapter_factory().is_none()); @@ -88,14 +211,23 @@ fn test_test_source_schema_adapter_factory() { // Verify adapter was set assert!(source_with_adapter.schema_adapter_factory().is_some()); - // Create an adapter and validate the output schema + // Create an adapter let adapter_factory = source_with_adapter.schema_adapter_factory().unwrap(); - let adapter = adapter_factory.create(&schema).unwrap(); - let output_schema = adapter.output_schema(); + let adapter = adapter_factory.create(Arc::clone(&schema), Arc::clone(&schema)); + + // Create a file schema to test mapping + let file_schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("value", DataType::Utf8, true), + ]); + + // Test column mapping + let id_index = adapter.map_column_index(0, &file_schema); + assert_eq!(id_index, Some(0)); - // Check modified column names - assert_eq!(output_schema.field(0).name(), "id_modified"); - assert_eq!(output_schema.field(1).name(), "value_modified"); + // Test schema mapping + let (_mapper, projection) = adapter.map_schema(&file_schema).unwrap(); + assert_eq!(projection.len(), 2); // Both columns should be included // Check file type remains unchanged assert_eq!(source_with_adapter.file_type(), "test"); @@ -103,9 +235,8 @@ fn test_test_source_schema_adapter_factory() { #[test] fn test_test_source_default() { - // Create a TestSource with default values for other fields - let batches = vec![]; - let source = TestSource::new(false, batches); + // Create a TestSource with default values + let source = TestSource::new(false); // Ensure schema_adapter_factory is None by default assert!(source.schema_adapter_factory().is_none()); diff --git a/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs index 2a0ab042f69db..9e0f3ff2838b6 100644 --- a/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs +++ b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs @@ -15,18 +15,20 @@ // specific language governing permissions and limitations // under the License. -#[cfg(feature = "parquet")] mod parquet_adapter_tests { use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use datafusion::datasource::object_store::ObjectStoreUrl; - use datafusion_common::Result; + use arrow::record_batch::RecordBatch; + use datafusion_common::{ColumnStatistics, DataFusionError, Result}; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::{ FileScanConfig, FileScanConfigBuilder, }; - use datafusion_datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory}; - use datafusion_datasource_parquet::file_format::apply_schema_adapter; - use datafusion_datasource_parquet::ParquetSource; + use datafusion_datasource::schema_adapter::{ + SchemaAdapter, SchemaAdapterFactory, SchemaMapper, + }; + use datafusion_datasource_parquet::source::ParquetSource; + use datafusion_execution::object_store::ObjectStoreUrl; + use std::fmt::Debug; use std::sync::Arc; /// A test schema adapter factory that adds prefix to column names @@ -36,11 +38,15 @@ mod parquet_adapter_tests { } impl SchemaAdapterFactory for PrefixAdapterFactory { - fn create(&self, schema: &Schema) -> Result> { - Ok(Box::new(PrefixAdapter { - input_schema: Arc::new(schema.clone()), + fn create( + &self, + projected_table_schema: SchemaRef, + _table_schema: SchemaRef, + ) -> Box { + Box::new(PrefixAdapter { + input_schema: projected_table_schema, prefix: self.prefix.clone(), - })) + }) } } @@ -52,30 +58,88 @@ mod parquet_adapter_tests { } impl SchemaAdapter for PrefixAdapter { - fn adapt( - &self, - record_batch: arrow::record_batch::RecordBatch, - ) -> Result { - // In a real adapter, we might transform the data - // For this test, we're just verifying the adapter was called correctly - Ok(record_batch) + fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { + let field = self.input_schema.field(index); + file_schema.fields.find(field.name()).map(|(i, _)| i) } - fn output_schema(&self) -> SchemaRef { - let fields = self - .input_schema - .fields() - .iter() - .map(|f| { - Field::new( - format!("{}{}", self.prefix, f.name()).as_str(), - f.data_type().clone(), - f.is_nullable(), + fn map_schema( + &self, + file_schema: &Schema, + ) -> Result<(Arc, Vec)> { + let mut projection = Vec::with_capacity(file_schema.fields().len()); + for (file_idx, file_field) in file_schema.fields().iter().enumerate() { + if self.input_schema.fields().find(file_field.name()).is_some() { + projection.push(file_idx); + } + } + + // Create a schema mapper that adds a prefix to column names + #[derive(Debug)] + struct PrefixSchemaMapping { + // Keep only the prefix field which is actually used in the implementation + prefix: String, + } + + impl SchemaMapper for PrefixSchemaMapping { + fn map_batch(&self, batch: RecordBatch) -> Result { + // Create a new schema with prefixed field names + let prefixed_fields: Vec = batch + .schema() + .fields() + .iter() + .map(|field| { + Field::new( + format!("{}{}", self.prefix, field.name()), + field.data_type().clone(), + field.is_nullable(), + ) + }) + .collect(); + let prefixed_schema = Arc::new(Schema::new(prefixed_fields)); + + // Create a new batch with the prefixed schema but the same data + let options = arrow::record_batch::RecordBatchOptions::default(); + RecordBatch::try_new_with_options( + prefixed_schema, + batch.columns().to_vec(), + &options, ) - }) - .collect(); + .map_err(|e| DataFusionError::ArrowError(e, None)) + } + + fn map_column_statistics( + &self, + stats: &[ColumnStatistics], + ) -> Result> { + // For testing, just return the input statistics + Ok(stats.to_vec()) + } + } + + Ok(( + Arc::new(PrefixSchemaMapping { + prefix: self.prefix.clone(), + }), + projection, + )) + } + } - Arc::new(Schema::new(fields)) + // Implementation of apply_schema_adapter for testing purposes + // This mimics the private function in the datafusion-parquet crate + fn apply_schema_adapter( + source: ParquetSource, + conf: &FileScanConfig, + ) -> Arc { + // Convert the ParquetSource to Arc + let file_source: Arc = Arc::new(source); + + // If the FileScanConfig.file_source() has a schema adapter factory, apply it + if let Some(factory) = conf.file_source().schema_adapter_factory() { + file_source.with_schema_adapter_factory(factory) + } else { + file_source } } @@ -100,8 +164,8 @@ mod parquet_adapter_tests { let config = FileScanConfigBuilder::new( ObjectStoreUrl::local_filesystem(), schema.clone(), + file_source.into(), // Pass file_source as the third parameter ) - .with_source(file_source) .build(); // Apply schema adapter to a new source @@ -112,8 +176,17 @@ mod parquet_adapter_tests { // Create adapter and test it produces expected schema let adapter_factory = result_source.schema_adapter_factory().unwrap(); - let adapter = adapter_factory.create(&schema).unwrap(); - let output_schema = adapter.output_schema(); + let adapter = adapter_factory.create(schema.clone(), schema.clone()); + + // Create a dummy batch to test the schema mapping + let dummy_batch = RecordBatch::new_empty(schema.clone()); + + // Get the file schema (which is the same as the table schema in this test) + let (mapper, _) = adapter.map_schema(&schema).unwrap(); + + // Apply the mapping to get the output schema + let mapped_batch = mapper.map_batch(dummy_batch).unwrap(); + let output_schema = mapped_batch.schema(); // Check the column names have the prefix assert_eq!(output_schema.field(0).name(), "test_id"); @@ -131,10 +204,14 @@ mod parquet_adapter_tests { // 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(); diff --git a/datafusion/datasource/src/test_util.rs b/datafusion/datasource/src/test_util.rs index e75c7c3a3dc5e..fbb88083acb55 100644 --- a/datafusion/datasource/src/test_util.rs +++ b/datafusion/datasource/src/test_util.rs @@ -17,6 +17,7 @@ use crate::{ file::FileSource, file_scan_config::FileScanConfig, file_stream::FileOpener, + schema_adapter::SchemaAdapterFactory, }; use std::sync::Arc; @@ -32,6 +33,7 @@ use object_store::ObjectStore; pub(crate) struct MockSource { metrics: ExecutionPlanMetricsSet, projected_statistics: Option, + schema_adapter_factory: Option>, } impl FileSource for MockSource { @@ -81,6 +83,20 @@ impl FileSource for MockSource { fn file_type(&self) -> &str { "mock" } + + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Arc { + Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + }) + } + + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() + } } /// Create a column expression From d8720f0be1632c81868c88e88b1fac2182b4ea1b Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 17:34:24 +0800 Subject: [PATCH 12/38] fix tests: refactor schema_adapter_factory methods in TestSource for improved clarity --- datafusion/core/src/datasource/mod.rs | 1 + .../core/tests/test_source_adapter_tests.rs | 15 ++++++++------- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index b866b5c46d160..dd313455f08bf 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -52,6 +52,7 @@ pub use datafusion_physical_expr::create_ordering; mod tests { use crate::prelude::SessionContext; + use std::fs; use std::sync::Arc; diff --git a/datafusion/core/tests/test_source_adapter_tests.rs b/datafusion/core/tests/test_source_adapter_tests.rs index ee8495c00be12..f69134bf495ec 100644 --- a/datafusion/core/tests/test_source_adapter_tests.rs +++ b/datafusion/core/tests/test_source_adapter_tests.rs @@ -47,17 +47,18 @@ impl TestSource { } impl FileSource for TestSource { - fn schema_adapter_factory(&self) -> Option> { - self.schema_adapter_factory.clone() - } - fn with_schema_adapter_factory( &self, schema_adapter_factory: Arc, ) -> Arc { - let mut new_source = self.clone(); - new_source.schema_adapter_factory = Some(schema_adapter_factory); - Arc::new(new_source) + Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + }) + } + + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() } fn file_type(&self) -> &str { From 652fbafad7a8944f51c0b4d6fb89a0c26a52da4a Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 18:40:37 +0800 Subject: [PATCH 13/38] feat: add macro for schema adapter methods in FileSource implementation --- datafusion/datasource-csv/src/source.rs | 17 +++------------ datafusion/datasource/src/file.rs | 28 +++++++++++++++++++++++++ 2 files changed, 31 insertions(+), 14 deletions(-) diff --git a/datafusion/datasource-csv/src/source.rs b/datafusion/datasource-csv/src/source.rs index 81c9eef0ffacf..d45080dc20776 100644 --- a/datafusion/datasource-csv/src/source.rs +++ b/datafusion/datasource-csv/src/source.rs @@ -29,7 +29,8 @@ use datafusion_datasource::file_compression_type::FileCompressionType; use datafusion_datasource::file_meta::FileMeta; use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; use datafusion_datasource::{ - as_file_source, calculate_range, FileRange, ListingTableUrl, RangeCalculation, + as_file_source, calculate_range, impl_schema_adapter_methods, FileRange, + ListingTableUrl, RangeCalculation, }; use arrow::csv; @@ -283,19 +284,7 @@ impl FileSource for CsvSource { } } - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Arc { - Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - }) - } - - fn schema_adapter_factory(&self) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } impl FileOpener for CsvOpener { diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 04b6b6975a766..7bc78d54153e5 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -136,3 +136,31 @@ pub trait FileSource: Send + Sync { /// Returns the current schema adapter factory if set fn schema_adapter_factory(&self) -> Option>; } + +/// Helper macro to generate the two schema‐adapter methods +/// +/// Place this inside *any* `impl FileSource for YourType { … }` to +/// avoid copy-pasting `with_schema_adapter_factory` and +/// `schema_adapter_factory`. +#[macro_export] +macro_rules! impl_schema_adapter_methods { + () => { + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: std::sync::Arc< + dyn $crate::schema_adapter::SchemaAdapterFactory, + >, + ) -> std::sync::Arc { + std::sync::Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + }) + } + + fn schema_adapter_factory( + &self, + ) -> Option> { + self.schema_adapter_factory.clone() + } + }; +} From fbd8c99db4ab7fe59d11a98af17947f54eab67cf Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 18:48:55 +0800 Subject: [PATCH 14/38] feat: use macro implement schema adapter methods for various FileSource implementations --- .../src/datasource/physical_plan/arrow_file.rs | 16 ++-------------- .../physical_optimizer/filter_pushdown/util.rs | 18 ++++-------------- datafusion/core/tests/test_adapter_updated.rs | 16 +++------------- .../core/tests/test_source_adapter_tests.rs | 17 +++-------------- datafusion/datasource-avro/src/source.rs | 15 ++------------- datafusion/datasource-json/src/source.rs | 18 +++--------------- datafusion/datasource-parquet/src/source.rs | 16 ++-------------- datafusion/datasource/src/test_util.rs | 16 ++-------------- 8 files changed, 21 insertions(+), 111 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index fb10a96473805..6de72aa8ff720 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -20,8 +20,8 @@ use std::sync::Arc; use crate::datasource::physical_plan::{FileMeta, FileOpenFuture, FileOpener}; use crate::error::Result; -use datafusion_datasource::as_file_source; use datafusion_datasource::schema_adapter::SchemaAdapterFactory; +use datafusion_datasource::{as_file_source, impl_schema_adapter_methods}; use arrow::buffer::Buffer; use arrow::datatypes::SchemaRef; @@ -99,19 +99,7 @@ impl FileSource for ArrowSource { "arrow" } - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Arc { - Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - }) - } - - fn schema_adapter_factory(&self) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } /// The struct arrow that implements `[FileOpener]` trait diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index d6e103b1080d0..5345e911c735b 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -31,7 +31,6 @@ use arrow::{array::RecordBatch, compute::concat_batches}; use datafusion::{datasource::object_store::ObjectStoreUrl, physical_plan::PhysicalExpr}; use datafusion_common::{config::ConfigOptions, Statistics}; use datafusion_common::{internal_err, Result}; -use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use datafusion_datasource::{ file::FileSource, file_scan_config::FileScanConfig, file_stream::FileOpener, }; @@ -42,6 +41,9 @@ use datafusion_datasource::{ file_scan_config::FileScanConfigBuilder, file_stream::FileOpenFuture, source::DataSourceExec, }; +use datafusion_datasource::{ + impl_schema_adapter_methods, schema_adapter::SchemaAdapterFactory, +}; use datafusion_physical_expr::conjunction; use datafusion_physical_expr_common::physical_expr::fmt_sql; use datafusion_physical_optimizer::PhysicalOptimizerRule; @@ -242,19 +244,7 @@ impl FileSource for TestSource { } } - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Arc { - Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - }) - } - - fn schema_adapter_factory(&self) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } #[derive(Debug, Clone)] diff --git a/datafusion/core/tests/test_adapter_updated.rs b/datafusion/core/tests/test_adapter_updated.rs index 3d56f296f36b0..cae05b96158ba 100644 --- a/datafusion/core/tests/test_adapter_updated.rs +++ b/datafusion/core/tests/test_adapter_updated.rs @@ -21,6 +21,7 @@ use datafusion_common::{ColumnStatistics, DataFusionError, Result, Statistics}; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::file_stream::FileOpener; +use datafusion_datasource::impl_schema_adapter_methods; use datafusion_datasource::schema_adapter::{ SchemaAdapter, SchemaAdapterFactory, SchemaMapper, }; @@ -45,19 +46,6 @@ impl TestSource { } impl FileSource for TestSource { - fn schema_adapter_factory(&self) -> Option> { - self.schema_adapter_factory.clone() - } - - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Arc { - let mut new_source = self.clone(); - new_source.schema_adapter_factory = Some(schema_adapter_factory); - Arc::new(new_source) - } - fn file_type(&self) -> &str { "test" } @@ -98,6 +86,8 @@ impl FileSource for TestSource { fn statistics(&self) -> Result { Ok(Statistics::default()) } + + impl_schema_adapter_methods!(); } /// A test schema adapter factory diff --git a/datafusion/core/tests/test_source_adapter_tests.rs b/datafusion/core/tests/test_source_adapter_tests.rs index f69134bf495ec..68f306285f1f9 100644 --- a/datafusion/core/tests/test_source_adapter_tests.rs +++ b/datafusion/core/tests/test_source_adapter_tests.rs @@ -20,6 +20,7 @@ use datafusion_common::{ColumnStatistics, DataFusionError, Result, Statistics}; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::file_stream::FileOpener; +use datafusion_datasource::impl_schema_adapter_methods; use datafusion_datasource::schema_adapter::{ SchemaAdapter, SchemaAdapterFactory, SchemaMapper, }; @@ -47,20 +48,6 @@ impl TestSource { } impl FileSource for TestSource { - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Arc { - Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - }) - } - - fn schema_adapter_factory(&self) -> Option> { - self.schema_adapter_factory.clone() - } - fn file_type(&self) -> &str { "test" } @@ -101,6 +88,8 @@ impl FileSource for TestSource { fn statistics(&self) -> Result { Ok(Statistics::default()) } + + impl_schema_adapter_methods!(); } impl DisplayAs for TestSource { diff --git a/datafusion/datasource-avro/src/source.rs b/datafusion/datasource-avro/src/source.rs index f40a0cb021c61..2fdf34b3cc56d 100644 --- a/datafusion/datasource-avro/src/source.rs +++ b/datafusion/datasource-avro/src/source.rs @@ -28,6 +28,7 @@ use datafusion_common::Statistics; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::file_stream::FileOpener; +use datafusion_datasource::impl_schema_adapter_methods; use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; @@ -126,19 +127,7 @@ impl FileSource for AvroSource { Ok(None) } - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Arc { - Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - }) - } - - fn schema_adapter_factory(&self) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } mod private { diff --git a/datafusion/datasource-json/src/source.rs b/datafusion/datasource-json/src/source.rs index 5cd06aa776734..187876522e48e 100644 --- a/datafusion/datasource-json/src/source.rs +++ b/datafusion/datasource-json/src/source.rs @@ -32,7 +32,8 @@ use datafusion_datasource::file_meta::FileMeta; use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use datafusion_datasource::{ - as_file_source, calculate_range, ListingTableUrl, RangeCalculation, + as_file_source, calculate_range, impl_schema_adapter_methods, ListingTableUrl, + RangeCalculation, }; use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; @@ -150,20 +151,7 @@ impl FileSource for JsonSource { fn file_type(&self) -> &str { "json" } - - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Arc { - Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - }) - } - - fn schema_adapter_factory(&self) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } impl FileOpener for JsonOpener { diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 4ff3e32a33736..4de129f961c6e 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -29,6 +29,7 @@ use crate::ParquetFileReaderFactory; use datafusion_common::config::ConfigOptions; use datafusion_datasource::as_file_source; use datafusion_datasource::file_stream::FileOpener; +use datafusion_datasource::impl_schema_adapter_methods; use datafusion_datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapterFactory, }; @@ -664,18 +665,5 @@ impl FileSource for ParquetSource { ); Ok(FilterPushdownPropagation::with_filters(filters).with_updated_node(source)) } - - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Arc { - Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - }) - } - - fn schema_adapter_factory(&self) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } diff --git a/datafusion/datasource/src/test_util.rs b/datafusion/datasource/src/test_util.rs index fbb88083acb55..aac61c7812a41 100644 --- a/datafusion/datasource/src/test_util.rs +++ b/datafusion/datasource/src/test_util.rs @@ -17,7 +17,7 @@ use crate::{ file::FileSource, file_scan_config::FileScanConfig, file_stream::FileOpener, - schema_adapter::SchemaAdapterFactory, + impl_schema_adapter_methods, schema_adapter::SchemaAdapterFactory, }; use std::sync::Arc; @@ -84,19 +84,7 @@ impl FileSource for MockSource { "mock" } - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Arc { - Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - }) - } - - fn schema_adapter_factory(&self) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } /// Create a column expression From 7e9f0705584275b84ff04c2b3ff8355b4546bf94 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 19:00:22 +0800 Subject: [PATCH 15/38] refactor: clean up unused schema adapter factory methods in ParquetSource --- datafusion/core/src/datasource/mod.rs | 34 +++++++++------------ datafusion/datasource-parquet/src/source.rs | 20 ------------ 2 files changed, 15 insertions(+), 39 deletions(-) diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index dd313455f08bf..f781d5f41f091 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -51,37 +51,33 @@ pub use datafusion_physical_expr::create_ordering; #[cfg(all(test, feature = "parquet"))] mod tests { - use crate::prelude::SessionContext; - - use std::fs; - use std::sync::Arc; - - use arrow::array::{Int32Array, StringArray}; - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use arrow::record_batch::RecordBatch; - use datafusion_common::test_util::batches_to_sort_string; - use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapter, SchemaAdapterFactory, SchemaMapper, }; - use datafusion_datasource::PartitionedFile; - use datafusion_datasource_parquet::source::ParquetSource; - - use datafusion_common::record_batch; - use ::object_store::path::Path; - use ::object_store::ObjectMeta; - use datafusion_datasource::source::DataSourceExec; + use crate::prelude::SessionContext; + use arrow::{ + array::{Int32Array, StringArray}, + datatypes::{DataType, Field, Schema, SchemaRef}, + record_batch::RecordBatch, + }; + use datafusion_common::{record_batch, test_util::batches_to_sort_string}; + use datafusion_datasource::{ + file::FileSource, file_scan_config::FileScanConfigBuilder, + source::DataSourceExec, PartitionedFile, + }; + use datafusion_datasource_parquet::source::ParquetSource; + use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_physical_plan::collect; + use std::{fs, sync::Arc}; use tempfile::TempDir; - + use ::{object_store::path::Path, object_store::ObjectMeta}; #[tokio::test] async fn can_override_schema_adapter() { // Test shows that SchemaAdapter can add a column that doesn't existing in the // record batches returned from parquet. This can be useful for schema evolution // where older files may not have all columns. - use datafusion_execution::object_store::ObjectStoreUrl; let tmp_dir = TempDir::new().unwrap(); let table_dir = tmp_dir.path().join("parquet_test"); fs::DirBuilder::new().create(table_dir.as_path()).unwrap(); diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 4de129f961c6e..c9a2739ce0ba0 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -51,7 +51,6 @@ use datafusion_physical_plan::DisplayFormatType; use itertools::Itertools; use object_store::ObjectStore; - /// Execution plan for reading one or more Parquet files. /// /// ```text @@ -345,25 +344,6 @@ impl ParquetSource { self } - /// return the optional schema adapter factory - pub fn schema_adapter_factory(&self) -> Option<&Arc> { - self.schema_adapter_factory.as_ref() - } - - /// Set optional schema adapter factory. - /// - /// [`SchemaAdapterFactory`] allows user to specify how fields from the - /// parquet file get mapped to that of the table schema. The default schema - /// adapter uses arrow's cast library to map the parquet fields to the table - /// schema. - pub fn with_schema_adapter_factory( - mut self, - schema_adapter_factory: Arc, - ) -> Self { - self.schema_adapter_factory = Some(schema_adapter_factory); - self - } - /// If true, the predicate will be used during the parquet scan. /// Defaults to false /// From 4c23e826c16442d295e1b95ffcb142d5c9df8f51 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 19:10:59 +0800 Subject: [PATCH 16/38] feat: add macro for generating schema adapter methods in FileSource implementations --- datafusion/datasource/src/file.rs | 37 ++++----------- datafusion/datasource/src/macros.rs | 73 +++++++++++++++++++++++++++++ datafusion/datasource/src/mod.rs | 2 + 3 files changed, 84 insertions(+), 28 deletions(-) create mode 100644 datafusion/datasource/src/macros.rs diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 7bc78d54153e5..92b4294756821 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -22,6 +22,9 @@ use std::fmt; use std::fmt::Formatter; use std::sync::Arc; +#[allow(unused_imports)] +use crate::impl_schema_adapter_methods; + use crate::file_groups::FileGroupPartitioner; use crate::file_scan_config::FileScanConfig; use crate::file_stream::FileOpener; @@ -128,39 +131,17 @@ pub trait FileSource: Send + Sync { /// [`SchemaAdapterFactory`] allows user to specify how fields from the /// file get mapped to that of the table schema. The default implementation /// returns the original source. + /// + /// Note: You can implement this method and `schema_adapter_factory` + /// automatically using the [`impl_schema_adapter_methods`] macro. fn with_schema_adapter_factory( &self, factory: Arc, ) -> Arc; /// Returns the current schema adapter factory if set + /// + /// Note: You can implement this method and `with_schema_adapter_factory` + /// automatically using the [`impl_schema_adapter_methods`] macro. fn schema_adapter_factory(&self) -> Option>; } - -/// Helper macro to generate the two schema‐adapter methods -/// -/// Place this inside *any* `impl FileSource for YourType { … }` to -/// avoid copy-pasting `with_schema_adapter_factory` and -/// `schema_adapter_factory`. -#[macro_export] -macro_rules! impl_schema_adapter_methods { - () => { - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: std::sync::Arc< - dyn $crate::schema_adapter::SchemaAdapterFactory, - >, - ) -> std::sync::Arc { - std::sync::Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - }) - } - - fn schema_adapter_factory( - &self, - ) -> Option> { - self.schema_adapter_factory.clone() - } - }; -} diff --git a/datafusion/datasource/src/macros.rs b/datafusion/datasource/src/macros.rs new file mode 100644 index 0000000000000..8a02091346355 --- /dev/null +++ b/datafusion/datasource/src/macros.rs @@ -0,0 +1,73 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Macros for the datafusion-datasource crate + +/// Helper macro to generate schema adapter methods for FileSource implementations +/// +/// Place this inside *any* `impl FileSource for YourType { … }` to +/// avoid copy-pasting `with_schema_adapter_factory` and +/// `schema_adapter_factory`. +/// +/// # Note on path resolution +/// When this macro is used: +/// - `$crate` expands to `datafusion_datasource` (the crate root) +/// - `$crate::file::FileSource` refers to the FileSource trait from this crate +/// - `$crate::schema_adapter::SchemaAdapterFactory` refers to the SchemaAdapterFactory trait +/// +/// # Example Usage +/// +/// ```rust,no_run +/// use std::sync::Arc; +/// use datafusion_datasource::file::FileSource; +/// use datafusion_datasource::impl_schema_adapter_methods; +/// use datafusion_datasource::schema_adapter::SchemaAdapterFactory; +/// +/// struct MyFileSource { +/// schema_adapter_factory: Option>, +/// // other fields... +/// } +/// +/// impl FileSource for MyFileSource { +/// // Implement other required methods... +/// +/// // Use the macro to implement schema adapter methods +/// impl_schema_adapter_methods!(); +/// } +/// ``` +#[macro_export] +macro_rules! impl_schema_adapter_methods { + () => { + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: std::sync::Arc< + dyn $crate::schema_adapter::SchemaAdapterFactory, + >, + ) -> std::sync::Arc { + std::sync::Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + }) + } + + fn schema_adapter_factory( + &self, + ) -> Option> { + self.schema_adapter_factory.clone() + } + }; +} diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index c79efd11fcc5e..bdf2553e61ab2 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -37,6 +37,7 @@ pub mod file_meta; pub mod file_scan_config; pub mod file_sink_config; pub mod file_stream; +pub mod macros; pub mod memory; pub mod schema_adapter; pub mod sink; @@ -49,6 +50,7 @@ pub mod test_util; pub mod url; pub mod write; pub use self::file::as_file_source; +pub use self::macros::impl_schema_adapter_methods; pub use self::url::ListingTableUrl; use crate::file_groups::FileGroup; use chrono::TimeZone; From e91eb1be6d008f1ff312138f3ce5d54c673de35d Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 19:15:37 +0800 Subject: [PATCH 17/38] refactor: re-export impl_schema_adapter_methods from crate root --- datafusion/datasource/src/mod.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index bdf2553e61ab2..cf034a1ca957e 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -50,9 +50,10 @@ pub mod test_util; pub mod url; pub mod write; pub use self::file::as_file_source; -pub use self::macros::impl_schema_adapter_methods; pub use self::url::ListingTableUrl; use crate::file_groups::FileGroup; +// Re-export the macro_export macro, which is already exported at the crate root +pub use crate::impl_schema_adapter_methods; use chrono::TimeZone; use datafusion_common::stats::Precision; use datafusion_common::{exec_datafusion_err, ColumnStatistics, Result}; From 9416efb7ca998db4e3ac8a5777d8281509e179e8 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 19:20:08 +0800 Subject: [PATCH 18/38] refactor: update macro usage and documentation for schema adapter methods --- datafusion/datasource/src/file.rs | 7 ++----- datafusion/datasource/src/macros.rs | 9 +++++++++ datafusion/datasource/src/mod.rs | 2 -- 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 92b4294756821..9eb53a306dc99 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -22,9 +22,6 @@ use std::fmt; use std::fmt::Formatter; use std::sync::Arc; -#[allow(unused_imports)] -use crate::impl_schema_adapter_methods; - use crate::file_groups::FileGroupPartitioner; use crate::file_scan_config::FileScanConfig; use crate::file_stream::FileOpener; @@ -133,7 +130,7 @@ pub trait FileSource: Send + Sync { /// returns the original source. /// /// Note: You can implement this method and `schema_adapter_factory` - /// automatically using the [`impl_schema_adapter_methods`] macro. + /// automatically using the [`crate::impl_schema_adapter_methods`] macro. fn with_schema_adapter_factory( &self, factory: Arc, @@ -142,6 +139,6 @@ pub trait FileSource: Send + Sync { /// Returns the current schema adapter factory if set /// /// Note: You can implement this method and `with_schema_adapter_factory` - /// automatically using the [`impl_schema_adapter_methods`] macro. + /// automatically using the [`crate::impl_schema_adapter_methods`] macro. fn schema_adapter_factory(&self) -> Option>; } diff --git a/datafusion/datasource/src/macros.rs b/datafusion/datasource/src/macros.rs index 8a02091346355..b44a1531cc3ee 100644 --- a/datafusion/datasource/src/macros.rs +++ b/datafusion/datasource/src/macros.rs @@ -23,6 +23,15 @@ /// avoid copy-pasting `with_schema_adapter_factory` and /// `schema_adapter_factory`. /// +/// # Availability +/// +/// This macro is exported at the crate root level via `#[macro_export]`, so it can be +/// imported directly from the crate: +/// +/// ```rust,no_run +/// use datafusion_datasource::impl_schema_adapter_methods; +/// ``` +/// /// # Note on path resolution /// When this macro is used: /// - `$crate` expands to `datafusion_datasource` (the crate root) diff --git a/datafusion/datasource/src/mod.rs b/datafusion/datasource/src/mod.rs index cf034a1ca957e..1c27cd4922c39 100644 --- a/datafusion/datasource/src/mod.rs +++ b/datafusion/datasource/src/mod.rs @@ -52,8 +52,6 @@ pub mod write; pub use self::file::as_file_source; pub use self::url::ListingTableUrl; use crate::file_groups::FileGroup; -// Re-export the macro_export macro, which is already exported at the crate root -pub use crate::impl_schema_adapter_methods; use chrono::TimeZone; use datafusion_common::stats::Precision; use datafusion_common::{exec_datafusion_err, ColumnStatistics, Result}; From 5fb40dfa1d93b08935b240b5f862c0f89980dc52 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 19:22:50 +0800 Subject: [PATCH 19/38] refactor: clean up import statements in datasource module --- datafusion/core/src/datasource/mod.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index f781d5f41f091..51499c4fe427d 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -69,9 +69,10 @@ mod tests { use datafusion_datasource_parquet::source::ParquetSource; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_physical_plan::collect; + use object_store::{path::Path, ObjectMeta}; use std::{fs, sync::Arc}; use tempfile::TempDir; - use ::{object_store::path::Path, object_store::ObjectMeta}; + #[tokio::test] async fn can_override_schema_adapter() { // Test shows that SchemaAdapter can add a column that doesn't existing in the From 413ebe15c90ed8a94c70d07d2c6d0956751c5fa2 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Thu, 22 May 2025 21:09:19 +0800 Subject: [PATCH 20/38] refactor: reorganize and clean up import statements in util.rs --- .../filter_pushdown/util.rs | 43 +++++++------------ 1 file changed, 15 insertions(+), 28 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index 5345e911c735b..ac1c89679f330 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -15,54 +15,41 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - fmt::{Display, Formatter}, -}; -use std::{ - pin::Pin, - sync::Arc, - task::{Context, Poll}, -}; - 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, Statistics}; -use datafusion_common::{internal_err, Result}; -use datafusion_datasource::{ - file::FileSource, file_scan_config::FileScanConfig, file_stream::FileOpener, -}; -use datafusion_datasource::{ - file_meta::FileMeta, schema_adapter::DefaultSchemaAdapterFactory, PartitionedFile, -}; +use datafusion_common::{config::ConfigOptions, internal_err, Result, Statistics}; use datafusion_datasource::{ + file::FileSource, file_meta::FileMeta, file_scan_config::FileScanConfig, file_scan_config::FileScanConfigBuilder, file_stream::FileOpenFuture, - source::DataSourceExec, -}; -use datafusion_datasource::{ - impl_schema_adapter_methods, schema_adapter::SchemaAdapterFactory, + file_stream::FileOpener, impl_schema_adapter_methods, + schema_adapter::DefaultSchemaAdapterFactory, schema_adapter::SchemaAdapterFactory, + source::DataSourceExec, PartitionedFile, }; use datafusion_physical_expr::conjunction; use datafusion_physical_expr_common::physical_expr::fmt_sql; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::{ - displayable, metrics::ExecutionPlanMetricsSet, DisplayFormatType, ExecutionPlan, -}; -use datafusion_physical_plan::{ + displayable, filter::FilterExec, filter_pushdown::{ ChildPushdownResult, FilterDescription, FilterPushdownPropagation, PredicateSupport, PredicateSupports, }, - DisplayAs, PlanProperties, + metrics::ExecutionPlanMetricsSet, + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, }; - use futures::stream::BoxStream; use futures::{FutureExt, Stream}; use object_store::ObjectStore; - +use std::{ + any::Any, + fmt::{Display, Formatter}, + pin::Pin, + sync::Arc, + task::{Context, Poll}, +}; pub struct TestOpener { batches: Vec, batch_size: Option, From f11134a3bddc1ec01e5e9b99715247b79db16d2b Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 23 May 2025 08:30:50 +0800 Subject: [PATCH 21/38] Resolve merge conflict --- .../datasource-parquet/src/file_format.rs | 45 +++++++++---------- 1 file changed, 22 insertions(+), 23 deletions(-) diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 35d5b07f88b36..c9b83b28ef784 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -1729,6 +1729,28 @@ fn create_max_min_accs( (max_values, min_values) } +/// Converts a ParquetSource to an Arc and applies the schema adapter factory +/// from the FileScanConfig if present. +/// +/// # Arguments +/// * `source` - The ParquetSource to convert +/// * `conf` - FileScanConfig that may contain a schema adapter factory +/// # Returns +/// The converted FileSource with schema adapter factory applied if provided +fn apply_schema_adapter( + source: ParquetSource, + conf: &FileScanConfig, +) -> Arc { + let file_source: Arc = source.into(); + + // If the FileScanConfig.file_source() has a schema adapter factory, apply it + if let Some(factory) = conf.file_source().schema_adapter_factory() { + file_source.with_schema_adapter_factory(factory.clone()) + } else { + file_source + } +} + #[cfg(test)] mod tests { use std::sync::Arc; @@ -1945,26 +1967,3 @@ mod tests { assert_eq!(result, expected_schema); } } - -/// Converts a ParquetSource to an Arc and applies the schema adapter factory -/// from the FileScanConfig if present. -/// -/// # Arguments -/// * `source` - The ParquetSource to convert -/// * `conf` - FileScanConfig that may contain a schema adapter factory -/// # Returns -/// The converted FileSource with schema adapter factory applied if provided -fn apply_schema_adapter( - source: ParquetSource, - conf: &FileScanConfig, -) -> Arc { - // Convert the ParquetSource to Arc - let file_source: Arc = source.into(); - - // If the FileScanConfig.file_source() has a schema adapter factory, apply it - if let Some(factory) = conf.file_source().schema_adapter_factory() { - file_source.with_schema_adapter_factory(factory.clone()) - } else { - file_source - } -} From c6ff4d56272c074bf43244587e5d2e4aaf2552e8 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 23 May 2025 08:40:59 +0800 Subject: [PATCH 22/38] Export macro with local inner macros for improved encapsulation --- datafusion/datasource/src/macros.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/datasource/src/macros.rs b/datafusion/datasource/src/macros.rs index b44a1531cc3ee..7f7950da6a888 100644 --- a/datafusion/datasource/src/macros.rs +++ b/datafusion/datasource/src/macros.rs @@ -58,7 +58,7 @@ /// impl_schema_adapter_methods!(); /// } /// ``` -#[macro_export] +#[macro_export(local_inner_macros)] macro_rules! impl_schema_adapter_methods { () => { fn with_schema_adapter_factory( From cb27246580806f1af87b6e9e4b3ae219be498a9d Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 23 May 2025 09:43:21 +0800 Subject: [PATCH 23/38] fix clippy error --- datafusion/core/src/datasource/mod.rs | 2 +- .../core/tests/schema_adapter_factory_tests.rs | 2 +- datafusion/core/tests/test_adapter_updated.rs | 14 ++++++++++++-- datafusion/datasource-parquet/src/file_format.rs | 4 +++- .../tests/apply_schema_adapter_tests.rs | 2 +- 5 files changed, 18 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 51499c4fe427d..f0c6771515a7f 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -126,7 +126,7 @@ mod tests { let base_conf = FileScanConfigBuilder::new( ObjectStoreUrl::local_filesystem(), schema, - source.into(), + source, ) .with_file(partitioned_file) .build(); diff --git a/datafusion/core/tests/schema_adapter_factory_tests.rs b/datafusion/core/tests/schema_adapter_factory_tests.rs index 4c840147abe22..ab60cc222d8d3 100644 --- a/datafusion/core/tests/schema_adapter_factory_tests.rs +++ b/datafusion/core/tests/schema_adapter_factory_tests.rs @@ -195,7 +195,7 @@ fn test_schema_adapter_preservation() { let config_builder = FileScanConfigBuilder::new( ObjectStoreUrl::parse("file:///path/to/parquet").unwrap(), schema.clone(), - file_source.clone().into(), + file_source.clone(), ); let config = config_builder.build(); diff --git a/datafusion/core/tests/test_adapter_updated.rs b/datafusion/core/tests/test_adapter_updated.rs index cae05b96158ba..88650abcff4aa 100644 --- a/datafusion/core/tests/test_adapter_updated.rs +++ b/datafusion/core/tests/test_adapter_updated.rs @@ -21,7 +21,6 @@ use datafusion_common::{ColumnStatistics, DataFusionError, Result, Statistics}; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::file_stream::FileOpener; -use datafusion_datasource::impl_schema_adapter_methods; use datafusion_datasource::schema_adapter::{ SchemaAdapter, SchemaAdapterFactory, SchemaMapper, }; @@ -87,7 +86,18 @@ impl FileSource for TestSource { Ok(Statistics::default()) } - impl_schema_adapter_methods!(); + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Arc { + Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + }) + } + + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() + } } /// A test schema adapter factory diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index c9b83b28ef784..1ad866fc6df66 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -50,6 +50,7 @@ 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::schema_adapter::SchemaAdapterFactory; use datafusion_datasource::sink::{DataSink, DataSinkExec}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; @@ -1745,7 +1746,8 @@ fn apply_schema_adapter( // If the FileScanConfig.file_source() has a schema adapter factory, apply it if let Some(factory) = conf.file_source().schema_adapter_factory() { - file_source.with_schema_adapter_factory(factory.clone()) + file_source + .with_schema_adapter_factory(Arc::::clone(&factory)) } else { file_source } diff --git a/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs index 9e0f3ff2838b6..33e3c2892f1bc 100644 --- a/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs +++ b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs @@ -164,7 +164,7 @@ mod parquet_adapter_tests { let config = FileScanConfigBuilder::new( ObjectStoreUrl::local_filesystem(), schema.clone(), - file_source.into(), // Pass file_source as the third parameter + file_source, ) .build(); From 613d115b26c339763cccb58a264a74bb4fb2d440 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 23 May 2025 11:27:29 +0800 Subject: [PATCH 24/38] fix doc tests --- datafusion/datasource/src/file_scan_config.rs | 12 +++- datafusion/datasource/src/macros.rs | 67 ++++++++++++++++++- 2 files changed, 74 insertions(+), 5 deletions(-) diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 0788ee87bad7a..09db526d26cc3 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -24,6 +24,8 @@ use std::{ }; use crate::file_groups::FileGroup; +#[allow(unused_imports)] +use crate::schema_adapter::SchemaAdapterFactory; use crate::{ display::FileGroupsDisplay, file::FileSource, @@ -83,6 +85,7 @@ use log::{debug, warn}; /// # use datafusion_execution::object_store::ObjectStoreUrl; /// # use datafusion_physical_plan::ExecutionPlan; /// # use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +/// # use datafusion_datasource::schema_adapter::SchemaAdapterFactory; /// # let file_schema = Arc::new(Schema::new(vec![ /// # Field::new("c1", DataType::Int32, false), /// # Field::new("c2", DataType::Int32, false), @@ -92,7 +95,8 @@ use log::{debug, warn}; /// # // Note: crate mock ParquetSource, as ParquetSource is not in the datasource crate /// #[derive(Clone)] /// # struct ParquetSource { -/// # projected_statistics: Option +/// # projected_statistics: Option, +/// # schema_adapter_factory: Option> /// # }; /// # impl FileSource for ParquetSource { /// # fn create_file_opener(&self, _: Arc, _: &FileScanConfig, _: usize) -> Arc { unimplemented!() } @@ -100,13 +104,15 @@ use log::{debug, warn}; /// # 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)} ) } +/// # 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) -> datafusion_common::Result { Ok(self.projected_statistics.clone().expect("projected_statistics should be set")) } /// # fn file_type(&self) -> &str { "parquet" } +/// # fn with_schema_adapter_factory(&self, factory: Arc) -> Arc { Arc::new(Self {projected_statistics: self.projected_statistics.clone(), schema_adapter_factory: Some(factory)} ) } +/// # fn schema_adapter_factory(&self) -> Option> { self.schema_adapter_factory.clone() } /// # } /// # impl ParquetSource { -/// # fn new() -> Self { Self {projected_statistics: None} } +/// # fn new() -> Self { Self {projected_statistics: None, schema_adapter_factory: None} } /// # } /// // create FileScan config for reading parquet files from file:// /// let object_store_url = ObjectStoreUrl::local_filesystem(); diff --git a/datafusion/datasource/src/macros.rs b/datafusion/datasource/src/macros.rs index 7f7950da6a888..c7a4058f2310e 100644 --- a/datafusion/datasource/src/macros.rs +++ b/datafusion/datasource/src/macros.rs @@ -42,17 +42,80 @@ /// /// ```rust,no_run /// use std::sync::Arc; +/// use std::any::Any; +/// use std::fmt::{Formatter, Display, self}; +/// use arrow::datatypes::SchemaRef; +/// use datafusion_common::{Result, Statistics}; +/// use object_store::ObjectStore; +/// use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +/// use datafusion_physical_plan::DisplayFormatType; +/// use datafusion_physical_expr_common::sort_expr::LexOrdering; /// use datafusion_datasource::file::FileSource; +/// use datafusion_datasource::file_stream::FileOpener; +/// use datafusion_datasource::file_scan_config::FileScanConfig; /// use datafusion_datasource::impl_schema_adapter_methods; /// use datafusion_datasource::schema_adapter::SchemaAdapterFactory; /// +/// #[derive(Clone)] /// struct MyFileSource { +/// schema: SchemaRef, +/// batch_size: usize, +/// statistics: Statistics, +/// projection: Option>, /// schema_adapter_factory: Option>, -/// // other fields... +/// metrics: ExecutionPlanMetricsSet, /// } /// /// impl FileSource for MyFileSource { -/// // Implement other required methods... +/// fn create_file_opener( +/// &self, +/// object_store: Arc, +/// base_config: &FileScanConfig, +/// partition: usize, +/// ) -> Arc { +/// // Implementation here +/// unimplemented!() +/// } +/// +/// fn as_any(&self) -> &dyn Any { +/// self +/// } +/// +/// fn with_batch_size(&self, batch_size: usize) -> Arc { +/// let mut new_source = self.clone(); +/// new_source.batch_size = batch_size; +/// Arc::new(new_source) +/// } +/// +/// fn with_schema(&self, schema: SchemaRef) -> Arc { +/// let mut new_source = self.clone(); +/// new_source.schema = schema; +/// Arc::new(new_source) +/// } +/// +/// fn with_projection(&self, config: &FileScanConfig) -> Arc { +/// let mut new_source = self.clone(); +/// new_source.projection = config.file_column_projection_indices(); +/// Arc::new(new_source) +/// } +/// +/// fn with_statistics(&self, statistics: Statistics) -> Arc { +/// let mut new_source = self.clone(); +/// new_source.statistics = statistics; +/// Arc::new(new_source) +/// } +/// +/// fn metrics(&self) -> &ExecutionPlanMetricsSet { +/// &self.metrics +/// } +/// +/// fn statistics(&self) -> Result { +/// Ok(self.statistics.clone()) +/// } +/// +/// fn file_type(&self) -> &str { +/// "my_file_type" +/// } /// /// // Use the macro to implement schema adapter methods /// impl_schema_adapter_methods!(); From d2027f111d64cd5e5fd05169710d3d6a0e15c07a Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 23 May 2025 12:32:23 +0800 Subject: [PATCH 25/38] fix CI error --- datafusion/core/tests/schema_adapter_factory_tests.rs | 9 ++++++--- datafusion/datasource-parquet/src/file_format.rs | 2 +- datafusion/datasource-parquet/src/source.rs | 2 +- datafusion/datasource/src/file.rs | 2 +- 4 files changed, 9 insertions(+), 6 deletions(-) diff --git a/datafusion/core/tests/schema_adapter_factory_tests.rs b/datafusion/core/tests/schema_adapter_factory_tests.rs index ab60cc222d8d3..1b40d948165e6 100644 --- a/datafusion/core/tests/schema_adapter_factory_tests.rs +++ b/datafusion/core/tests/schema_adapter_factory_tests.rs @@ -24,6 +24,7 @@ use std::sync::Arc; // Import ArrowSource from the correct location // ArrowSource is part of the core DataFusion package use datafusion::datasource::physical_plan::ArrowSource; +use datafusion_datasource::PartitionedFile; #[cfg(feature = "parquet")] use datafusion_datasource_parquet::source::ParquetSource; @@ -177,7 +178,7 @@ fn test_file_source_conversion() { #[cfg(feature = "parquet")] #[test] fn test_schema_adapter_preservation() { - use datafusion::datasource::object_store::ObjectStoreUrl; + use datafusion::execution::object_store::ObjectStoreUrl; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; // Create a test schema @@ -193,10 +194,12 @@ fn test_schema_adapter_preservation() { // Create a FileScanConfig with the source let config_builder = FileScanConfigBuilder::new( - ObjectStoreUrl::parse("file:///path/to/parquet").unwrap(), + ObjectStoreUrl::local_filesystem(), schema.clone(), file_source.clone(), - ); + ) + // Add a file to make it valid + .with_file(PartitionedFile::new("test.parquet", 100)); let config = config_builder.build(); diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 1ad866fc6df66..3865834de9534 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -1730,7 +1730,7 @@ fn create_max_min_accs( (max_values, min_values) } -/// Converts a ParquetSource to an Arc and applies the schema adapter factory +/// Converts a ParquetSource to an Arc<dyn FileSource> and applies the schema adapter factory /// from the FileScanConfig if present. /// /// # Arguments diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 98c1cfd81d406..c0f848d63f261 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -428,7 +428,7 @@ pub(crate) fn parse_coerce_int96_string( } } -/// Allows easy conversion from ParquetSource to Arc +/// Allows easy conversion from ParquetSource to Arc<dyn FileSource> impl From for Arc { fn from(source: ParquetSource) -> Self { as_file_source(source) diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 9eb53a306dc99..d0557e9f0872c 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -36,7 +36,7 @@ use datafusion_physical_plan::DisplayFormatType; use object_store::ObjectStore; -/// Helper function to convert any type implementing FileSource to Arc +/// Helper function to convert any type implementing FileSource to Arc<dyn FileSource> pub fn as_file_source(source: T) -> Arc { Arc::new(source) } From 727032ba56e51cce2713eaa5687b960ab5284388 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Fri, 23 May 2025 14:55:25 +0800 Subject: [PATCH 26/38] Add metrics initialization to TestSource constructor --- datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index ac1c89679f330..dc4d77194c082 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -117,6 +117,7 @@ impl TestSource { fn new(support: bool, batches: Vec) -> Self { Self { support, + metrics: ExecutionPlanMetricsSet::new(), batches, ..Default::default() } From 148148c6ebe071650fae9d179bc54eb3499540e0 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 26 May 2025 10:53:11 +0800 Subject: [PATCH 27/38] Add comment for test_multi_source_schema_adapter_reuse --- .../integration_tests/schema_adapter_integration_tests.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/datafusion/core/tests/integration_tests/schema_adapter_integration_tests.rs b/datafusion/core/tests/integration_tests/schema_adapter_integration_tests.rs index 98ca808eebf4e..25cbb605a1953 100644 --- a/datafusion/core/tests/integration_tests/schema_adapter_integration_tests.rs +++ b/datafusion/core/tests/integration_tests/schema_adapter_integration_tests.rs @@ -148,6 +148,12 @@ async fn test_parquet_integration_with_schema_adapter() -> Result<()> { #[tokio::test] async fn test_multi_source_schema_adapter_reuse() -> Result<()> { + // This test verifies that the same schema adapter factory can be reused + // across different file source types. This is important for ensuring that: + // 1. The schema adapter factory interface works uniformly across all source types + // 2. The factory can be shared and cloned efficiently using Arc + // 3. Various data source implementations correctly implement the schema adapter factory pattern + // Create a test factory let factory = Arc::new(UppercaseAdapterFactory {}); From d3b1680fc7d59112d210813ef2b0fc8f22c1aa9e Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 26 May 2025 11:03:50 +0800 Subject: [PATCH 28/38] reduce test files, move non-redundant tests, consolidate in one file --- .../schema_adapter_integration_tests.rs | 57 +++++ .../tests/schema_adapter_factory_tests.rs | 208 ------------------ 2 files changed, 57 insertions(+), 208 deletions(-) delete mode 100644 datafusion/core/tests/schema_adapter_factory_tests.rs diff --git a/datafusion/core/tests/integration_tests/schema_adapter_integration_tests.rs b/datafusion/core/tests/integration_tests/schema_adapter_integration_tests.rs index 25cbb605a1953..38c2ee582a616 100644 --- a/datafusion/core/tests/integration_tests/schema_adapter_integration_tests.rs +++ b/datafusion/core/tests/integration_tests/schema_adapter_integration_tests.rs @@ -27,6 +27,7 @@ use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfigBuilder; use datafusion_datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory}; use datafusion_datasource::source::DataSourceExec; +use datafusion_datasource::PartitionedFile; use std::sync::Arc; use tempfile::TempDir; @@ -201,3 +202,59 @@ fn test_from_implementations() { #[cfg(feature = "json")] test_from_impl::("json"); } + +/// A simple test schema adapter factory that doesn't modify the schema +#[derive(Debug)] +struct TestSchemaAdapterFactory {} + +impl SchemaAdapterFactory for TestSchemaAdapterFactory { + fn create(&self, schema: &Schema) -> Result> { + Ok(Box::new(TestSchemaAdapter { + input_schema: Arc::new(schema.clone()), + })) + } +} + +/// A test schema adapter that passes through data unmodified +#[derive(Debug)] +struct TestSchemaAdapter { + input_schema: SchemaRef, +} + +impl SchemaAdapter for TestSchemaAdapter { + fn adapt(&self, record_batch: RecordBatch) -> Result { + // Just pass through the batch unmodified + Ok(record_batch) + } + + fn output_schema(&self) -> SchemaRef { + self.input_schema.clone() + } +} + +#[cfg(feature = "parquet")] +#[test] +fn test_schema_adapter_preservation() { + // Create a test schema + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, true), + ])); + + // Create source with schema adapter factory + let source = ParquetSource::default(); + let factory = Arc::new(TestSchemaAdapterFactory {}); + let file_source = source.with_schema_adapter_factory(factory); + + // Create a FileScanConfig with the source + let config_builder = + FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), schema.clone()) + .with_source(file_source.clone()) + // Add a file to make it valid + .with_file(PartitionedFile::new("test.parquet", 100)); + + let config = config_builder.build(); + + // Verify the schema adapter factory is present in the file source + assert!(config.source().schema_adapter_factory().is_some()); +} diff --git a/datafusion/core/tests/schema_adapter_factory_tests.rs b/datafusion/core/tests/schema_adapter_factory_tests.rs deleted file mode 100644 index 1b40d948165e6..0000000000000 --- a/datafusion/core/tests/schema_adapter_factory_tests.rs +++ /dev/null @@ -1,208 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::ColumnStatistics; -use datafusion_datasource::file::FileSource; -use datafusion_datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory}; -use std::sync::Arc; - -// Import ArrowSource from the correct location -// ArrowSource is part of the core DataFusion package -use datafusion::datasource::physical_plan::ArrowSource; -use datafusion_datasource::PartitionedFile; - -#[cfg(feature = "parquet")] -use datafusion_datasource_parquet::source::ParquetSource; - -#[cfg(feature = "avro")] -use datafusion_datasource_avro::source::AvroSource; - -// JSON and CSV sources are not available in the current feature set -// #[cfg(feature = "json")] -// use datafusion_datasource_json::JsonSource; - -// #[cfg(feature = "csv")] -// use datafusion_datasource_csv::CsvSource; - -/// A test schema adapter factory that adds an extra column -#[derive(Debug)] -struct TestSchemaAdapterFactory {} - -impl SchemaAdapterFactory for TestSchemaAdapterFactory { - fn create( - &self, - projected_table_schema: SchemaRef, - _table_schema: SchemaRef, - ) -> Box { - Box::new(TestSchemaAdapter { - input_schema: projected_table_schema, - }) - } -} - -/// A test schema adapter that adds a column -#[derive(Debug)] -struct TestSchemaAdapter { - input_schema: SchemaRef, -} - -impl SchemaAdapter for TestSchemaAdapter { - fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { - let field = self.input_schema.field(index); - file_schema.fields.find(field.name()).map(|(i, _)| i) - } - - fn map_schema( - &self, - file_schema: &Schema, - ) -> datafusion_common::Result<( - Arc, - Vec, - )> { - let mut projection = Vec::with_capacity(file_schema.fields().len()); - for (file_idx, file_field) in file_schema.fields().iter().enumerate() { - if self.input_schema.fields().find(file_field.name()).is_some() { - projection.push(file_idx); - } - } - - // Create a schema mapper that adds an adapted_column - #[derive(Debug)] - struct TestSchemaMapping { - #[allow(dead_code)] - input_schema: SchemaRef, - } - - impl datafusion_datasource::schema_adapter::SchemaMapper for TestSchemaMapping { - fn map_batch( - &self, - batch: arrow::record_batch::RecordBatch, - ) -> datafusion_common::Result { - // In a real adapter, we would transform the record batch here - // For this test, we're just verifying the adapter was called correctly - Ok(batch) - } - - fn map_column_statistics( - &self, - file_col_statistics: &[ColumnStatistics], - ) -> datafusion_common::Result> { - // For testing, just return the input statistics - Ok(file_col_statistics.to_vec()) - } - } - - Ok(( - Arc::new(TestSchemaMapping { - input_schema: self.input_schema.clone(), - }), - projection, - )) - } -} - -// General function to test schema adapter factory functionality for any file source -fn test_generic_schema_adapter_factory(file_type: &str) { - let source = T::default(); - - // Test that schema adapter factory is initially None - assert!(source.schema_adapter_factory().is_none()); - - // Add a schema adapter factory - let factory = Arc::new(TestSchemaAdapterFactory {}); - let source_with_adapter = source.with_schema_adapter_factory(factory); - - // Verify schema adapter factory is now set - assert!(source_with_adapter.schema_adapter_factory().is_some()); - - // Check that file_type method returns the expected value - assert_eq!(source_with_adapter.file_type(), file_type); -} - -#[test] -fn test_arrow_source_schema_adapter_factory() { - test_generic_schema_adapter_factory::("arrow"); -} - -#[cfg(feature = "parquet")] -#[test] -fn test_parquet_source_schema_adapter_factory() { - test_generic_schema_adapter_factory::("parquet"); -} - -#[cfg(feature = "avro")] -#[test] -fn test_avro_source_schema_adapter_factory() { - test_generic_schema_adapter_factory::("avro"); -} - -// JSON and CSV sources are not available in the current feature set -// #[test] -// fn test_json_source_schema_adapter_factory() { -// test_generic_schema_adapter_factory::("json"); -// } - -// #[test] -// fn test_csv_source_schema_adapter_factory() { -// test_generic_schema_adapter_factory::("csv"); -// } - -#[test] -fn test_file_source_conversion() { - // Test the as_file_source function - let arrow_source = ArrowSource::default(); - let file_source = datafusion_datasource::as_file_source(arrow_source); - assert_eq!(file_source.file_type(), "arrow"); - - // Test the From implementation for ArrowSource - let arrow_source = ArrowSource::default(); - let file_source: Arc = arrow_source.into(); - assert_eq!(file_source.file_type(), "arrow"); -} - -#[cfg(feature = "parquet")] -#[test] -fn test_schema_adapter_preservation() { - use datafusion::execution::object_store::ObjectStoreUrl; - use datafusion_datasource::file_scan_config::FileScanConfigBuilder; - - // Create a test schema - let schema = Arc::new(Schema::new(vec![ - Field::new("id", DataType::Int32, false), - Field::new("name", DataType::Utf8, true), - ])); - - // Create source with schema adapter factory - let source = ParquetSource::default(); - let factory = Arc::new(TestSchemaAdapterFactory {}); - let file_source = source.with_schema_adapter_factory(factory); - - // Create a FileScanConfig with the source - let config_builder = FileScanConfigBuilder::new( - ObjectStoreUrl::local_filesystem(), - schema.clone(), - file_source.clone(), - ) - // Add a file to make it valid - .with_file(PartitionedFile::new("test.parquet", 100)); - - let config = config_builder.build(); - - // Verify the schema adapter factory is present in the file source - assert!(config.file_source().schema_adapter_factory().is_some()); -} From 79a56f68b3aba8d9f1d71058cf752e24f69d2b30 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 26 May 2025 11:12:49 +0800 Subject: [PATCH 29/38] test_schema_adapter - add comments --- datafusion/core/tests/test_adapter_updated.rs | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/datafusion/core/tests/test_adapter_updated.rs b/datafusion/core/tests/test_adapter_updated.rs index 88650abcff4aa..c85b9a3447361 100644 --- a/datafusion/core/tests/test_adapter_updated.rs +++ b/datafusion/core/tests/test_adapter_updated.rs @@ -164,6 +164,19 @@ impl SchemaMapper for TestSchemaMapping { #[test] fn test_schema_adapter() { + // This test verifies the functionality of the SchemaAdapter and SchemaAdapterFactory + // components used in DataFusion's file sources. + // + // The test specifically checks: + // 1. Creating and attaching a schema adapter factory to a file source + // 2. Creating a schema adapter using the factory + // 3. The schema adapter's ability to map column indices between a table schema and a file schema + // 4. The schema adapter's ability to create a projection that selects only the columns + // from the file schema that are present in the table schema + // + // Schema adapters are used when the schema of data in files doesn't exactly match + // the schema expected by the query engine, allowing for field mapping and data transformation. + // Create a test schema let table_schema = Arc::new(Schema::new(vec![ Field::new("id", DataType::Int32, false), From 55dc418958b4611c8207f6e05dfca821c17b9091 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 26 May 2025 11:17:53 +0800 Subject: [PATCH 30/38] remove redundant tests --- .../core/tests/test_source_adapter_tests.rs | 233 ------------------ 1 file changed, 233 deletions(-) delete mode 100644 datafusion/core/tests/test_source_adapter_tests.rs diff --git a/datafusion/core/tests/test_source_adapter_tests.rs b/datafusion/core/tests/test_source_adapter_tests.rs deleted file mode 100644 index 68f306285f1f9..0000000000000 --- a/datafusion/core/tests/test_source_adapter_tests.rs +++ /dev/null @@ -1,233 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::{ColumnStatistics, DataFusionError, Result, Statistics}; -use datafusion_datasource::file::FileSource; -use datafusion_datasource::file_scan_config::FileScanConfig; -use datafusion_datasource::file_stream::FileOpener; -use datafusion_datasource::impl_schema_adapter_methods; -use datafusion_datasource::schema_adapter::{ - SchemaAdapter, SchemaAdapterFactory, SchemaMapper, -}; -use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; -use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; -use object_store::ObjectStore; -use std::fmt::Debug; -use std::sync::Arc; - -// Simple TestSource implementation for testing without dependency on private module -#[derive(Clone, Debug)] -struct TestSource { - #[allow(dead_code)] - has_adapter: bool, - schema_adapter_factory: Option>, -} - -impl TestSource { - fn new(has_adapter: bool) -> Self { - Self { - has_adapter, - schema_adapter_factory: None, - } - } -} - -impl FileSource for TestSource { - fn file_type(&self) -> &str { - "test" - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn create_file_opener( - &self, - _store: Arc, - _conf: &FileScanConfig, - _index: usize, - ) -> Arc { - unimplemented!("Not needed for this test") - } - - fn with_batch_size(&self, _batch_size: usize) -> Arc { - Arc::new(self.clone()) - } - - fn with_schema(&self, _schema: SchemaRef) -> Arc { - Arc::new(self.clone()) - } - - fn with_projection(&self, _projection: &FileScanConfig) -> Arc { - Arc::new(self.clone()) - } - - fn with_statistics(&self, _statistics: Statistics) -> Arc { - Arc::new(self.clone()) - } - - fn metrics(&self) -> &ExecutionPlanMetricsSet { - unimplemented!("Not needed for this test") - } - - fn statistics(&self) -> Result { - Ok(Statistics::default()) - } - - impl_schema_adapter_methods!(); -} - -impl DisplayAs for TestSource { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default - | DisplayFormatType::Verbose - | DisplayFormatType::TreeRender => { - write!(f, "TestSource") - } - } - } -} - -/// A simple schema adapter factory for testing -#[derive(Debug)] -struct TestFilterPushdownAdapterFactory {} - -impl SchemaAdapterFactory for TestFilterPushdownAdapterFactory { - fn create( - &self, - projected_table_schema: SchemaRef, - _table_schema: SchemaRef, - ) -> Box { - Box::new(TestFilterPushdownAdapter { - input_schema: projected_table_schema, - }) - } -} - -/// A simple schema adapter for testing -#[derive(Debug)] -struct TestFilterPushdownAdapter { - input_schema: SchemaRef, -} - -impl SchemaAdapter for TestFilterPushdownAdapter { - fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { - let field = self.input_schema.field(index); - file_schema.fields.find(field.name()).map(|(i, _)| i) - } - - fn map_schema( - &self, - file_schema: &Schema, - ) -> Result<(Arc, Vec)> { - let mut projection = Vec::with_capacity(file_schema.fields().len()); - for (file_idx, file_field) in file_schema.fields().iter().enumerate() { - if self.input_schema.fields().find(file_field.name()).is_some() { - projection.push(file_idx); - } - } - - // Create a schema mapper that modifies column names - #[derive(Debug)] - struct TestSchemaMapping { - #[allow(dead_code)] - input_schema: SchemaRef, - } - - impl SchemaMapper for TestSchemaMapping { - fn map_batch( - &self, - batch: arrow::record_batch::RecordBatch, - ) -> Result { - // For testing, just return the original batch - Ok(batch) - } - - fn map_column_statistics( - &self, - file_col_statistics: &[ColumnStatistics], - ) -> Result> { - // For testing, just return the input statistics - Ok(file_col_statistics.to_vec()) - } - } - - Ok(( - Arc::new(TestSchemaMapping { - input_schema: self.input_schema.clone(), - }), - projection, - )) - } -} - -#[test] -fn test_test_source_schema_adapter_factory() { - // Create a TestSource instance - let schema = Arc::new(Schema::new(vec![ - Field::new("id", DataType::Int32, false), - Field::new("value", DataType::Utf8, true), - ])); - - let source = TestSource::new(true); - - // Verify initial state has no adapter - assert!(source.schema_adapter_factory().is_none()); - - // Apply an adapter factory - let factory = Arc::new(TestFilterPushdownAdapterFactory {}); - let source_with_adapter = source.with_schema_adapter_factory(factory); - - // Verify adapter was set - assert!(source_with_adapter.schema_adapter_factory().is_some()); - - // Create an adapter - let adapter_factory = source_with_adapter.schema_adapter_factory().unwrap(); - let adapter = adapter_factory.create(Arc::clone(&schema), Arc::clone(&schema)); - - // Create a file schema to test mapping - let file_schema = Schema::new(vec![ - Field::new("id", DataType::Int32, false), - Field::new("value", DataType::Utf8, true), - ]); - - // Test column mapping - let id_index = adapter.map_column_index(0, &file_schema); - assert_eq!(id_index, Some(0)); - - // Test schema mapping - let (_mapper, projection) = adapter.map_schema(&file_schema).unwrap(); - assert_eq!(projection.len(), 2); // Both columns should be included - - // Check file type remains unchanged - assert_eq!(source_with_adapter.file_type(), "test"); -} - -#[test] -fn test_test_source_default() { - // Create a TestSource with default values - let source = TestSource::new(false); - - // Ensure schema_adapter_factory is None by default - assert!(source.schema_adapter_factory().is_none()); -} From e8f8df4de9e1065ae48893935d3db9cc14bcd865 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 26 May 2025 11:29:45 +0800 Subject: [PATCH 31/38] Refactor schema adapter application to use ParquetSource method directly --- .../datasource-parquet/src/file_format.rs | 25 +------------------ datafusion/datasource-parquet/src/source.rs | 19 ++++++++++++++ .../tests/apply_schema_adapter_tests.rs | 13 ++-------- 3 files changed, 22 insertions(+), 35 deletions(-) diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index 3865834de9534..af9e6544b7142 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -424,7 +424,7 @@ impl FileFormat for ParquetFormat { source = source.with_metadata_size_hint(metadata_size_hint) } // Apply schema adapter factory before building the new config - let file_source = apply_schema_adapter(source, &conf); + let file_source = source.apply_schema_adapter(&conf); let conf = FileScanConfigBuilder::from(conf) .with_source(file_source) @@ -1730,29 +1730,6 @@ fn create_max_min_accs( (max_values, min_values) } -/// Converts a ParquetSource to an Arc<dyn FileSource> and applies the schema adapter factory -/// from the FileScanConfig if present. -/// -/// # Arguments -/// * `source` - The ParquetSource to convert -/// * `conf` - FileScanConfig that may contain a schema adapter factory -/// # Returns -/// The converted FileSource with schema adapter factory applied if provided -fn apply_schema_adapter( - source: ParquetSource, - conf: &FileScanConfig, -) -> Arc { - let file_source: Arc = source.into(); - - // If the FileScanConfig.file_source() has a schema adapter factory, apply it - if let Some(factory) = conf.file_source().schema_adapter_factory() { - file_source - .with_schema_adapter_factory(Arc::::clone(&factory)) - } else { - file_source - } -} - #[cfg(test)] mod tests { use std::sync::Arc; diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index c0f848d63f261..30b774d08f1d6 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -408,6 +408,25 @@ impl ParquetSource { fn bloom_filter_on_read(&self) -> bool { self.table_parquet_options.global.bloom_filter_on_read } + + /// Applies schema adapter factory from the FileScanConfig if present. + /// + /// # Arguments + /// * `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) -> Arc { + let file_source: Arc = self.into(); + + // If the FileScanConfig.file_source() has a schema adapter factory, apply it + if let Some(factory) = conf.file_source().schema_adapter_factory() { + file_source.with_schema_adapter_factory( + Arc::::clone(&factory), + ) + } else { + file_source + } + } } /// Parses datafusion.common.config.ParquetOptions.coerce_int96 String to a arrow_schema.datatype.TimeUnit diff --git a/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs index 33e3c2892f1bc..0765eb1c9dd94 100644 --- a/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs +++ b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs @@ -126,21 +126,12 @@ mod parquet_adapter_tests { } } - // Implementation of apply_schema_adapter for testing purposes - // This mimics the private function in the datafusion-parquet crate + // Use the apply_schema_adapter method on ParquetSource fn apply_schema_adapter( source: ParquetSource, conf: &FileScanConfig, ) -> Arc { - // Convert the ParquetSource to Arc - let file_source: Arc = Arc::new(source); - - // If the FileScanConfig.file_source() has a schema adapter factory, apply it - if let Some(factory) = conf.file_source().schema_adapter_factory() { - file_source.with_schema_adapter_factory(factory) - } else { - file_source - } + source.apply_schema_adapter(conf) } #[test] From 6154b2dabf6b6a6774e9822bb3bc3806f6127735 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 26 May 2025 11:46:15 +0800 Subject: [PATCH 32/38] Refactor apply_schema_adapter usage to call method directly on ParquetSource --- .../tests/apply_schema_adapter_tests.rs | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs index 0765eb1c9dd94..edf828e71c272 100644 --- a/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs +++ b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs @@ -126,14 +126,6 @@ mod parquet_adapter_tests { } } - // Use the apply_schema_adapter method on ParquetSource - fn apply_schema_adapter( - source: ParquetSource, - conf: &FileScanConfig, - ) -> Arc { - source.apply_schema_adapter(conf) - } - #[test] fn test_apply_schema_adapter_with_factory() { // Create a schema @@ -160,7 +152,7 @@ mod parquet_adapter_tests { .build(); // Apply schema adapter to a new source - let result_source = apply_schema_adapter(source, &config); + let result_source = source.apply_schema_adapter(&config); // Verify the adapter was applied assert!(result_source.schema_adapter_factory().is_some()); @@ -207,7 +199,7 @@ mod parquet_adapter_tests { .build(); // Apply schema adapter function - should pass through the source unchanged - let result_source = apply_schema_adapter(source, &config); + let result_source = source.apply_schema_adapter(&config); // Verify no adapter was applied assert!(result_source.schema_adapter_factory().is_none()); From 208b1cc996bfc77171ce5eaef50ca3fd296c9d40 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 26 May 2025 11:58:18 +0800 Subject: [PATCH 33/38] remove macro --- .../datasource/physical_plan/arrow_file.rs | 18 +- .../filter_pushdown/util.rs | 18 +- .../core/tests/test_source_adapter_tests.rs | 247 ++++++++++++++++++ datafusion/datasource-avro/src/source.rs | 16 +- datafusion/datasource-csv/src/source.rs | 18 +- datafusion/datasource-json/src/source.rs | 18 +- datafusion/datasource-parquet/src/source.rs | 16 +- datafusion/datasource/src/file.rs | 23 +- datafusion/datasource/src/macros.rs | 129 +-------- datafusion/datasource/src/test_util.rs | 16 +- 10 files changed, 369 insertions(+), 150 deletions(-) create mode 100644 datafusion/core/tests/test_source_adapter_tests.rs diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 6de72aa8ff720..f8ce571f25d03 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use crate::datasource::physical_plan::{FileMeta, FileOpenFuture, FileOpener}; use crate::error::Result; use datafusion_datasource::schema_adapter::SchemaAdapterFactory; -use datafusion_datasource::{as_file_source, impl_schema_adapter_methods}; +use datafusion_datasource::as_file_source; use arrow::buffer::Buffer; use arrow::datatypes::SchemaRef; @@ -99,7 +99,21 @@ impl FileSource for ArrowSource { "arrow" } - impl_schema_adapter_methods!(); + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Result> { + Ok(Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + })) + } + + fn schema_adapter_factory( + &self, + ) -> Option> { + self.schema_adapter_factory.clone() + } } /// The struct arrow that implements `[FileOpener]` trait diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index dc4d77194c082..51ac66a60d980 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -23,7 +23,7 @@ use datafusion_common::{config::ConfigOptions, internal_err, Result, Statistics} use datafusion_datasource::{ file::FileSource, file_meta::FileMeta, file_scan_config::FileScanConfig, file_scan_config::FileScanConfigBuilder, file_stream::FileOpenFuture, - file_stream::FileOpener, impl_schema_adapter_methods, + file_stream::FileOpener, schema_adapter::DefaultSchemaAdapterFactory, schema_adapter::SchemaAdapterFactory, source::DataSourceExec, PartitionedFile, }; @@ -232,7 +232,21 @@ impl FileSource for TestSource { } } - impl_schema_adapter_methods!(); + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Result> { + Ok(Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + })) + } + + fn schema_adapter_factory( + &self, + ) -> Option> { + self.schema_adapter_factory.clone() + } } #[derive(Debug, Clone)] diff --git a/datafusion/core/tests/test_source_adapter_tests.rs b/datafusion/core/tests/test_source_adapter_tests.rs new file mode 100644 index 0000000000000..5e9752600aba6 --- /dev/null +++ b/datafusion/core/tests/test_source_adapter_tests.rs @@ -0,0 +1,247 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion_common::{ColumnStatistics, DataFusionError, Result, Statistics}; +use datafusion_datasource::file::FileSource; +use datafusion_datasource::file_scan_config::FileScanConfig; +use datafusion_datasource::file_stream::FileOpener; +// Removed import of impl_schema_adapter_methods +use datafusion_datasource::schema_adapter::{ + SchemaAdapter, SchemaAdapterFactory, SchemaMapper, +}; +use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; +use object_store::ObjectStore; +use std::fmt::Debug; +use std::sync::Arc; + +// Simple TestSource implementation for testing without dependency on private module +#[derive(Clone, Debug)] +struct TestSource { + #[allow(dead_code)] + has_adapter: bool, + schema_adapter_factory: Option>, +} + +impl TestSource { + fn new(has_adapter: bool) -> Self { + Self { + has_adapter, + schema_adapter_factory: None, + } + } +} + +impl FileSource for TestSource { + fn file_type(&self) -> &str { + "test" + } + + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn create_file_opener( + &self, + _store: Arc, + _conf: &FileScanConfig, + _index: usize, + ) -> Arc { + unimplemented!("Not needed for this test") + } + + fn with_batch_size(&self, _batch_size: usize) -> Arc { + Arc::new(self.clone()) + } + + fn with_schema(&self, _schema: SchemaRef) -> Arc { + Arc::new(self.clone()) + } + + fn with_projection(&self, _projection: &FileScanConfig) -> Arc { + Arc::new(self.clone()) + } + + fn with_statistics(&self, _statistics: Statistics) -> Arc { + Arc::new(self.clone()) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + unimplemented!("Not needed for this test") + } + + fn statistics(&self) -> Result { + Ok(Statistics::default()) + } + + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Result> { + Ok(Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + })) + } + + fn schema_adapter_factory( + &self, + ) -> Option> { + self.schema_adapter_factory.clone() + } +} + +impl DisplayAs for TestSource { + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default + | DisplayFormatType::Verbose + | DisplayFormatType::TreeRender => { + write!(f, "TestSource") + } + } + } +} + +/// A simple schema adapter factory for testing +#[derive(Debug)] +struct TestFilterPushdownAdapterFactory {} + +impl SchemaAdapterFactory for TestFilterPushdownAdapterFactory { + fn create( + &self, + projected_table_schema: SchemaRef, + _table_schema: SchemaRef, + ) -> Box { + Box::new(TestFilterPushdownAdapter { + input_schema: projected_table_schema, + }) + } +} + +/// A simple schema adapter for testing +#[derive(Debug)] +struct TestFilterPushdownAdapter { + input_schema: SchemaRef, +} + +impl SchemaAdapter for TestFilterPushdownAdapter { + fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { + let field = self.input_schema.field(index); + file_schema.fields.find(field.name()).map(|(i, _)| i) + } + + fn map_schema( + &self, + file_schema: &Schema, + ) -> Result<(Arc, Vec)> { + let mut projection = Vec::with_capacity(file_schema.fields().len()); + for (file_idx, file_field) in file_schema.fields().iter().enumerate() { + if self.input_schema.fields().find(file_field.name()).is_some() { + projection.push(file_idx); + } + } + + // Create a schema mapper that modifies column names + #[derive(Debug)] + struct TestSchemaMapping { + #[allow(dead_code)] + input_schema: SchemaRef, + } + + impl SchemaMapper for TestSchemaMapping { + fn map_batch( + &self, + batch: arrow::record_batch::RecordBatch, + ) -> Result { + // For testing, just return the original batch + Ok(batch) + } + + fn map_column_statistics( + &self, + file_col_statistics: &[ColumnStatistics], + ) -> Result> { + // For testing, just return the input statistics + Ok(file_col_statistics.to_vec()) + } + } + + Ok(( + Arc::new(TestSchemaMapping { + input_schema: self.input_schema.clone(), + }), + projection, + )) + } +} + +#[test] +fn test_test_source_schema_adapter_factory() { + // Create a TestSource instance + let schema = Arc::new(Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("value", DataType::Utf8, true), + ])); + + let source = TestSource::new(true); + + // Verify initial state has no adapter + assert!(source.schema_adapter_factory().is_none()); + + // Apply an adapter factory + let factory = Arc::new(TestFilterPushdownAdapterFactory {}); + let source_with_adapter = source.with_schema_adapter_factory(factory); + + // Verify adapter was set + assert!(source_with_adapter.schema_adapter_factory().is_some()); + + // Create an adapter + let adapter_factory = source_with_adapter.schema_adapter_factory().unwrap(); + let adapter = adapter_factory.create(Arc::clone(&schema), Arc::clone(&schema)); + + // Create a file schema to test mapping + let file_schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("value", DataType::Utf8, true), + ]); + + // Test column mapping + let id_index = adapter.map_column_index(0, &file_schema); + assert_eq!(id_index, Some(0)); + + // Test schema mapping + let (_mapper, projection) = adapter.map_schema(&file_schema).unwrap(); + assert_eq!(projection.len(), 2); // Both columns should be included + + // Check file type remains unchanged + assert_eq!(source_with_adapter.file_type(), "test"); +} + +#[test] +fn test_test_source_default() { + // Create a TestSource with default values + let source = TestSource::new(false); + + // Ensure schema_adapter_factory is None by default + assert!(source.schema_adapter_factory().is_none()); +} diff --git a/datafusion/datasource-avro/src/source.rs b/datafusion/datasource-avro/src/source.rs index 2fdf34b3cc56d..82fd00fd60a77 100644 --- a/datafusion/datasource-avro/src/source.rs +++ b/datafusion/datasource-avro/src/source.rs @@ -28,7 +28,7 @@ use datafusion_common::Statistics; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::file_stream::FileOpener; -use datafusion_datasource::impl_schema_adapter_methods; +// Removed import of impl_schema_adapter_methods use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; @@ -127,7 +127,19 @@ impl FileSource for AvroSource { Ok(None) } - impl_schema_adapter_methods!(); + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Result> { + Ok(Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + })) + } + + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() + } } mod private { diff --git a/datafusion/datasource-csv/src/source.rs b/datafusion/datasource-csv/src/source.rs index d45080dc20776..b9aff37c14714 100644 --- a/datafusion/datasource-csv/src/source.rs +++ b/datafusion/datasource-csv/src/source.rs @@ -29,7 +29,7 @@ use datafusion_datasource::file_compression_type::FileCompressionType; use datafusion_datasource::file_meta::FileMeta; use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; use datafusion_datasource::{ - as_file_source, calculate_range, impl_schema_adapter_methods, FileRange, + as_file_source, calculate_range, FileRange, ListingTableUrl, RangeCalculation, }; @@ -284,7 +284,21 @@ impl FileSource for CsvSource { } } - impl_schema_adapter_methods!(); + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Result> { + Ok(Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + })) + } + + fn schema_adapter_factory( + &self, + ) -> Option> { + self.schema_adapter_factory.clone() + } } impl FileOpener for CsvOpener { diff --git a/datafusion/datasource-json/src/source.rs b/datafusion/datasource-json/src/source.rs index 187876522e48e..fa0b5061729b1 100644 --- a/datafusion/datasource-json/src/source.rs +++ b/datafusion/datasource-json/src/source.rs @@ -32,7 +32,7 @@ use datafusion_datasource::file_meta::FileMeta; use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use datafusion_datasource::{ - as_file_source, calculate_range, impl_schema_adapter_methods, ListingTableUrl, + as_file_source, calculate_range, ListingTableUrl, RangeCalculation, }; use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; @@ -151,7 +151,21 @@ impl FileSource for JsonSource { fn file_type(&self) -> &str { "json" } - impl_schema_adapter_methods!(); + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Result> { + Ok(Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + })) + } + + fn schema_adapter_factory( + &self, + ) -> Option> { + self.schema_adapter_factory.clone() + } } impl FileOpener for JsonOpener { diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 30b774d08f1d6..1f55f91e109d3 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -29,7 +29,7 @@ use crate::ParquetFileReaderFactory; use datafusion_common::config::ConfigOptions; use datafusion_datasource::as_file_source; use datafusion_datasource::file_stream::FileOpener; -use datafusion_datasource::impl_schema_adapter_methods; +// Removed import of impl_schema_adapter_methods use datafusion_datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapterFactory, }; @@ -664,5 +664,17 @@ impl FileSource for ParquetSource { ); Ok(FilterPushdownPropagation::with_filters(filters).with_updated_node(source)) } - impl_schema_adapter_methods!(); + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Result> { + Ok(Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + })) + } + + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() + } } diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index d0557e9f0872c..d67ce3bd731a5 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -28,7 +28,7 @@ use crate::file_stream::FileOpener; use crate::schema_adapter::SchemaAdapterFactory; use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; -use datafusion_common::{Result, Statistics}; +use datafusion_common::{DataFusionError, Result, Statistics}; use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; use datafusion_physical_plan::filter_pushdown::FilterPushdownPropagation; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; @@ -126,19 +126,22 @@ pub trait FileSource: Send + Sync { /// Set optional schema adapter factory. /// /// [`SchemaAdapterFactory`] allows user to specify how fields from the - /// file get mapped to that of the table schema. The default implementation - /// returns the original source. + /// file get mapped to that of the table schema. /// - /// Note: You can implement this method and `schema_adapter_factory` - /// automatically using the [`crate::impl_schema_adapter_methods`] macro. + /// Default implementation returns NotImplemented error fn with_schema_adapter_factory( &self, - factory: Arc, - ) -> Arc; + _factory: Arc, + ) -> Result> { + Err(DataFusionError::NotImplemented( + "with_schema_adapter_factory not implemented for this FileSource".to_string(), + )) + } /// Returns the current schema adapter factory if set /// - /// Note: You can implement this method and `with_schema_adapter_factory` - /// automatically using the [`crate::impl_schema_adapter_methods`] macro. - fn schema_adapter_factory(&self) -> Option>; + /// Default implementation returns None + fn schema_adapter_factory(&self) -> Option> { + None + } } diff --git a/datafusion/datasource/src/macros.rs b/datafusion/datasource/src/macros.rs index c7a4058f2310e..9a5f37d913a5f 100644 --- a/datafusion/datasource/src/macros.rs +++ b/datafusion/datasource/src/macros.rs @@ -17,129 +17,6 @@ //! Macros for the datafusion-datasource crate -/// Helper macro to generate schema adapter methods for FileSource implementations -/// -/// Place this inside *any* `impl FileSource for YourType { … }` to -/// avoid copy-pasting `with_schema_adapter_factory` and -/// `schema_adapter_factory`. -/// -/// # Availability -/// -/// This macro is exported at the crate root level via `#[macro_export]`, so it can be -/// imported directly from the crate: -/// -/// ```rust,no_run -/// use datafusion_datasource::impl_schema_adapter_methods; -/// ``` -/// -/// # Note on path resolution -/// When this macro is used: -/// - `$crate` expands to `datafusion_datasource` (the crate root) -/// - `$crate::file::FileSource` refers to the FileSource trait from this crate -/// - `$crate::schema_adapter::SchemaAdapterFactory` refers to the SchemaAdapterFactory trait -/// -/// # Example Usage -/// -/// ```rust,no_run -/// use std::sync::Arc; -/// use std::any::Any; -/// use std::fmt::{Formatter, Display, self}; -/// use arrow::datatypes::SchemaRef; -/// use datafusion_common::{Result, Statistics}; -/// use object_store::ObjectStore; -/// use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; -/// use datafusion_physical_plan::DisplayFormatType; -/// use datafusion_physical_expr_common::sort_expr::LexOrdering; -/// use datafusion_datasource::file::FileSource; -/// use datafusion_datasource::file_stream::FileOpener; -/// use datafusion_datasource::file_scan_config::FileScanConfig; -/// use datafusion_datasource::impl_schema_adapter_methods; -/// use datafusion_datasource::schema_adapter::SchemaAdapterFactory; -/// -/// #[derive(Clone)] -/// struct MyFileSource { -/// schema: SchemaRef, -/// batch_size: usize, -/// statistics: Statistics, -/// projection: Option>, -/// schema_adapter_factory: Option>, -/// metrics: ExecutionPlanMetricsSet, -/// } -/// -/// impl FileSource for MyFileSource { -/// fn create_file_opener( -/// &self, -/// object_store: Arc, -/// base_config: &FileScanConfig, -/// partition: usize, -/// ) -> Arc { -/// // Implementation here -/// unimplemented!() -/// } -/// -/// fn as_any(&self) -> &dyn Any { -/// self -/// } -/// -/// fn with_batch_size(&self, batch_size: usize) -> Arc { -/// let mut new_source = self.clone(); -/// new_source.batch_size = batch_size; -/// Arc::new(new_source) -/// } -/// -/// fn with_schema(&self, schema: SchemaRef) -> Arc { -/// let mut new_source = self.clone(); -/// new_source.schema = schema; -/// Arc::new(new_source) -/// } -/// -/// fn with_projection(&self, config: &FileScanConfig) -> Arc { -/// let mut new_source = self.clone(); -/// new_source.projection = config.file_column_projection_indices(); -/// Arc::new(new_source) -/// } -/// -/// fn with_statistics(&self, statistics: Statistics) -> Arc { -/// let mut new_source = self.clone(); -/// new_source.statistics = statistics; -/// Arc::new(new_source) -/// } -/// -/// fn metrics(&self) -> &ExecutionPlanMetricsSet { -/// &self.metrics -/// } -/// -/// fn statistics(&self) -> Result { -/// Ok(self.statistics.clone()) -/// } -/// -/// fn file_type(&self) -> &str { -/// "my_file_type" -/// } -/// -/// // Use the macro to implement schema adapter methods -/// impl_schema_adapter_methods!(); -/// } -/// ``` -#[macro_export(local_inner_macros)] -macro_rules! impl_schema_adapter_methods { - () => { - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: std::sync::Arc< - dyn $crate::schema_adapter::SchemaAdapterFactory, - >, - ) -> std::sync::Arc { - std::sync::Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - }) - } - - fn schema_adapter_factory( - &self, - ) -> Option> { - self.schema_adapter_factory.clone() - } - }; -} +//! Macros for the datafusion-datasource crate +// The `impl_schema_adapter_methods` macro has been removed. +// The `FileSource` trait now provides default implementations for schema adapter methods. diff --git a/datafusion/datasource/src/test_util.rs b/datafusion/datasource/src/test_util.rs index aac61c7812a41..e4a5114aa073e 100644 --- a/datafusion/datasource/src/test_util.rs +++ b/datafusion/datasource/src/test_util.rs @@ -17,7 +17,7 @@ use crate::{ file::FileSource, file_scan_config::FileScanConfig, file_stream::FileOpener, - impl_schema_adapter_methods, schema_adapter::SchemaAdapterFactory, + schema_adapter::SchemaAdapterFactory, }; use std::sync::Arc; @@ -84,7 +84,19 @@ impl FileSource for MockSource { "mock" } - impl_schema_adapter_methods!(); + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc, + ) -> Result> { + Ok(Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + })) + } + + fn schema_adapter_factory(&self) -> Option> { + self.schema_adapter_factory.clone() + } } /// Create a column expression From fd6dd78f49e0195d01885b3fde988b04402f2c94 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 26 May 2025 12:03:51 +0800 Subject: [PATCH 34/38] Revert "remove macro" This reverts commit 208b1cc996bfc77171ce5eaef50ca3fd296c9d40. --- .../datasource/physical_plan/arrow_file.rs | 18 +- .../filter_pushdown/util.rs | 18 +- .../core/tests/test_source_adapter_tests.rs | 247 ------------------ datafusion/datasource-avro/src/source.rs | 16 +- datafusion/datasource-csv/src/source.rs | 18 +- datafusion/datasource-json/src/source.rs | 18 +- datafusion/datasource-parquet/src/source.rs | 16 +- datafusion/datasource/src/file.rs | 23 +- datafusion/datasource/src/macros.rs | 129 ++++++++- datafusion/datasource/src/test_util.rs | 16 +- 10 files changed, 150 insertions(+), 369 deletions(-) delete mode 100644 datafusion/core/tests/test_source_adapter_tests.rs diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index f8ce571f25d03..6de72aa8ff720 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use crate::datasource::physical_plan::{FileMeta, FileOpenFuture, FileOpener}; use crate::error::Result; use datafusion_datasource::schema_adapter::SchemaAdapterFactory; -use datafusion_datasource::as_file_source; +use datafusion_datasource::{as_file_source, impl_schema_adapter_methods}; use arrow::buffer::Buffer; use arrow::datatypes::SchemaRef; @@ -99,21 +99,7 @@ impl FileSource for ArrowSource { "arrow" } - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Result> { - Ok(Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - })) - } - - fn schema_adapter_factory( - &self, - ) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } /// The struct arrow that implements `[FileOpener]` trait diff --git a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs index 51ac66a60d980..dc4d77194c082 100644 --- a/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs +++ b/datafusion/core/tests/physical_optimizer/filter_pushdown/util.rs @@ -23,7 +23,7 @@ use datafusion_common::{config::ConfigOptions, internal_err, Result, Statistics} use datafusion_datasource::{ file::FileSource, file_meta::FileMeta, file_scan_config::FileScanConfig, file_scan_config::FileScanConfigBuilder, file_stream::FileOpenFuture, - file_stream::FileOpener, + file_stream::FileOpener, impl_schema_adapter_methods, schema_adapter::DefaultSchemaAdapterFactory, schema_adapter::SchemaAdapterFactory, source::DataSourceExec, PartitionedFile, }; @@ -232,21 +232,7 @@ impl FileSource for TestSource { } } - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Result> { - Ok(Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - })) - } - - fn schema_adapter_factory( - &self, - ) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } #[derive(Debug, Clone)] diff --git a/datafusion/core/tests/test_source_adapter_tests.rs b/datafusion/core/tests/test_source_adapter_tests.rs deleted file mode 100644 index 5e9752600aba6..0000000000000 --- a/datafusion/core/tests/test_source_adapter_tests.rs +++ /dev/null @@ -1,247 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::{ColumnStatistics, DataFusionError, Result, Statistics}; -use datafusion_datasource::file::FileSource; -use datafusion_datasource::file_scan_config::FileScanConfig; -use datafusion_datasource::file_stream::FileOpener; -// Removed import of impl_schema_adapter_methods -use datafusion_datasource::schema_adapter::{ - SchemaAdapter, SchemaAdapterFactory, SchemaMapper, -}; -use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; -use datafusion_physical_plan::{DisplayAs, DisplayFormatType}; -use object_store::ObjectStore; -use std::fmt::Debug; -use std::sync::Arc; - -// Simple TestSource implementation for testing without dependency on private module -#[derive(Clone, Debug)] -struct TestSource { - #[allow(dead_code)] - has_adapter: bool, - schema_adapter_factory: Option>, -} - -impl TestSource { - fn new(has_adapter: bool) -> Self { - Self { - has_adapter, - schema_adapter_factory: None, - } - } -} - -impl FileSource for TestSource { - fn file_type(&self) -> &str { - "test" - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn create_file_opener( - &self, - _store: Arc, - _conf: &FileScanConfig, - _index: usize, - ) -> Arc { - unimplemented!("Not needed for this test") - } - - fn with_batch_size(&self, _batch_size: usize) -> Arc { - Arc::new(self.clone()) - } - - fn with_schema(&self, _schema: SchemaRef) -> Arc { - Arc::new(self.clone()) - } - - fn with_projection(&self, _projection: &FileScanConfig) -> Arc { - Arc::new(self.clone()) - } - - fn with_statistics(&self, _statistics: Statistics) -> Arc { - Arc::new(self.clone()) - } - - fn metrics(&self) -> &ExecutionPlanMetricsSet { - unimplemented!("Not needed for this test") - } - - fn statistics(&self) -> Result { - Ok(Statistics::default()) - } - - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Result> { - Ok(Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - })) - } - - fn schema_adapter_factory( - &self, - ) -> Option> { - self.schema_adapter_factory.clone() - } -} - -impl DisplayAs for TestSource { - fn fmt_as( - &self, - t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - match t { - DisplayFormatType::Default - | DisplayFormatType::Verbose - | DisplayFormatType::TreeRender => { - write!(f, "TestSource") - } - } - } -} - -/// A simple schema adapter factory for testing -#[derive(Debug)] -struct TestFilterPushdownAdapterFactory {} - -impl SchemaAdapterFactory for TestFilterPushdownAdapterFactory { - fn create( - &self, - projected_table_schema: SchemaRef, - _table_schema: SchemaRef, - ) -> Box { - Box::new(TestFilterPushdownAdapter { - input_schema: projected_table_schema, - }) - } -} - -/// A simple schema adapter for testing -#[derive(Debug)] -struct TestFilterPushdownAdapter { - input_schema: SchemaRef, -} - -impl SchemaAdapter for TestFilterPushdownAdapter { - fn map_column_index(&self, index: usize, file_schema: &Schema) -> Option { - let field = self.input_schema.field(index); - file_schema.fields.find(field.name()).map(|(i, _)| i) - } - - fn map_schema( - &self, - file_schema: &Schema, - ) -> Result<(Arc, Vec)> { - let mut projection = Vec::with_capacity(file_schema.fields().len()); - for (file_idx, file_field) in file_schema.fields().iter().enumerate() { - if self.input_schema.fields().find(file_field.name()).is_some() { - projection.push(file_idx); - } - } - - // Create a schema mapper that modifies column names - #[derive(Debug)] - struct TestSchemaMapping { - #[allow(dead_code)] - input_schema: SchemaRef, - } - - impl SchemaMapper for TestSchemaMapping { - fn map_batch( - &self, - batch: arrow::record_batch::RecordBatch, - ) -> Result { - // For testing, just return the original batch - Ok(batch) - } - - fn map_column_statistics( - &self, - file_col_statistics: &[ColumnStatistics], - ) -> Result> { - // For testing, just return the input statistics - Ok(file_col_statistics.to_vec()) - } - } - - Ok(( - Arc::new(TestSchemaMapping { - input_schema: self.input_schema.clone(), - }), - projection, - )) - } -} - -#[test] -fn test_test_source_schema_adapter_factory() { - // Create a TestSource instance - let schema = Arc::new(Schema::new(vec![ - Field::new("id", DataType::Int32, false), - Field::new("value", DataType::Utf8, true), - ])); - - let source = TestSource::new(true); - - // Verify initial state has no adapter - assert!(source.schema_adapter_factory().is_none()); - - // Apply an adapter factory - let factory = Arc::new(TestFilterPushdownAdapterFactory {}); - let source_with_adapter = source.with_schema_adapter_factory(factory); - - // Verify adapter was set - assert!(source_with_adapter.schema_adapter_factory().is_some()); - - // Create an adapter - let adapter_factory = source_with_adapter.schema_adapter_factory().unwrap(); - let adapter = adapter_factory.create(Arc::clone(&schema), Arc::clone(&schema)); - - // Create a file schema to test mapping - let file_schema = Schema::new(vec![ - Field::new("id", DataType::Int32, false), - Field::new("value", DataType::Utf8, true), - ]); - - // Test column mapping - let id_index = adapter.map_column_index(0, &file_schema); - assert_eq!(id_index, Some(0)); - - // Test schema mapping - let (_mapper, projection) = adapter.map_schema(&file_schema).unwrap(); - assert_eq!(projection.len(), 2); // Both columns should be included - - // Check file type remains unchanged - assert_eq!(source_with_adapter.file_type(), "test"); -} - -#[test] -fn test_test_source_default() { - // Create a TestSource with default values - let source = TestSource::new(false); - - // Ensure schema_adapter_factory is None by default - assert!(source.schema_adapter_factory().is_none()); -} diff --git a/datafusion/datasource-avro/src/source.rs b/datafusion/datasource-avro/src/source.rs index 82fd00fd60a77..2fdf34b3cc56d 100644 --- a/datafusion/datasource-avro/src/source.rs +++ b/datafusion/datasource-avro/src/source.rs @@ -28,7 +28,7 @@ use datafusion_common::Statistics; use datafusion_datasource::file::FileSource; use datafusion_datasource::file_scan_config::FileScanConfig; use datafusion_datasource::file_stream::FileOpener; -// Removed import of impl_schema_adapter_methods +use datafusion_datasource::impl_schema_adapter_methods; use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; @@ -127,19 +127,7 @@ impl FileSource for AvroSource { Ok(None) } - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Result> { - Ok(Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - })) - } - - fn schema_adapter_factory(&self) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } mod private { diff --git a/datafusion/datasource-csv/src/source.rs b/datafusion/datasource-csv/src/source.rs index b9aff37c14714..d45080dc20776 100644 --- a/datafusion/datasource-csv/src/source.rs +++ b/datafusion/datasource-csv/src/source.rs @@ -29,7 +29,7 @@ use datafusion_datasource::file_compression_type::FileCompressionType; use datafusion_datasource::file_meta::FileMeta; use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; use datafusion_datasource::{ - as_file_source, calculate_range, FileRange, + as_file_source, calculate_range, impl_schema_adapter_methods, FileRange, ListingTableUrl, RangeCalculation, }; @@ -284,21 +284,7 @@ impl FileSource for CsvSource { } } - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Result> { - Ok(Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - })) - } - - fn schema_adapter_factory( - &self, - ) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } impl FileOpener for CsvOpener { diff --git a/datafusion/datasource-json/src/source.rs b/datafusion/datasource-json/src/source.rs index fa0b5061729b1..187876522e48e 100644 --- a/datafusion/datasource-json/src/source.rs +++ b/datafusion/datasource-json/src/source.rs @@ -32,7 +32,7 @@ use datafusion_datasource::file_meta::FileMeta; use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener}; use datafusion_datasource::schema_adapter::SchemaAdapterFactory; use datafusion_datasource::{ - as_file_source, calculate_range, ListingTableUrl, + as_file_source, calculate_range, impl_schema_adapter_methods, ListingTableUrl, RangeCalculation, }; use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; @@ -151,21 +151,7 @@ impl FileSource for JsonSource { fn file_type(&self) -> &str { "json" } - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Result> { - Ok(Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - })) - } - - fn schema_adapter_factory( - &self, - ) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } impl FileOpener for JsonOpener { diff --git a/datafusion/datasource-parquet/src/source.rs b/datafusion/datasource-parquet/src/source.rs index 1f55f91e109d3..30b774d08f1d6 100644 --- a/datafusion/datasource-parquet/src/source.rs +++ b/datafusion/datasource-parquet/src/source.rs @@ -29,7 +29,7 @@ use crate::ParquetFileReaderFactory; use datafusion_common::config::ConfigOptions; use datafusion_datasource::as_file_source; use datafusion_datasource::file_stream::FileOpener; -// Removed import of impl_schema_adapter_methods +use datafusion_datasource::impl_schema_adapter_methods; use datafusion_datasource::schema_adapter::{ DefaultSchemaAdapterFactory, SchemaAdapterFactory, }; @@ -664,17 +664,5 @@ impl FileSource for ParquetSource { ); Ok(FilterPushdownPropagation::with_filters(filters).with_updated_node(source)) } - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Result> { - Ok(Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - })) - } - - fn schema_adapter_factory(&self) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index d67ce3bd731a5..d0557e9f0872c 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -28,7 +28,7 @@ use crate::file_stream::FileOpener; use crate::schema_adapter::SchemaAdapterFactory; use arrow::datatypes::SchemaRef; use datafusion_common::config::ConfigOptions; -use datafusion_common::{DataFusionError, Result, Statistics}; +use datafusion_common::{Result, Statistics}; use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; use datafusion_physical_plan::filter_pushdown::FilterPushdownPropagation; use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; @@ -126,22 +126,19 @@ pub trait FileSource: Send + Sync { /// Set optional schema adapter factory. /// /// [`SchemaAdapterFactory`] allows user to specify how fields from the - /// file get mapped to that of the table schema. + /// file get mapped to that of the table schema. The default implementation + /// returns the original source. /// - /// Default implementation returns NotImplemented error + /// Note: You can implement this method and `schema_adapter_factory` + /// automatically using the [`crate::impl_schema_adapter_methods`] macro. fn with_schema_adapter_factory( &self, - _factory: Arc, - ) -> Result> { - Err(DataFusionError::NotImplemented( - "with_schema_adapter_factory not implemented for this FileSource".to_string(), - )) - } + factory: Arc, + ) -> Arc; /// Returns the current schema adapter factory if set /// - /// Default implementation returns None - fn schema_adapter_factory(&self) -> Option> { - None - } + /// Note: You can implement this method and `with_schema_adapter_factory` + /// automatically using the [`crate::impl_schema_adapter_methods`] macro. + fn schema_adapter_factory(&self) -> Option>; } diff --git a/datafusion/datasource/src/macros.rs b/datafusion/datasource/src/macros.rs index 9a5f37d913a5f..c7a4058f2310e 100644 --- a/datafusion/datasource/src/macros.rs +++ b/datafusion/datasource/src/macros.rs @@ -17,6 +17,129 @@ //! Macros for the datafusion-datasource crate -//! Macros for the datafusion-datasource crate -// The `impl_schema_adapter_methods` macro has been removed. -// The `FileSource` trait now provides default implementations for schema adapter methods. +/// Helper macro to generate schema adapter methods for FileSource implementations +/// +/// Place this inside *any* `impl FileSource for YourType { … }` to +/// avoid copy-pasting `with_schema_adapter_factory` and +/// `schema_adapter_factory`. +/// +/// # Availability +/// +/// This macro is exported at the crate root level via `#[macro_export]`, so it can be +/// imported directly from the crate: +/// +/// ```rust,no_run +/// use datafusion_datasource::impl_schema_adapter_methods; +/// ``` +/// +/// # Note on path resolution +/// When this macro is used: +/// - `$crate` expands to `datafusion_datasource` (the crate root) +/// - `$crate::file::FileSource` refers to the FileSource trait from this crate +/// - `$crate::schema_adapter::SchemaAdapterFactory` refers to the SchemaAdapterFactory trait +/// +/// # Example Usage +/// +/// ```rust,no_run +/// use std::sync::Arc; +/// use std::any::Any; +/// use std::fmt::{Formatter, Display, self}; +/// use arrow::datatypes::SchemaRef; +/// use datafusion_common::{Result, Statistics}; +/// use object_store::ObjectStore; +/// use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +/// use datafusion_physical_plan::DisplayFormatType; +/// use datafusion_physical_expr_common::sort_expr::LexOrdering; +/// use datafusion_datasource::file::FileSource; +/// use datafusion_datasource::file_stream::FileOpener; +/// use datafusion_datasource::file_scan_config::FileScanConfig; +/// use datafusion_datasource::impl_schema_adapter_methods; +/// use datafusion_datasource::schema_adapter::SchemaAdapterFactory; +/// +/// #[derive(Clone)] +/// struct MyFileSource { +/// schema: SchemaRef, +/// batch_size: usize, +/// statistics: Statistics, +/// projection: Option>, +/// schema_adapter_factory: Option>, +/// metrics: ExecutionPlanMetricsSet, +/// } +/// +/// impl FileSource for MyFileSource { +/// fn create_file_opener( +/// &self, +/// object_store: Arc, +/// base_config: &FileScanConfig, +/// partition: usize, +/// ) -> Arc { +/// // Implementation here +/// unimplemented!() +/// } +/// +/// fn as_any(&self) -> &dyn Any { +/// self +/// } +/// +/// fn with_batch_size(&self, batch_size: usize) -> Arc { +/// let mut new_source = self.clone(); +/// new_source.batch_size = batch_size; +/// Arc::new(new_source) +/// } +/// +/// fn with_schema(&self, schema: SchemaRef) -> Arc { +/// let mut new_source = self.clone(); +/// new_source.schema = schema; +/// Arc::new(new_source) +/// } +/// +/// fn with_projection(&self, config: &FileScanConfig) -> Arc { +/// let mut new_source = self.clone(); +/// new_source.projection = config.file_column_projection_indices(); +/// Arc::new(new_source) +/// } +/// +/// fn with_statistics(&self, statistics: Statistics) -> Arc { +/// let mut new_source = self.clone(); +/// new_source.statistics = statistics; +/// Arc::new(new_source) +/// } +/// +/// fn metrics(&self) -> &ExecutionPlanMetricsSet { +/// &self.metrics +/// } +/// +/// fn statistics(&self) -> Result { +/// Ok(self.statistics.clone()) +/// } +/// +/// fn file_type(&self) -> &str { +/// "my_file_type" +/// } +/// +/// // Use the macro to implement schema adapter methods +/// impl_schema_adapter_methods!(); +/// } +/// ``` +#[macro_export(local_inner_macros)] +macro_rules! impl_schema_adapter_methods { + () => { + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: std::sync::Arc< + dyn $crate::schema_adapter::SchemaAdapterFactory, + >, + ) -> std::sync::Arc { + std::sync::Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + ..self.clone() + }) + } + + fn schema_adapter_factory( + &self, + ) -> Option> { + self.schema_adapter_factory.clone() + } + }; +} diff --git a/datafusion/datasource/src/test_util.rs b/datafusion/datasource/src/test_util.rs index e4a5114aa073e..aac61c7812a41 100644 --- a/datafusion/datasource/src/test_util.rs +++ b/datafusion/datasource/src/test_util.rs @@ -17,7 +17,7 @@ use crate::{ file::FileSource, file_scan_config::FileScanConfig, file_stream::FileOpener, - schema_adapter::SchemaAdapterFactory, + impl_schema_adapter_methods, schema_adapter::SchemaAdapterFactory, }; use std::sync::Arc; @@ -84,19 +84,7 @@ impl FileSource for MockSource { "mock" } - fn with_schema_adapter_factory( - &self, - schema_adapter_factory: Arc, - ) -> Result> { - Ok(Arc::new(Self { - schema_adapter_factory: Some(schema_adapter_factory), - ..self.clone() - })) - } - - fn schema_adapter_factory(&self) -> Option> { - self.schema_adapter_factory.clone() - } + impl_schema_adapter_methods!(); } /// Create a column expression From ee07b69fbcc0a41bd2a859f5f0fa328b35d4e69d Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 26 May 2025 12:07:28 +0800 Subject: [PATCH 35/38] FileSource - provide default implementations for schema_adapter_factory methods --- datafusion/datasource/src/file.rs | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index d0557e9f0872c..11524edf1d1da 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -134,11 +134,19 @@ pub trait FileSource: Send + Sync { fn with_schema_adapter_factory( &self, factory: Arc, - ) -> Arc; + ) -> Arc + where + Self: Sized + Clone, + { + // clone the _concrete_ self, then wrap it + Arc::new(self.clone()) as Arc + } /// Returns the current schema adapter factory if set /// /// Note: You can implement this method and `with_schema_adapter_factory` /// automatically using the [`crate::impl_schema_adapter_methods`] macro. - fn schema_adapter_factory(&self) -> Option>; + fn schema_adapter_factory(&self) -> Option> { + None + } } From 16eb25d47f3625081750603c7af4c89f5de3e025 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 26 May 2025 12:16:47 +0800 Subject: [PATCH 36/38] Revert "FileSource - provide default implementations for schema_adapter_factory methods" This reverts commit ee07b69fbcc0a41bd2a859f5f0fa328b35d4e69d. --- datafusion/datasource/src/file.rs | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/datafusion/datasource/src/file.rs b/datafusion/datasource/src/file.rs index 11524edf1d1da..d0557e9f0872c 100644 --- a/datafusion/datasource/src/file.rs +++ b/datafusion/datasource/src/file.rs @@ -134,19 +134,11 @@ pub trait FileSource: Send + Sync { fn with_schema_adapter_factory( &self, factory: Arc, - ) -> Arc - where - Self: Sized + Clone, - { - // clone the _concrete_ self, then wrap it - Arc::new(self.clone()) as Arc - } + ) -> Arc; /// Returns the current schema adapter factory if set /// /// Note: You can implement this method and `with_schema_adapter_factory` /// automatically using the [`crate::impl_schema_adapter_methods`] macro. - fn schema_adapter_factory(&self) -> Option> { - None - } + fn schema_adapter_factory(&self) -> Option>; } From f890e8d5e6986ae3cf28fa563bbe6491c03176f1 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 26 May 2025 13:23:46 +0800 Subject: [PATCH 37/38] Remove unused import of SchemaAdapterFactory from file_format.rs --- datafusion/datasource-parquet/src/file_format.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/datasource-parquet/src/file_format.rs b/datafusion/datasource-parquet/src/file_format.rs index af9e6544b7142..851e33644381d 100644 --- a/datafusion/datasource-parquet/src/file_format.rs +++ b/datafusion/datasource-parquet/src/file_format.rs @@ -50,7 +50,6 @@ 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::schema_adapter::SchemaAdapterFactory; use datafusion_datasource::sink::{DataSink, DataSinkExec}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; From 999e0cd842a0d194ca07a92b529915fb209a44a4 Mon Sep 17 00:00:00 2001 From: Siew Kam Onn Date: Mon, 26 May 2025 15:28:15 +0800 Subject: [PATCH 38/38] Refactor imports in apply_schema_adapter_tests.rs for improved readability --- .../tests/apply_schema_adapter_tests.rs | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs index edf828e71c272..89406fb742dce 100644 --- a/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs +++ b/datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs @@ -16,20 +16,19 @@ // under the License. mod parquet_adapter_tests { - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use arrow::record_batch::RecordBatch; - use datafusion_common::{ColumnStatistics, DataFusionError, Result}; - use datafusion_datasource::file::FileSource; - use datafusion_datasource::file_scan_config::{ - FileScanConfig, FileScanConfigBuilder, + use arrow::{ + datatypes::{DataType, Field, Schema, SchemaRef}, + record_batch::RecordBatch, }; - use datafusion_datasource::schema_adapter::{ - SchemaAdapter, SchemaAdapterFactory, SchemaMapper, + use datafusion_common::{ColumnStatistics, DataFusionError, Result}; + use datafusion_datasource::{ + file::FileSource, + file_scan_config::FileScanConfigBuilder, + schema_adapter::{SchemaAdapter, SchemaAdapterFactory, SchemaMapper}, }; use datafusion_datasource_parquet::source::ParquetSource; use datafusion_execution::object_store::ObjectStoreUrl; - use std::fmt::Debug; - use std::sync::Arc; + use std::{fmt::Debug, sync::Arc}; /// A test schema adapter factory that adds prefix to column names #[derive(Debug)]