alamb commented on code in PR #16148: URL: https://github.com/apache/datafusion/pull/16148#discussion_r2105358307
########## 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<Arc<dyn SchemaAdapterFactory>>, +} + +impl TestSource { + fn new() -> Self { + Self { + schema_adapter_factory: None, + } + } +} + +impl FileSource for TestSource { + fn file_type(&self) -> &str { + "test" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn create_file_opener( + &self, + _store: Arc<dyn ObjectStore>, + _conf: &FileScanConfig, + _index: usize, + ) -> Arc<dyn FileOpener> { + unimplemented!("Not needed for this test") + } + + fn with_batch_size(&self, _batch_size: usize) -> Arc<dyn FileSource> { + Arc::new(self.clone()) + } + + fn with_schema(&self, _schema: SchemaRef) -> Arc<dyn FileSource> { + Arc::new(self.clone()) + } + + fn with_projection(&self, _projection: &FileScanConfig) -> Arc<dyn FileSource> { + Arc::new(self.clone()) + } + + fn with_statistics(&self, _statistics: Statistics) -> Arc<dyn FileSource> { + Arc::new(self.clone()) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + unimplemented!("Not needed for this test") + } + + fn statistics(&self) -> Result<Statistics, DataFusionError> { + Ok(Statistics::default()) + } + + fn with_schema_adapter_factory( + &self, + schema_adapter_factory: Arc<dyn SchemaAdapterFactory>, + ) -> Arc<dyn FileSource> { + Arc::new(Self { + schema_adapter_factory: Some(schema_adapter_factory), + }) + } + + fn schema_adapter_factory(&self) -> Option<Arc<dyn SchemaAdapterFactory>> { + self.schema_adapter_factory.clone() + } +} + +/// A test schema adapter factory +#[derive(Debug)] +struct TestSchemaAdapterFactory {} + +impl SchemaAdapterFactory for TestSchemaAdapterFactory { + fn create( + &self, + projected_table_schema: SchemaRef, + _table_schema: SchemaRef, + ) -> Box<dyn SchemaAdapter> { + 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<usize> { + 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<dyn SchemaMapper>, Vec<usize>)> { + 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<RecordBatch> { + // For testing, just return the original batch + Ok(batch) + } + + fn map_column_statistics( + &self, + stats: &[ColumnStatistics], + ) -> Result<Vec<ColumnStatistics>> { + // 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 Review Comment: What is this test covering? I don't understand what additional coverage it is adding ########## 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 Review Comment: this is very cool ########## datafusion/datasource-parquet/tests/apply_schema_adapter_tests.rs: ########## @@ -0,0 +1,224 @@ +// 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. + +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 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 + #[derive(Debug)] + struct PrefixAdapterFactory { + prefix: String, + } + + impl SchemaAdapterFactory for PrefixAdapterFactory { + fn create( + &self, + projected_table_schema: SchemaRef, + _table_schema: SchemaRef, + ) -> Box<dyn SchemaAdapter> { + Box::new(PrefixAdapter { + input_schema: projected_table_schema, + 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 map_column_index(&self, index: usize, file_schema: &Schema) -> Option<usize> { + 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<dyn SchemaMapper>, Vec<usize>)> { + 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<RecordBatch> { + // Create a new schema with prefixed field names + let prefixed_fields: Vec<Field> = 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, + ) + .map_err(|e| DataFusionError::ArrowError(e, None)) + } + + fn map_column_statistics( + &self, + stats: &[ColumnStatistics], + ) -> Result<Vec<ColumnStatistics>> { + // For testing, just return the input statistics + Ok(stats.to_vec()) + } + } + + Ok(( + Arc::new(PrefixSchemaMapping { + prefix: self.prefix.clone(), + }), + projection, + )) + } + } + + // Implementation of apply_schema_adapter for testing purposes + // This mimics the private function in the datafusion-parquet crate + fn apply_schema_adapter( Review Comment: If we left the method on ParquetSource it wouldn't have to be replicated 🤔 ########## datafusion/core/tests/test_source_adapter_tests.rs: ########## @@ -0,0 +1,233 @@ +// 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<Arc<dyn SchemaAdapterFactory>>, +} + +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<dyn ObjectStore>, + _conf: &FileScanConfig, + _index: usize, + ) -> Arc<dyn FileOpener> { + unimplemented!("Not needed for this test") + } + + fn with_batch_size(&self, _batch_size: usize) -> Arc<dyn FileSource> { + Arc::new(self.clone()) + } + + fn with_schema(&self, _schema: SchemaRef) -> Arc<dyn FileSource> { + Arc::new(self.clone()) + } + + fn with_projection(&self, _projection: &FileScanConfig) -> Arc<dyn FileSource> { + Arc::new(self.clone()) + } + + fn with_statistics(&self, _statistics: Statistics) -> Arc<dyn FileSource> { + Arc::new(self.clone()) + } + + fn metrics(&self) -> &ExecutionPlanMetricsSet { + unimplemented!("Not needed for this test") + } + + fn statistics(&self) -> Result<Statistics, DataFusionError> { + 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<dyn SchemaAdapter> { + 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<usize> { + 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<dyn SchemaMapper>, Vec<usize>)> { + 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<arrow::record_batch::RecordBatch> { + // For testing, just return the original batch + Ok(batch) + } + + fn map_column_statistics( + &self, + file_col_statistics: &[ColumnStatistics], + ) -> Result<Vec<ColumnStatistics>> { + // 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() { Review Comment: same comment here -- I am not sure what extra coverage this is adding and it adds a new binary ########## 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<Box<dyn SchemaAdapter>> { + 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<RecordBatch> { + // 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<()> { Review Comment: it is not entirely clear to me what this test is verifying ########## datafusion/datasource/src/test_util.rs: ########## @@ -81,6 +83,8 @@ impl FileSource for MockSource { fn file_type(&self) -> &str { "mock" } + + impl_schema_adapter_methods!(); Review Comment: I would personally suggest not adding this macro as I think peoeple will likely just have their IDE fill it out or let the compiler tell them what to do What I suggest we do is change the `with_schema_adapter_factory()` method return signature to return `Result` and provide default implementations in the the trait that return `Error(NotYetImplemented)` That way users won't need to change their implementations if they don't use schema adapters at all ########## datafusion/core/tests/schema_adapter_factory_tests.rs: ########## @@ -0,0 +1,208 @@ +// Licensed to the Apache Software Foundation (ASF) under one Review Comment: I am not sure what additional coverage `datafusion/core/tests/schema_adapter_factory_tests.rs` adds in addition to the integration test in `datafusion/core/tests/integration_tests/schema_adapter_integration_tests.rs` Also if it does add additional coverage can you please include this as part of the other core_integration tests? Each new file in in `datafusion/core/tests` results in a new binary which each take 10s of MB For example, I build this to check and the binary is 57 MB on my machine (it is even more with normal `dev` profile) ```shell $cargo test --profile=ci --test schema_adapter_factory_tests ... Running tests/schema_adapter_factory_tests.rs (target/ci/deps/schema_adapter_factory_tests-b2997559eccc9857) ... $ du -h target/ci/deps/schema_adapter_factory_tests-b2997559eccc9857 57M target/ci/deps/schema_adapter_factory_tests-b2997559eccc9857 ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org